diff --git a/conf/broker.conf b/conf/broker.conf index 1fc578a287f6f..bd37eb9f9a113 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -352,6 +352,10 @@ dispatchThrottlingRatePerTopicInMsg=0 # default message-byte dispatch-throttling dispatchThrottlingRatePerTopicInByte=0 +# Apply dispatch rate limiting on batch message instead individual +# messages with in batch message. (Default is disabled) +dispatchThrottlingOnBatchMessageEnabled=false + # Default number of message dispatching throttling-limit for a subscription. # Using a value of 0, is disabling default message dispatch-throttling. dispatchThrottlingRatePerSubscriptionInMsg=0 @@ -391,6 +395,15 @@ dispatcherMinReadBatchSize=1 # Max number of entries to dispatch for a shared subscription. By default it is 20 entries. dispatcherMaxRoundRobinBatchSize=20 +# The read failure backoff initial time in milliseconds. By default it is 15s. +dispatcherReadFailureBackoffInitialTimeInMs=15000 + +# The read failure backoff max time in milliseconds. By default it is 60s. +dispatcherReadFailureBackoffMaxTimeInMs=60000 + +# The read failure backoff mandatory stop time in milliseconds. By default it is 0s. +dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 + # Precise dispathcer flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/conf/standalone.conf b/conf/standalone.conf index 2e0273a2f3ea0..87d9e058f465c 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -234,6 +234,10 @@ dispatchThrottlingRatePerTopicInMsg=0 # default message-byte dispatch-throttling dispatchThrottlingRatePerTopicInByte=0 +# Apply dispatch rate limiting on batch message instead individual +# messages with in batch message. (Default is disabled) +dispatchThrottlingOnBatchMessageEnabled=false + # Dispatch rate-limiting relative to publish rate. # (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate: # throttle-dispatch-rate = (publish-rate + configured dispatch-rate). @@ -243,6 +247,15 @@ dispatchThrottlingRateRelativeToPublishRate=false # backlog. dispatchThrottlingOnNonBacklogConsumerEnabled=true +# The read failure backoff initial time in milliseconds. By default it is 15s. +dispatcherReadFailureBackoffInitialTimeInMs=15000 + +# The read failure backoff max time in milliseconds. By default it is 60s. +dispatcherReadFailureBackoffMaxTimeInMs=60000 + +# The read failure backoff mandatory stop time in milliseconds. By default it is 0s. +dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 + # Precise dispathcer flow control according to history message number of each entry preciseDispatcherFlowControl=false diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 4af64550fe53b..72ee1a1f9c3ea 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -425,7 +425,11 @@ void markDelete(Position position, Map properties) * @param newReadPosition * the position where to move the cursor */ - void seek(Position newReadPosition); + default void seek(Position newReadPosition) { + seek(newReadPosition, false); + } + + void seek(Position newReadPosition, boolean force); /** * Clear the cursor backlog. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java index 6a0ac2c650c49..7660031d24b02 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCacheImpl.java @@ -158,8 +158,8 @@ public void invalidateEntries(final PositionImpl lastPosition) { Pair removed = entries.removeRange(firstPosition, lastPosition, false); int entriesRemoved = removed.getLeft(); long sizeRemoved = removed.getRight(); - if (log.isDebugEnabled()) { - log.debug("[{}] Invalidated entries up to {} - Entries removed: {} - Size removed: {}", ml.getName(), + if (log.isTraceEnabled()) { + log.trace("[{}] Invalidated entries up to {} - Entries removed: {} - Size removed: {}", ml.getName(), lastPosition, entriesRemoved, sizeRemoved); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 235736da760dc..dd715ff7588d3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2163,18 +2163,16 @@ public void rewind() { } @Override - public void seek(Position newReadPositionInt) { + public void seek(Position newReadPositionInt, boolean force) { checkArgument(newReadPositionInt instanceof PositionImpl); PositionImpl newReadPosition = (PositionImpl) newReadPositionInt; lock.writeLock().lock(); try { - if (newReadPosition.compareTo(markDeletePosition) <= 0) { + if (!force && newReadPosition.compareTo(markDeletePosition) <= 0) { // Make sure the newReadPosition comes after the mark delete position newReadPosition = ledger.getNextValidPosition(markDeletePosition); } - - PositionImpl oldReadPosition = readPosition; readPosition = newReadPosition; } finally { lock.writeLock().unlock(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 6b9c0094a127e..57e1964a2c332 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -175,7 +175,7 @@ public void rewind() { } @Override - public void seek(Position newReadPosition) { + public void seek(Position newReadPosition, boolean force) { } @Override diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 006c287610c93..b48879e79e3a9 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -170,7 +170,7 @@ protected void startBKCluster(String ledgerPath) throws Exception { // Create Bookie Servers (B1, B2, B3) for (int i = 0; i < numBookies; i++) { - if (ledgerPath != "") { + if (!"".equals(ledgerPath)) { ServerConfiguration configuration = newServerConfiguration(ledgerPath + "/ledgers"); startBookie(configuration, ledgerPath + "/ledgers"); }else { @@ -226,7 +226,7 @@ protected ServerConfiguration newServerConfiguration(int port, String zkServers, File[] ledgerDirs, String ledgerRootPath) { ServerConfiguration conf = new ServerConfiguration(baseConf); conf.setBookiePort(port); - if (ledgerRootPath != "") { + if (!"".equals(ledgerRootPath)) { conf.setMetadataServiceUri("zk://" + zkUtil.getZooKeeperConnectString() + ledgerRootPath); }else { conf.setZkServers(zkServers); diff --git a/pom.xml b/pom.xml index 534440739b491..9d54f4956e4bf 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,7 @@ flexible messaging model and an intuitive client API. 0.7.3 2.1.0 3.18.1 - 1.18.20 + 1.18.22 1.3.2 2.3.1 1.2.0 @@ -248,6 +248,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 6.1.6 + 2.3.0 rename-netty-native-libs.sh diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 4c7ee850ad195..a6205c3c580bd 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -699,6 +699,12 @@ public class ServiceConfiguration implements PulsarConfiguration { doc = "Default number of message-bytes dispatching throttling-limit for every topic. \n\n" + "Using a value of 0, is disabling default message-byte dispatch-throttling") private long dispatchThrottlingRatePerTopicInByte = 0; + @FieldContext( + dynamic = true, + category = CATEGORY_POLICIES, + doc = "Apply dispatch rate limiting on batch message instead individual " + + "messages with in batch message. (Default is disabled)") + private boolean dispatchThrottlingOnBatchMessageEnabled = false; @FieldContext( dynamic = true, @@ -779,6 +785,27 @@ public class ServiceConfiguration implements PulsarConfiguration { ) private int dispatcherMinReadBatchSize = 1; + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "The read failure backoff initial time in milliseconds. By default it is 15s." + ) + private int dispatcherReadFailureBackoffInitialTimeInMs = 15000; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "The read failure backoff max time in milliseconds. By default it is 60s." + ) + private int dispatcherReadFailureBackoffMaxTimeInMs = 60000; + + @FieldContext( + dynamic = true, + category = CATEGORY_SERVER, + doc = "The read failure backoff mandatory stop time in milliseconds. By default it is 0s." + ) + private int dispatcherReadFailureBackoffMandatoryStopTimeInMs = 0; + @FieldContext( dynamic = true, category = CATEGORY_SERVER, diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfigurationUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfigurationUtils.java index 8401723ed7a9f..c47b8ca79812d 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfigurationUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfigurationUtils.java @@ -83,7 +83,7 @@ public static String getAppliedAdvertisedAddress(ServiceConfiguration configurat /** * Gets the internal advertised listener for broker-to-broker communication. - * @return an advertised listener + * @return a non-null advertised listener */ public static AdvertisedListener getInternalListener(ServiceConfiguration config) { Map result = MultipleListenerValidator diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/validator/MultipleListenerValidator.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/validator/MultipleListenerValidator.java index 82258c3170e77..ce02da974d52e 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/validator/MultipleListenerValidator.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/validator/MultipleListenerValidator.java @@ -97,7 +97,6 @@ public static Map validateAndAnalysisAdvertisedListe } } String hostPort = String.format("%s:%d", uri.getHost(), uri.getPort()); - reverseMappings.computeIfAbsent(hostPort, k -> Sets.newTreeSet()); Set sets = reverseMappings.computeIfAbsent(hostPort, k -> Sets.newTreeSet()); sets.add(entry.getKey()); if (sets.size() > 1) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java index 14f5003f379b6..acba76ac118be 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java @@ -1412,7 +1412,7 @@ public TransactionBufferClient getTransactionBufferClient() { */ protected String brokerUrl(ServiceConfiguration config) { AdvertisedListener internalListener = ServiceConfigurationUtils.getInternalListener(config); - return internalListener != null && internalListener.getBrokerServiceUrl() != null + return internalListener.getBrokerServiceUrl() != null ? internalListener.getBrokerServiceUrl().toString() : null; } @@ -1425,7 +1425,7 @@ public static String brokerUrl(String host, int port) { */ public String brokerUrlTls(ServiceConfiguration config) { AdvertisedListener internalListener = ServiceConfigurationUtils.getInternalListener(config); - return internalListener != null && internalListener.getBrokerServiceUrlTls() != null + return internalListener.getBrokerServiceUrlTls() != null ? internalListener.getBrokerServiceUrlTls().toString() : null; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java index 9c89be92c4cc5..acb0bb593168d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/ThresholdShedder.java @@ -102,7 +102,7 @@ public Multimap findBundlesForUnloading(final LoadData loadData, log.info( "Attempting to shed load on {}, which has max resource usage above avgUsage and threshold {}%" + " > {}% + {}% -- Offloading at least {} MByte/s of traffic, left throughput {} MByte/s", - broker, currentUsage, avgUsage, threshold, minimumThroughputToOffload / MB, + broker, 100 * currentUsage, 100 * avgUsage, 100 * threshold, minimumThroughputToOffload / MB, (brokerCurrentThroughput - minimumThroughputToOffload) / MB); MutableDouble trafficMarkedToOffload = new MutableDouble(0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index f98cfe59c0813..b53de2f791ba4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -47,10 +47,12 @@ public abstract class AbstractBaseDispatcher implements Dispatcher { protected final Subscription subscription; protected final ServiceConfiguration serviceConfig; + protected final boolean dispatchThrottlingOnBatchMessageEnabled; protected AbstractBaseDispatcher(Subscription subscription, ServiceConfiguration serviceConfig) { this.subscription = subscription; this.serviceConfig = serviceConfig; + this.dispatchThrottlingOnBatchMessageEnabled = serviceConfig.isDispatchThrottlingOnBatchMessageEnabled(); } /** @@ -97,24 +99,26 @@ protected int updateEntryWrapperWithMetadata(EntryWrapper[] entryWrappers, List< * @param sendMessageInfo * an object where the total size in messages and bytes will be returned back to the caller */ - public void filterEntriesForConsumer(List entries, EntryBatchSizes batchSizes, + public int filterEntriesForConsumer(List entries, EntryBatchSizes batchSizes, SendMessageInfo sendMessageInfo, EntryBatchIndexesAcks indexesAcks, ManagedCursor cursor, boolean isReplayRead) { - filterEntriesForConsumer(Optional.empty(), 0, entries, batchSizes, sendMessageInfo, indexesAcks, cursor, + return filterEntriesForConsumer(Optional.empty(), 0, entries, batchSizes, sendMessageInfo, indexesAcks, cursor, isReplayRead); } - public void filterEntriesForConsumer(Optional entryWrapper, int entryWrapperOffset, + public int filterEntriesForConsumer(Optional entryWrapper, int entryWrapperOffset, List entries, EntryBatchSizes batchSizes, SendMessageInfo sendMessageInfo, EntryBatchIndexesAcks indexesAcks, ManagedCursor cursor, boolean isReplayRead) { int totalMessages = 0; long totalBytes = 0; int totalChunkedMessages = 0; + int totalEntries = 0; for (int i = 0, entriesSize = entries.size(); i < entriesSize; i++) { Entry entry = entries.get(i); if (entry == null) { continue; } + totalEntries++; ByteBuf metadataAndPayload = entry.getDataBuffer(); int entryWrapperIndex = i + entryWrapperOffset; MessageMetadata msgMetadata = entryWrapper.isPresent() && entryWrapper.get()[entryWrapperIndex] != null @@ -182,6 +186,7 @@ && trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { sendMessageInfo.setTotalMessages(totalMessages); sendMessageInfo.setTotalBytes(totalBytes); sendMessageInfo.setTotalChunkedMessages(totalChunkedMessages); + return totalEntries; } /** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java index f5dc4b778aa09..76fcfdcfeae02 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesService.java @@ -118,7 +118,7 @@ private CompletableFuture sendTopicPolicyEvent(TopicName topicName, Action } } }); - }) + }) ); } }); @@ -141,6 +141,17 @@ private PulsarEvent getPulsarEvent(TopicName topicName, ActionType actionType, T } private void notifyListener(Message msg) { + // delete policies + if (msg.getValue() == null) { + TopicName topicName = TopicName.get(TopicName.get(msg.getKey()).getPartitionedTopicName()); + if (listeners.get(topicName) != null) { + for (TopicPolicyListener listener : listeners.get(topicName)) { + listener.onUpdate(null); + } + } + return; + } + if (!EventType.TOPIC_POLICY.equals(msg.getValue().getEventType())) { return; } @@ -252,12 +263,11 @@ public void unLoad(NamespaceBundle bundle) { removeOwnedNamespaceBundleAsync(bundle); } - @Override - public boolean test(NamespaceBundle namespaceBundle) { - return true; - } - - }); + @Override + public boolean test(NamespaceBundle namespaceBundle) { + return true; + } + }); } private void initPolicesCache(SystemTopicClient.Reader reader, CompletableFuture future) { @@ -393,7 +403,8 @@ private void fetchTopicPoliciesAsyncAndCloseReader(SystemTopicClient.Reader entries) { int start = 0; long totalMessagesSent = 0; long totalBytesSent = 0; + long totalEntries = 0; int avgBatchSizePerMsg = remainingMessages > 0 ? Math.max(remainingMessages / entries.size(), 1) : 1; int firstAvailableConsumerPermits, currentTotalAvailablePermits; @@ -541,8 +542,9 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(entriesForThisConsumer.size()); - filterEntriesForConsumer(Optional.ofNullable(entryWrappers), start, entriesForThisConsumer, - batchSizes, sendMessageInfo, batchIndexesAcks, cursor, readType == ReadType.Replay); + totalEntries += filterEntriesForConsumer(Optional.ofNullable(entryWrappers), start, + entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, + readType == ReadType.Replay); c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), redeliveryTracker); @@ -571,13 +573,14 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { } // acquire message-dispatch permits for already delivered messages + long permits = dispatchThrottlingOnBatchMessageEnabled ? totalEntries : totalMessagesSent; if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { if (topic.getDispatchRateLimiter().isPresent()) { - topic.getDispatchRateLimiter().get().tryDispatchPermit(totalMessagesSent, totalBytesSent); + topic.getDispatchRateLimiter().get().tryDispatchPermit(permits, totalBytesSent); } if (dispatchRateLimiter.isPresent()) { - dispatchRateLimiter.get().tryDispatchPermit(totalMessagesSent, totalBytesSent); + dispatchRateLimiter.get().tryDispatchPermit(permits, totalBytesSent); } } 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 fa7ac03a18221..c2b4b34e057dd 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 @@ -65,8 +65,7 @@ public class PersistentDispatcherSingleActiveConsumer extends AbstractDispatcher protected volatile boolean havePendingRead = false; protected volatile int readBatchSize; - protected final Backoff readFailureBackoff = new Backoff(15, TimeUnit.SECONDS, - 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + protected final Backoff readFailureBackoff; private volatile ScheduledFuture readOnActiveConsumerTask = null; private final RedeliveryTracker redeliveryTracker; @@ -80,6 +79,10 @@ public PersistentDispatcherSingleActiveConsumer(ManagedCursor cursor, SubType su : ""/* NonDurableCursor doesn't have name */); this.cursor = cursor; this.readBatchSize = serviceConfig.getDispatcherMaxReadBatchSize(); + this.readFailureBackoff = new Backoff(serviceConfig.getDispatcherReadFailureBackoffInitialTimeInMs(), + TimeUnit.MILLISECONDS, serviceConfig.getDispatcherReadFailureBackoffMaxTimeInMs(), + TimeUnit.MILLISECONDS, serviceConfig.getDispatcherReadFailureBackoffMandatoryStopTimeInMs(), + TimeUnit.MILLISECONDS); this.redeliveryTracker = RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED; this.initializeDispatchRateLimiterIfNeeded(Optional.empty()); } @@ -213,15 +216,16 @@ protected void dispatchEntriesToConsumer(Consumer currentConsumer, List e redeliveryTracker) .addListener(future -> { if (future.isSuccess()) { + int permits = dispatchThrottlingOnBatchMessageEnabled ? entries.size() + : sendMessageInfo.getTotalMessages(); // acquire message-dispatch permits for already delivered messages if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { if (topic.getDispatchRateLimiter().isPresent()) { - topic.getDispatchRateLimiter().get().tryDispatchPermit(sendMessageInfo.getTotalMessages(), + topic.getDispatchRateLimiter().get().tryDispatchPermit(permits, sendMessageInfo.getTotalBytes()); } - dispatchRateLimiter.ifPresent(rateLimiter -> - rateLimiter.tryDispatchPermit(sendMessageInfo.getTotalMessages(), + rateLimiter.tryDispatchPermit(permits, sendMessageInfo.getTotalBytes())); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index f3bcbf2b2e9c9..6f4c4ebb5f7b6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -155,6 +155,7 @@ protected Map> initialValue() throws Exception { protected void sendMessagesToConsumers(ReadType readType, List entries) { long totalMessagesSent = 0; long totalBytesSent = 0; + long totalEntries = 0; int entriesCount = entries.size(); // Trigger read more messages @@ -229,8 +230,8 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(messagesForC); EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(messagesForC); - filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, batchIndexesAcks, cursor, - readType == ReadType.Replay); + totalEntries += filterEntriesForConsumer(entriesWithSameKey, batchSizes, sendMessageInfo, + batchIndexesAcks, cursor, readType == ReadType.Replay); consumer.sendMessages(entriesWithSameKey, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), @@ -252,12 +253,13 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { // acquire message-dispatch permits for already delivered messages if (serviceConfig.isDispatchThrottlingOnNonBacklogConsumerEnabled() || !cursor.isActive()) { + long permits = dispatchThrottlingOnBatchMessageEnabled ? totalEntries : totalMessagesSent; if (topic.getDispatchRateLimiter().isPresent()) { - topic.getDispatchRateLimiter().get().tryDispatchPermit(totalMessagesSent, totalBytesSent); + topic.getDispatchRateLimiter().get().tryDispatchPermit(permits, totalBytesSent); } if (dispatchRateLimiter.isPresent()) { - dispatchRateLimiter.get().tryDispatchPermit(totalMessagesSent, totalBytesSent); + dispatchRateLimiter.get().tryDispatchPermit(permits, totalBytesSent); } } 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 a1cdbd997c529..9f2a56703bb15 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 @@ -1023,6 +1023,11 @@ public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { log.debug("[{}][{}] Error deleting cursor for subscription", topic, subscriptionName, exception); } + if (exception instanceof ManagedLedgerException.ManagedLedgerNotFoundException) { + unsubscribeFuture.complete(null); + lastActive = System.nanoTime(); + return; + } unsubscribeFuture.completeExceptionally(new PersistenceException(exception)); } }, null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java index b60f4ba21dc29..fb88878d8c826 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java @@ -120,12 +120,23 @@ public CompletableFuture closeAsync() { cursor.asyncClose(new AsyncCallbacks.CloseCallback() { @Override public void closeComplete(Object ctx) { - try { - managedLedger.close(); - } catch (Exception e) { - completableFuture.completeExceptionally(e); - } - completableFuture.complete(null); + managedLedger.asyncClose(new AsyncCallbacks.CloseCallback() { + + @Override + public void closeComplete(Object ctx) { + if (log.isDebugEnabled()) { + log.debug("[{}][{}] MLPendingAckStore closed successfully!", managedLedger.getName(), ctx); + } + completableFuture.complete(null); + } + + @Override + public void closeFailed(ManagedLedgerException exception, Object ctx) { + log.error("[{}][{}] MLPendingAckStore closed failed,exception={}", managedLedger.getName(), + ctx, exception); + completableFuture.completeExceptionally(exception); + } + }, ctx); } @Override 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 131341318e0f9..4bc166412d6d6 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 @@ -122,7 +122,12 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, return readEntries(context.ledger, startPoint, endPoint) .thenAccept((entries) -> { Entry lastEntry = entries.get(entries.size() - 1); - cursor.seek(lastEntry.getPosition().getNext()); + // The compaction task depends on the last snapshot and the incremental + // entries to build the new snapshot. So for the compaction cursor, we + // need to force seek the read position to ensure the compactor can read + // the complete last snapshot because of the compactor will read the data + // before the compaction cursor mark delete position + cursor.seek(lastEntry.getPosition().getNext(), true); callback.readEntriesComplete(entries, consumer); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MaxUnackedMessagesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MaxUnackedMessagesTest.java index d6ebdcc66f521..dc0f26c9b5985 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MaxUnackedMessagesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/MaxUnackedMessagesTest.java @@ -207,6 +207,13 @@ public void testMaxUnackedMessagesOnConsumerAppliedApi() throws Exception { Integer max = admin.topics().getMaxUnackedMessagesOnConsumer(topicName, true); assertEquals(max.intValue(), pulsar.getConfiguration().getMaxUnackedMessagesPerConsumer()); + admin.namespaces().setMaxUnackedMessagesPerConsumer(myNamespace, 15); + Awaitility.await().untilAsserted(() + -> assertEquals(admin.namespaces().getMaxUnackedMessagesPerConsumer(myNamespace).intValue(), 15)); + admin.namespaces().removeMaxUnackedMessagesPerConsumer(myNamespace); + Awaitility.await().untilAsserted(() + -> assertEquals(admin.namespaces().getMaxUnackedMessagesPerConsumer(myNamespace), null)); + admin.namespaces().setMaxUnackedMessagesPerConsumer(myNamespace, 10); Awaitility.await().untilAsserted(() -> assertNotNull(admin.namespaces().getMaxUnackedMessagesPerConsumer(myNamespace))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 59494fb799d0c..0383333cb6002 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -2643,4 +2643,27 @@ public void testDoNotCreateSystemTopicForHeartbeatNamespace() { }); } + @Test + public void testLoopCreateAndDeleteTopicPolicies() throws Exception { + final String topic = testTopic + UUID.randomUUID(); + + int n = 0; + while (n < 2) { + n++; + pulsarClient.newProducer().topic(topic).create().close(); + Awaitility.await().untilAsserted(() -> { + Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNull(); + }); + + admin.topics().setMaxConsumersPerSubscription(topic, 1); + Awaitility.await().untilAsserted(() -> { + Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNotNull(); + }); + + admin.topics().delete(topic); + Awaitility.await().untilAsserted(() -> { + Assertions.assertThat(pulsar.getTopicPoliciesService().getTopicPolicies(TopicName.get(topic))).isNull(); + }); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java similarity index 99% rename from pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java index 91260098a3c31..fda8693dd8478 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMesgsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/RGUsageMTAggrWaitForAllMsgsTest.java @@ -20,7 +20,6 @@ import com.google.common.collect.Sets; import io.prometheus.client.Summary; -import java.util.Collections; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; import org.apache.pulsar.broker.resourcegroup.ResourceGroupService.ResourceGroupUsageStatsType; @@ -59,7 +58,7 @@ // The tenants and namespaces in those topics are associated with a set of resource-groups (RGs). // After sending/receiving all the messages, traffic usage statistics, and Prometheus-metrics // are verified on the RGs. -public class RGUsageMTAggrWaitForAllMesgsTest extends ProducerConsumerBase { +public class RGUsageMTAggrWaitForAllMsgsTest extends ProducerConsumerBase { @BeforeClass @Override protected void setup() throws Exception { @@ -350,13 +349,12 @@ private boolean tenantRGEqualsNamespaceRG(String[] topicStrings) throws PulsarCl } } if ((numEqualRGs + numUnEqualRGs != numTopics) || (numEqualRGs > 0 && numUnEqualRGs > 0)) { - String errMesg = String.format("Found {} topics with equal RGs and {} with unequal, on {} topics", + String errMesg = String.format("Found %s topics with equal RGs and %s with unequal, on %s topics", numEqualRGs, numUnEqualRGs, numTopics); throw new PulsarClientException(errMesg); - } else if (numEqualRGs == numTopics) { - return true; + } else { + return numEqualRGs == numTopics; } - return false; } private void registerTenantsAndNamespaces(String[] topicStrings) throws Exception { @@ -788,7 +786,7 @@ private void verifyRGMetrics(String[] topicStrings, Assert.assertNotEquals(ninetethPercentileValue, 0); } - private static final Logger log = LoggerFactory.getLogger(RGUsageMTAggrWaitForAllMesgsTest.class); + private static final Logger log = LoggerFactory.getLogger(RGUsageMTAggrWaitForAllMsgsTest.class); // Empirically, there appears to be a 45-byte overhead for metadata, imposed by Pulsar runtime. private static final int PER_MESSAGE_METADATA_OHEAD = 45; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java index a89d759e7ab29..08c1f6163f253 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupUsageAggregationTest.java @@ -130,8 +130,8 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { .subscriptionType(SubscriptionType.Shared) .subscribe(); } catch (PulsarClientException p) { - final String errMesg = String.format("Got exception while building consumer: ex={}", p.getMessage()); - Assert.assertTrue(false, errMesg); + final String errMsg = String.format("Got exception while building consumer: ex=%s", p.getMessage()); + Assert.fail(errMsg); } final TopicName myTopic = TopicName.get(topicString); @@ -146,16 +146,15 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { int recvdNumBytes = 0; int recvdNumMsgs = 0; for (int ix = 0; ix < NumMessagesToSend; ix++) { - MessageId prodMesgId = null; byte[] mesg; try { - mesg = String.format("Hi, ix={}", ix).getBytes(); + mesg = String.format("Hi, ix=%s", ix).getBytes(); producer.send(mesg); sentNumBytes += mesg.length; sentNumMsgs++; } catch (PulsarClientException p) { - final String errMesg = String.format("Got exception while sending {}-th time: ex={}", ix, p.getMessage()); - Assert.assertTrue(false, errMesg); + final String errMsg = String.format("Got exception while sending %s-th time: ex=%s", ix, p.getMessage()); + Assert.fail(errMsg); } } producer.close(); @@ -169,9 +168,9 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { message = consumer.receive(); recvdNumBytes += message.getValue().length; } catch (PulsarClientException p) { - final String errMesg = String.format("Got exception in while receiving {}-th mesg at consumer: ex={}", + final String errMesg = String.format("Got exception in while receiving %s-th mesg at consumer: ex=%s", recvdNumMsgs, p.getMessage()); - Assert.assertTrue(false, errMesg); + Assert.fail(errMesg); } // log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); recvdNumMsgs++; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java index 7332307a612d2..e8182d77a4900 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceUsageTransportManagerTest.java @@ -26,7 +26,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.ClusterDataImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java index 9f098935efabf..123e366773e26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java @@ -27,7 +27,6 @@ import com.google.common.base.Splitter; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.jsonwebtoken.SignatureAlgorithm; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -47,15 +46,12 @@ import java.util.Random; import java.util.TreeMap; import java.util.UUID; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.crypto.SecretKey; import javax.naming.AuthenticationException; import lombok.Cleanup; -import org.apache.bookkeeper.client.BookKeeper; import org.apache.commons.io.IOUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -74,7 +70,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.compaction.Compactor; -import org.apache.pulsar.compaction.TwoPhaseCompactor; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -1186,8 +1181,6 @@ public void testCompaction() throws Exception { .value(data) .send(); } - ScheduledExecutorService compactionScheduler = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat("compactor").setDaemon(true).build()); Compactor compactor = pulsar.getCompactor(true); compactor.compact(topicName).get(); statsOut = new ByteArrayOutputStream(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java index f605028a1a70d..05e7363ea37d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageDispatchThrottlingTest.java @@ -540,6 +540,84 @@ public void testRateLimitingMultipleConsumers() throws Exception { log.info("-- Exiting {} test --", methodName); } + @Test + public void testRateLimitingWithBatchMsgEnabled() throws Exception { + log.info("-- Starting {} test --", methodName); + + conf.setDispatchThrottlingOnBatchMessageEnabled(true); + + final String namespace = "my-property/throttling_ns"; + final String topicName = "persistent://" + namespace + "/throttlingMultipleConsumers"; + + final int messageRate = 5; + DispatchRate dispatchRate = DispatchRate.builder().dispatchThrottlingRateInMsg(messageRate) + .dispatchThrottlingRateInByte(-1).ratePeriodInSecond(360).build(); + admin.namespaces().createNamespace(namespace, Sets.newHashSet("test")); + admin.namespaces().setDispatchRate(namespace, dispatchRate); + + final int messagesPerBatch = 100; + final int numProducedMessages = messageRate * messagesPerBatch; + // create producer and topic + Producer producer = pulsarClient.newProducer().topic(topicName).enableBatching(true) + .batchingMaxPublishDelay(1, TimeUnit.SECONDS).batchingMaxMessages(messagesPerBatch).create(); + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topicName).get(); + boolean isMessageRateUpdate = false; + int retry = 5; + for (int i = 0; i < retry; i++) { + if (topic.getDispatchRateLimiter().get().getDispatchRateOnMsg() > 0) { + isMessageRateUpdate = true; + break; + } else { + if (i != retry - 1) { + Thread.sleep(100); + } + } + } + Assert.assertTrue(isMessageRateUpdate); + Assert.assertEquals(admin.namespaces().getDispatchRate(namespace), dispatchRate); + + final AtomicInteger totalReceived = new AtomicInteger(0); + + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer().topic(topicName) + .subscriptionName("my-subscriber-name").subscriptionType(SubscriptionType.Shared) + .messageListener((c1, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + String receivedMessage = new String(msg.getData()); + log.debug("Received message [{}] in the listener", receivedMessage); + totalReceived.incrementAndGet(); + }); + Consumer consumer1 = consumerBuilder.subscribe(); + Consumer consumer2 = consumerBuilder.subscribe(); + Consumer consumer3 = consumerBuilder.subscribe(); + Consumer consumer4 = consumerBuilder.subscribe(); + Consumer consumer5 = consumerBuilder.subscribe(); + + // deactive cursors + deactiveCursors((ManagedLedgerImpl) topic.getManagedLedger()); + + // Asynchronously produce messages + CountDownLatch latch = new CountDownLatch(numProducedMessages); + for (int i = 0; i < numProducedMessages; i++) { + final String message = "my-message-" + i; + producer.sendAsync(message.getBytes()).thenAccept(__ -> latch.countDown()); + } + + latch.await(); + + Awaitility.await().atMost(10, TimeUnit.SECONDS).until(() -> totalReceived.get() == numProducedMessages); + + // consumer should not have received all published message due to message-rate throttling + Assert.assertEquals(totalReceived.get(), numProducedMessages); + + consumer1.close(); + consumer2.close(); + consumer3.close(); + consumer4.close(); + consumer5.close(); + producer.close(); + log.info("-- Exiting {} test --", methodName); + } + @Test(dataProvider = "subscriptions", timeOut = 5000) public void testClusterRateLimitingConfiguration(SubscriptionType subscription) throws Exception { log.info("-- Starting {} test --", methodName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java index fc84a62dac1cd..48d20c677c181 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/RetryTopicTest.java @@ -18,18 +18,19 @@ */ package org.apache.pulsar.client.api; -import lombok.Cleanup; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; +import com.google.common.collect.Sets; +import java.util.Set; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.pulsar.client.util.RetryMessageUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.concurrent.TimeUnit; - -import static org.testng.Assert.assertNull; - @Test(groups = "broker-api") public class RetryTopicTest extends ProducerConsumerBase { @@ -119,6 +120,100 @@ public void testRetryTopic() throws Exception { checkConsumer.close(); } + @Test + public void testRetryTopicProperties() throws Exception { + final String topic = "persistent://my-property/my-ns/retry-topic"; + + final int maxRedeliveryCount = 3; + + final int sendMessages = 10; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .enableRetry(true) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + @Cleanup + PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0); + Consumer deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES) + .topic("persistent://my-property/my-ns/retry-topic-my-subscription-DLQ") + .subscriptionName("my-subscription") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + Set originMessageIds = Sets.newHashSet(); + for (int i = 0; i < sendMessages; i++) { + MessageId msgId = producer.send(String.format("Hello Pulsar [%d]", i).getBytes()); + originMessageIds.add(msgId.toString()); + } + + producer.close(); + + int totalReceived = 0; + Set retryMessageIds = Sets.newHashSet(); + do { + Message message = consumer.receive(); + log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + // retry message + if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { + // check the REAL_TOPIC property + assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + retryMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); + } + consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); + totalReceived++; + } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); + + // check the REAL_TOPIC property + assertEquals(retryMessageIds, originMessageIds); + + int totalInDeadLetter = 0; + Set deadLetterMessageIds = Sets.newHashSet(); + do { + Message message = deadLetterConsumer.receive(); + log.info("dead letter consumer received message : {} {}", message.getMessageId(), + new String(message.getData())); + // dead letter message + if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { + // check the REAL_TOPIC property + assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + deadLetterMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); + } + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } while (totalInDeadLetter < sendMessages); + + assertEquals(deadLetterMessageIds, originMessageIds); + + deadLetterConsumer.close(); + consumer.close(); + + Consumer checkConsumer = this.pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Message checkMessage = checkConsumer.receive(3, TimeUnit.SECONDS); + if (checkMessage != null) { + log.info("check consumer received message : {} {}", checkMessage.getMessageId(), + new String(checkMessage.getData())); + } + assertNull(checkMessage); + + checkConsumer.close(); + } + //Issue 9327: do compatibility check in case of the default retry and dead letter topic name changed @Test public void testRetryTopicNameForCompatibility () throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java index 3d410884e3042..2dd6f8aa4dfeb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.compaction; +import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.google.common.collect.Sets; @@ -437,4 +438,63 @@ public void testLastMessageIdForCompactedLedger() throws Exception { reader.readNext(); Assert.assertFalse(reader.hasMessageAvailable()); } + + @Test + public void testDoNotLossTheLastCompactedLedgerData() throws Exception { + String topic = "persistent://my-property/use/my-ns/testDoNotLossTheLastCompactedLedgerData-" + + UUID.randomUUID(); + final int numMessages = 2000; + final int keys = 200; + final String msg = "Test"; + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .blockIfQueueFull(true) + .maxPendingMessages(numMessages) + .enableBatching(false) + .create(); + CompletableFuture lastMessage = null; + for (int i = 0; i < numMessages; ++i) { + lastMessage = producer.newMessage().key(i % keys + "").value(msg).sendAsync(); + } + producer.flush(); + lastMessage.join(); + admin.topics().triggerCompaction(topic); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topic); + Assert.assertNotEquals(stats.compactedLedger.ledgerId, -1); + Assert.assertEquals(stats.compactedLedger.entries, keys); + Assert.assertEquals(admin.topics().getStats(topic) + .getSubscriptions().get(COMPACTION_SUBSCRIPTION).getConsumers().size(), 0); + }); + admin.topics().unload(topic); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topic); + Assert.assertEquals(stats.ledgers.size(), 1); + Assert.assertEquals(admin.topics().getStats(topic) + .getSubscriptions().get(COMPACTION_SUBSCRIPTION).getConsumers().size(), 0); + }); + admin.topics().unload(topic); + // Send one more key to and then to trigger the compaction + producer.newMessage().key(keys + "").value(msg).send(); + admin.topics().triggerCompaction(topic); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topic); + Assert.assertEquals(stats.compactedLedger.entries, keys + 1); + }); + + // Make sure the reader can get all data from the compacted ledger and original ledger. + Reader reader = pulsarClient.newReader(Schema.STRING) + .topic(topic) + .startMessageId(MessageId.earliest) + .readCompacted(true) + .create(); + int received = 0; + while (reader.hasMessageAvailable()) { + reader.readNext(); + received++; + } + Assert.assertEquals(received, keys + 1); + reader.close(); + producer.close(); + } } diff --git a/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java b/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java index 6323f172d1175..319e25db79766 100644 --- a/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java +++ b/pulsar-client-auth-athenz/src/test/java/org/apache/pulsar/client/impl/auth/AuthenticationAthenzTest.java @@ -103,7 +103,7 @@ public void testGetAuthData() throws Exception { int count = 0; for (Map.Entry header : auth.getAuthData().getHttpHeaders()) { - if (header.getKey() == ZTSClient.getHeader()) { + if (header.getKey().equals(ZTSClient.getHeader())) { com.yahoo.athenz.auth.token.RoleToken roleTokenFromHeader = new com.yahoo.athenz.auth.token.RoleToken( header.getValue()); assertEquals(roleTokenFromHeader.getPrincipal(), String.format("%s.%s", TENANT_DOMAIN, TENANT_SERVICE)); diff --git a/pulsar-client-cpp/lib/ClientConnection.cc b/pulsar-client-cpp/lib/ClientConnection.cc index 4b8a3dc4a420f..e1ca324640d8e 100644 --- a/pulsar-client-cpp/lib/ClientConnection.cc +++ b/pulsar-client-cpp/lib/ClientConnection.cc @@ -162,11 +162,11 @@ ClientConnection::ClientConnection(const std::string& logicalAddress, const std: resolver_(executor_->createTcpResolver()), socket_(executor_->createSocket()), #if BOOST_VERSION >= 107000 - strand_(boost::asio::make_strand(executor_->io_service_->get_executor())), + strand_(boost::asio::make_strand(executor_->getIOService().get_executor())), #elif BOOST_VERSION >= 106600 - strand_(executor_->io_service_->get_executor()), + strand_(executor_->getIOService().get_executor()), #else - strand_(*(executor_->io_service_)), + strand_(executor_->getIOService()), #endif logicalAddress_(logicalAddress), physicalAddress_(physicalAddress), @@ -183,7 +183,7 @@ ClientConnection::ClientConnection(const std::string& logicalAddress, const std: #if BOOST_VERSION >= 105400 boost::asio::ssl::context ctx(boost::asio::ssl::context::tlsv12_client); #else - boost::asio::ssl::context ctx(*executor_->io_service_, boost::asio::ssl::context::tlsv1_client); + boost::asio::ssl::context ctx(executor_->getIOService(), boost::asio::ssl::context::tlsv1_client); #endif Url serviceUrl; Url::parse(physicalAddress, serviceUrl); @@ -240,7 +240,7 @@ ClientConnection::ClientConnection(const std::string& logicalAddress, const std: } } - tlsSocket_ = executor_->createTlsSocket(socket_, ctx); + tlsSocket_ = ExecutorService::createTlsSocket(socket_, ctx); LOG_DEBUG("TLS SNI Host: " << serviceUrl.host()); if (!SSL_set_tlsext_host_name(tlsSocket_->native_handle(), serviceUrl.host().c_str())) { diff --git a/pulsar-client-cpp/lib/ExecutorService.cc b/pulsar-client-cpp/lib/ExecutorService.cc index 4db31124fc0e6..9cfbd82881d3e 100644 --- a/pulsar-client-cpp/lib/ExecutorService.cc +++ b/pulsar-client-cpp/lib/ExecutorService.cc @@ -27,22 +27,40 @@ DECLARE_LOG_OBJECT() namespace pulsar { -ExecutorService::ExecutorService() - : io_service_(new boost::asio::io_service()), - work_(new BackgroundWork(*io_service_)), - worker_(std::bind(&ExecutorService::startWorker, this, io_service_)) {} +ExecutorService::ExecutorService() {} ExecutorService::~ExecutorService() { close(); } -void ExecutorService::startWorker(std::shared_ptr io_service) { io_service_->run(); } +void ExecutorService::start() { + auto self = shared_from_this(); + std::thread t{[self] { + if (self->isClosed()) { + return; + } + boost::system::error_code ec; + self->getIOService().run(ec); + if (ec) { + LOG_ERROR("Failed to run io_service: " << ec.message()); + } + }}; + t.detach(); +} + +ExecutorServicePtr ExecutorService::create() { + // make_shared cannot access the private constructor, so we need to expose the private constructor via a + // derived class. + struct ExecutorServiceImpl : public ExecutorService {}; + + auto executor = std::make_shared(); + executor->start(); + return std::static_pointer_cast(executor); +} /* * factory method of boost::asio::ip::tcp::socket associated with io_service_ instance * @ returns shared_ptr to this socket */ -SocketPtr ExecutorService::createSocket() { - return SocketPtr(new boost::asio::ip::tcp::socket(*io_service_)); -} +SocketPtr ExecutorService::createSocket() { return SocketPtr(new boost::asio::ip::tcp::socket(io_service_)); } TlsSocketPtr ExecutorService::createTlsSocket(SocketPtr &socket, boost::asio::ssl::context &ctx) { return std::shared_ptr >( @@ -54,11 +72,11 @@ TlsSocketPtr ExecutorService::createTlsSocket(SocketPtr &socket, boost::asio::ss * @returns shraed_ptr to resolver object */ TcpResolverPtr ExecutorService::createTcpResolver() { - return TcpResolverPtr(new boost::asio::ip::tcp::resolver(*io_service_)); + return TcpResolverPtr(new boost::asio::ip::tcp::resolver(io_service_)); } DeadlineTimerPtr ExecutorService::createDeadlineTimer() { - return DeadlineTimerPtr(new boost::asio::deadline_timer(*io_service_)); + return DeadlineTimerPtr(new boost::asio::deadline_timer(io_service_)); } void ExecutorService::close() { @@ -67,21 +85,10 @@ void ExecutorService::close() { return; } - io_service_->stop(); - work_.reset(); - // Detach the worker thread instead of join to avoid potential deadlock - if (worker_.joinable()) { - try { - worker_.detach(); - } catch (const std::system_error &e) { - // This condition will happen if we're forking the process, therefore the thread was not ported to - // the child side of the fork and the detach would be failing. - LOG_DEBUG("Failed to detach thread: " << e.what()); - } - } + io_service_.stop(); } -void ExecutorService::postWork(std::function task) { io_service_->post(task); } +void ExecutorService::postWork(std::function task) { io_service_.post(task); } ///////////////////// @@ -93,7 +100,7 @@ ExecutorServicePtr ExecutorServiceProvider::get() { int idx = executorIdx_++ % executors_.size(); if (!executors_[idx]) { - executors_[idx] = std::make_shared(); + executors_[idx] = ExecutorService::create(); } return executors_[idx]; diff --git a/pulsar-client-cpp/lib/ExecutorService.h b/pulsar-client-cpp/lib/ExecutorService.h index 6746936190566..6b0909194b7d9 100644 --- a/pulsar-client-cpp/lib/ExecutorService.h +++ b/pulsar-client-cpp/lib/ExecutorService.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -34,51 +33,48 @@ typedef std::shared_ptr SocketPtr; typedef std::shared_ptr > TlsSocketPtr; typedef std::shared_ptr TcpResolverPtr; typedef std::shared_ptr DeadlineTimerPtr; -class PULSAR_PUBLIC ExecutorService : private boost::noncopyable { - friend class ClientConnection; - +class PULSAR_PUBLIC ExecutorService : public std::enable_shared_from_this { public: - ExecutorService(); + using IOService = boost::asio::io_service; + using SharedPtr = std::shared_ptr; + + static SharedPtr create(); ~ExecutorService(); + ExecutorService(const ExecutorService &) = delete; + ExecutorService &operator=(const ExecutorService &) = delete; + SocketPtr createSocket(); - TlsSocketPtr createTlsSocket(SocketPtr &socket, boost::asio::ssl::context &ctx); + static TlsSocketPtr createTlsSocket(SocketPtr &socket, boost::asio::ssl::context &ctx); TcpResolverPtr createTcpResolver(); DeadlineTimerPtr createDeadlineTimer(); void postWork(std::function task); + void close(); - boost::asio::io_service &getIOService() { return *io_service_; } + IOService &getIOService() { return io_service_; } + bool isClosed() const noexcept { return closed_; } private: - /* - * only called once and within lock so no need to worry about thread-safety - */ - void startWorker(std::shared_ptr io_service); - /* * io_service is our interface to os, io object schedule async ops on this object */ - std::shared_ptr io_service_; + IOService io_service_; /* * work will not let io_service.run() return even after it has finished work * it will keep it running in the background so we don't have to take care of it */ - typedef boost::asio::io_service::work BackgroundWork; - std::unique_ptr work_; - - /* - * worker thread which runs until work object is destroyed, it's running io_service::run in - * background invoking async handlers as they are finished and result is available from - * io_service - */ - std::thread worker_; + IOService::work work_{io_service_}; std::atomic_bool closed_{false}; + + ExecutorService(); + + void start(); }; -typedef std::shared_ptr ExecutorServicePtr; +using ExecutorServicePtr = ExecutorService::SharedPtr; class PULSAR_PUBLIC ExecutorServiceProvider { public: diff --git a/pulsar-client-cpp/tests/PeriodicTaskTest.cc b/pulsar-client-cpp/tests/PeriodicTaskTest.cc index 11c1c62ec3f2b..2c1da70e80e3c 100644 --- a/pulsar-client-cpp/tests/PeriodicTaskTest.cc +++ b/pulsar-client-cpp/tests/PeriodicTaskTest.cc @@ -29,11 +29,11 @@ DECLARE_LOG_OBJECT() using namespace pulsar; TEST(PeriodicTaskTest, testCountdownTask) { - ExecutorService executor; + auto executor = ExecutorService::create(); std::atomic_int count{5}; - auto task = std::make_shared(executor.getIOService(), 200); + auto task = std::make_shared(executor->getIOService(), 200); task->setCallback([task, &count](const PeriodicTask::ErrorCode& ec) { if (--count <= 0) { task->stop(); @@ -56,13 +56,13 @@ TEST(PeriodicTaskTest, testCountdownTask) { ASSERT_EQ(count.load(), 0); task->stop(); - executor.close(); + executor->close(); } TEST(PeriodicTaskTest, testNegativePeriod) { - ExecutorService executor; + auto executor = ExecutorService::create(); - auto task = std::make_shared(executor.getIOService(), -1); + auto task = std::make_shared(executor->getIOService(), -1); std::atomic_bool callbackTriggered{false}; task->setCallback([&callbackTriggered](const PeriodicTask::ErrorCode& ec) { callbackTriggered = true; }); @@ -71,5 +71,5 @@ TEST(PeriodicTaskTest, testNegativePeriod) { ASSERT_EQ(callbackTriggered.load(), false); task->stop(); - executor.close(); + executor->close(); } 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 60fb6acf1d9a6..b311417f22b10 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 @@ -652,6 +652,9 @@ public void namespaces() throws Exception { namespaces.run(split("set-max-unacked-messages-per-consumer myprop/clust/ns1 -c 3")); verify(mockNamespaces).setMaxUnackedMessagesPerConsumer("myprop/clust/ns1", 3); + namespaces.run(split("remove-max-unacked-messages-per-consumer myprop/clust/ns1")); + verify(mockNamespaces).removeMaxUnackedMessagesPerConsumer("myprop/clust/ns1"); + mockNamespaces = mock(Namespaces.class); when(admin.namespaces()).thenReturn(mockNamespaces); namespaces = new CmdNamespaces(() -> admin); diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java index 2f1cf72db813a..6a1428ac15ab1 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CliCommand.java @@ -188,7 +188,11 @@ void print(Map items) { void print(T item) { try { - System.out.println(writer.writeValueAsString(item)); + if (item instanceof String) { + System.out.println(item); + } else { + System.out.println(writer.writeValueAsString(item)); + } } catch (Exception e) { throw new RuntimeException(e); } 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 c642288383aeb..50baa0276f1c2 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 @@ -1590,6 +1590,18 @@ void run() throws PulsarAdminException { } } + @Parameters(commandDescription = "Remove maxUnackedMessagesPerConsumer for a namespace") + private class RemoveMaxUnackedMessagesPerConsumer extends CliCommand { + @Parameter(description = "tenant/namespace", required = true) + private java.util.List params; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + getAdmin().namespaces().removeMaxUnackedMessagesPerConsumer(namespace); + } + } + @Parameters(commandDescription = "Get maxUnackedMessagesPerSubscription for a namespace") private class GetMaxUnackedMessagesPerSubscription extends CliCommand { @Parameter(description = "tenant/namespace", required = true) @@ -2438,6 +2450,7 @@ public CmdNamespaces(Supplier admin) { jcommander.addCommand("get-max-unacked-messages-per-consumer", new GetMaxUnackedMessagesPerConsumer()); jcommander.addCommand("set-max-unacked-messages-per-consumer", new SetMaxUnackedMessagesPerConsumer()); + jcommander.addCommand("remove-max-unacked-messages-per-consumer", new RemoveMaxUnackedMessagesPerConsumer()); jcommander.addCommand("get-compaction-threshold", new GetCompactionThreshold()); jcommander.addCommand("set-compaction-threshold", new SetCompactionThreshold()); diff --git a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java index 9dee2b1414ad7..fb53600db00f4 100644 --- a/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java +++ b/pulsar-client-tools/src/test/java/org/apache/pulsar/admin/cli/TestCmdTopics.java @@ -18,8 +18,19 @@ */ package org.apache.pulsar.admin.cli; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Lists; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; import java.util.ArrayList; import java.util.List; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats.LedgerInfo; import org.testng.Assert; @@ -54,4 +65,27 @@ public void testFindFirstLedgerWithinThreshold() throws Exception { Assert.assertEquals(CmdTopics.findFirstLedgerWithinThreshold(ledgers, 5000), new MessageIdImpl(1, 0, -1)); } + + @Test + public void testListCmd() throws Exception { + List topicList = Lists.newArrayList("persistent://public/default/t1", "persistent://public/default/t2", + "persistent://public/default/t3"); + + Topics topics = mock(Topics.class); + doReturn(topicList).when(topics).getList(anyString(), any()); + + PulsarAdmin admin = mock(PulsarAdmin.class); + when(admin.topics()).thenReturn(topics); + + CmdTopics cmd = new CmdTopics(() -> admin); + + PrintStream defaultSystemOut = System.out; + try (ByteArrayOutputStream out = new ByteArrayOutputStream(); PrintStream ps = new PrintStream(out)) { + System.setOut(ps); + cmd.run("list public/default".split("\\s+")); + Assert.assertEquals(out.toString(), String.join("\n", topicList) + "\n"); + } finally { + System.setOut(defaultSystemOut); + } + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 45505e48a205e..b5cdc7f505661 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -717,13 +717,13 @@ protected void onNegativeAcksSend(Set messageIds) { protected void onAckTimeoutSend(Set messageIds) { if (interceptors != null) { - interceptors. onAckTimeoutSend(this, messageIds); + interceptors.onAckTimeoutSend(this, messageIds); } } protected void onPartitionsChange(String topicName, int partitions) { if (interceptors != null) { - interceptors. onPartitionsChange(topicName, partitions); + interceptors.onPartitionsChange(topicName, partitions); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index b29d6c73deff6..5108b8cca0e77 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -649,8 +649,8 @@ private SortedMap getPropertiesMap(Message message, String or if (message.getProperties() != null) { propertiesMap.putAll(message.getProperties()); } - propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC, originTopicNameStr); - propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID, originMessageIdStr); + propertiesMap.putIfAbsent(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC, originTopicNameStr); + propertiesMap.putIfAbsent(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID, originMessageIdStr); return propertiesMap; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index bbdd7864987bf..16cfa0c11ec19 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -28,7 +28,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import static org.apache.pulsar.client.impl.UnAckedMessageTracker.addChunkedMessageIdsAndRemoveFromSequnceMap; +import static org.apache.pulsar.client.impl.UnAckedMessageTracker.addChunkedMessageIdsAndRemoveFromSequenceMap; class NegativeAcksTracker { @@ -63,7 +63,7 @@ private synchronized void triggerRedelivery(Timeout t) { long now = System.nanoTime(); nackedMessages.forEach((msgId, timestamp) -> { if (timestamp < now) { - addChunkedMessageIdsAndRemoveFromSequnceMap(msgId, messagesToRedeliver, this.consumer); + addChunkedMessageIdsAndRemoveFromSequenceMap(msgId, messagesToRedeliver, this.consumer); messagesToRedeliver.add(msgId); } }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java index 0f27a63ee8e5b..305845e1f5f03 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java @@ -148,6 +148,8 @@ public class ProducerImpl extends ProducerBase implements TimerTask, Conne private Optional topicEpoch = Optional.empty(); private final List previousExceptions = new CopyOnWriteArrayList(); + private boolean errorState; + @SuppressWarnings("rawtypes") private static final AtomicLongFieldUpdater msgIdGeneratorUpdater = AtomicLongFieldUpdater .newUpdater(ProducerImpl.class, "msgIdGenerator"); @@ -258,6 +260,21 @@ public ProducerImpl(PulsarClientImpl client, String topic, ProducerConfiguration grabCnx(); } + protected void semaphoreRelease(final int releaseCountRequest) { + if (semaphore.isPresent()) { + if (!errorState) { + final int availablePermits = semaphore.get().availablePermits(); + if (availablePermits - releaseCountRequest < 0) { + log.error("Semaphore permit release count request greater then availablePermits" + + " : availablePermits={}, releaseCountRequest={}", + availablePermits, releaseCountRequest); + errorState = true; + } + } + semaphore.get().release(releaseCountRequest); + } + } + protected OpSendMsgQueue createPendingMessagesQueue() { return new OpSendMsgQueue(); } @@ -1005,9 +1022,9 @@ private long getHighestSequenceId(OpSendMsg op) { } private void releaseSemaphoreForSendOp(OpSendMsg op) { - if (semaphore.isPresent()) { - semaphore.get().release(isBatchMessagingEnabled() ? op.numMessagesInBatch : 1); - } + + semaphoreRelease(isBatchMessagingEnabled() ? op.numMessagesInBatch : 1); + client.getMemoryLimitController().releaseMemory(op.uncompressedSize); } @@ -1457,7 +1474,14 @@ public void connectionOpened(final ClientCnx cnx) { cnx.channel().close(); return null; } - log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + if (cause instanceof PulsarClientException.ProducerFencedException) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Failed to create producer: {}", + topic, producerName, cause.getMessage()); + } + } else { + log.error("[{}] [{}] Failed to create producer: {}", topic, producerName, cause.getMessage()); + } // Close the producer since topic does not exists. if (cause instanceof PulsarClientException.TopicDoesNotExistException) { closeAsync().whenComplete((v, ex) -> { @@ -1706,7 +1730,7 @@ private void failPendingMessages(ClientCnx cnx, PulsarClientException ex) { }); pendingMessages.clear(); - semaphore.ifPresent(s -> s.release(releaseCount.get())); + semaphoreRelease(releaseCount.get()); if (batchMessagingEnabled) { failPendingBatchMessages(ex); } @@ -1732,7 +1756,7 @@ private void failPendingBatchMessages(PulsarClientException ex) { } final int numMessagesInBatch = batchMessageContainer.getNumMessagesInBatch(); batchMessageContainer.discard(ex); - semaphore.ifPresent(s -> s.release(numMessagesInBatch)); + semaphoreRelease(numMessagesInBatch); } @Override @@ -1776,9 +1800,9 @@ private void batchMessageAndSend() { } } catch (PulsarClientException e) { Thread.currentThread().interrupt(); - semaphore.ifPresent(s -> s.release(batchMessageContainer.getNumMessagesInBatch())); + semaphoreRelease(batchMessageContainer.getNumMessagesInBatch()); } catch (Throwable t) { - semaphore.ifPresent(s -> s.release(batchMessageContainer.getNumMessagesInBatch())); + semaphoreRelease(batchMessageContainer.getNumMessagesInBatch()); log.warn("[{}] [{}] error while create opSendMsg by batch message container", topic, producerName, t); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java index db616f20450a4..ff07156972f62 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageTracker.java @@ -30,6 +30,7 @@ import java.io.Closeable; import java.util.ArrayDeque; +import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.Set; @@ -130,7 +131,7 @@ public void run(Timeout t) throws Exception { if (!headPartition.isEmpty()) { log.warn("[{}] {} messages have timed-out", consumerBase, headPartition.size()); headPartition.forEach(messageId -> { - addChunkedMessageIdsAndRemoveFromSequnceMap(messageId, messageIds, consumerBase); + addChunkedMessageIdsAndRemoveFromSequenceMap(messageId, messageIds, consumerBase); messageIds.add(messageId); messageIdPartitionMap.remove(messageId); }); @@ -150,14 +151,12 @@ public void run(Timeout t) throws Exception { }, this.tickDurationInMs, TimeUnit.MILLISECONDS); } - public static void addChunkedMessageIdsAndRemoveFromSequnceMap(MessageId messageId, Set messageIds, - ConsumerBase consumerBase) { + public static void addChunkedMessageIdsAndRemoveFromSequenceMap(MessageId messageId, Set messageIds, + ConsumerBase consumerBase) { if (messageId instanceof MessageIdImpl) { MessageIdImpl[] chunkedMsgIds = consumerBase.unAckedChunkedMessageIdSequenceMap.get((MessageIdImpl) messageId); if (chunkedMsgIds != null && chunkedMsgIds.length > 0) { - for (MessageIdImpl msgId : chunkedMsgIds) { - messageIds.add(msgId); - } + Collections.addAll(messageIds, chunkedMsgIds); } consumerBase.unAckedChunkedMessageIdSequenceMap.remove((MessageIdImpl) messageId); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/DefaultCryptoKeyReaderTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/DefaultCryptoKeyReaderTest.java index f1e4ccecb262e..2df1a9adc5771 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/DefaultCryptoKeyReaderTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/DefaultCryptoKeyReaderTest.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.commons.lang3.SystemUtils; import org.testng.annotations.Test; public class DefaultCryptoKeyReaderTest { @@ -85,14 +86,27 @@ public void testGetKeys() throws Exception { assertNull(keyReader1.getPublicKey("key0", null).getKey()); assertNull(keyReader1.getPrivateKey("key0", null).getKey()); - DefaultCryptoKeyReader keyReader2 = DefaultCryptoKeyReader.builder().defaultPublicKey("file:" + ecdsaPublicKey) + DefaultCryptoKeyReaderBuilder auxBuilder = DefaultCryptoKeyReader.builder().defaultPublicKey("file:" + ecdsaPublicKey) .defaultPrivateKey("file:" + ecdsaPrivateKey) - .publicKey("key1", - "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBQVUJMSUMgS0VZLS0tLS0KTUlJQklqQU5CZ2txaGtpRzl3MEJBUUVGQUFPQ0FROEFNSUlCQ2dLQ0FRRUF6elRUenNTc1pGWWxXeWJack1OdwphRGpncWluSU5vNXlOa0h1UkJQZzJyNTZCRWFIb1U1eStjY0RoeXhCR0NLUFprVGNRYXN2WWdXSjNzSFJLQWxOCmRaTkc4R3QzazJTcmZEcnJ0ajFLTDNHNk5XUkE4VHF5Umt4eGw1dnBBTWM2OVVqWDlIUHdTemxtckM3WlhtMWUKU3dZVFY3Kzdxcy82OUpMQm5yTUpjc2wrSXlYVWFoaFJuOHcyRmtzOUpXcmlOS2kxUFNnQ1BqTWpnS0JGN3lhRQpBVEowR01TTWM4RnZYV3dGSnNXQldRa1V3Z3FsRXhSMU1EaVZWQnR3OVF0SkIyOUlOaTBORHMyUGViNjFEdDQ5Ck5abE4va2xKQ1hJVXRCU0lxZzlvK2lSS1Z3WExIbklNMFdIVm5tUm4yTUswbmYwMy9Ed0NJVm5iNWVsVG9aNzIKOHdJREFRQUIKLS0tLS1FTkQgUFVCTElDIEtFWS0tLS0tCg==") // crypto_rsa_public.key - .privateKey("key1", - "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBSU0EgUFJJVkFURSBLRVktLS0tLQpNSUlFcEFJQkFBS0NBUUVBenpUVHpzU3NaRllsV3liWnJNTndhRGpncWluSU5vNXlOa0h1UkJQZzJyNTZCRWFICm9VNXkrY2NEaHl4QkdDS1Baa1RjUWFzdllnV0ozc0hSS0FsTmRaTkc4R3QzazJTcmZEcnJ0ajFLTDNHNk5XUkEKOFRxeVJreHhsNXZwQU1jNjlValg5SFB3U3psbXJDN1pYbTFlU3dZVFY3Kzdxcy82OUpMQm5yTUpjc2wrSXlYVQphaGhSbjh3MkZrczlKV3JpTktpMVBTZ0NQak1qZ0tCRjd5YUVBVEowR01TTWM4RnZYV3dGSnNXQldRa1V3Z3FsCkV4UjFNRGlWVkJ0dzlRdEpCMjlJTmkwTkRzMlBlYjYxRHQ0OU5abE4va2xKQ1hJVXRCU0lxZzlvK2lSS1Z3WEwKSG5JTTBXSFZubVJuMk1LMG5mMDMvRHdDSVZuYjVlbFRvWjcyOHdJREFRQUJBb0lCQUhXRnZmaVJua0dPaHNPTApabnpSb01qTU1janh4OGdCeFErM0YxL3ZjbUkvRk0rbC9UbGxXRnNKSUp3allveEExZHFvaGRDTk9tTzdSblpjCnNiZW1oeE4veEFXS3ZwaVB5Wis5ZjRHdWc0d2pVZjBFYnIwamtJZkV4Y3k2dGs0bHNlLzdMOWxMaE9mMWw2RmoKTlJDVXNaMlZ4WlRJZjdXakh2Qm02SUNOaFhkZmdjL1RPWC9INEJCTXh5UWtrbXZTN3lRSFBtbmVrVnBDandYaQpSZ2RQT3BCU0hVQXN1TGMzY2RPN0R6U2xYQnkrUjNVQjViQzk3ZWZTVHd4bU1kY0dVTlFoMTdDdXcrb3UyT0xKCmwvV3lNQkpnS1AwenA4TUkyWUNQMHRvRTFWVjBGV2lzaU5VZHl3Mm1tZHNLQlBDdFpXNEpmL2F2UkxqQ3B5ODMKZ3llSGk0a0NnWUVBN2ZhYzh1L1dvVWNSeGZ3YmV4eFZOSWI1anBWZ1EyMlhYVXZjT0JBMzE0NUhGSDRrRDlRcwpPbE9NNDhpRVgxR3ErRk9iK1RrVmEzeWVFYnlFSFZtTnhtN1pxREVsR2xQbkhIZ1dKZlZvNGx0ZW1rTlE4Y1FJCkNpRGhVSDdEOWlHZDRUckxxK3U4Slkvb3kwZHBKeWFKL0dzTlB3alZ6TWlBOWtEdUkyS0tScGNDZ1lFQTN1bHAKc1p5ODJFaWJUK1lZTnNoZkF5elBjUWMrQ1ZkY3BTM3lFVU1jTXJyR1QySUhiQWhsZHpBV3ZuaTdQRjFDdllvQgplb1BTR2lFR01Bd0FmYVVJNHBzWnZBVFpUZitHV0tpemxIODIwbHc0dFkyTlcydFlGd0RjWjZFUEtkcTlaQ096CkxmeXcyTmhMcWkyRnBGeUFwY1hsQTkyVVVJMEZjWENDdEFLSjJnVUNnWUVBc0k1bWVyVktpTlRETWlOUWZISlUKSWFuM3BUdmRkWW50WVhKMGpVQXpQb0s0NkZLRERSOStSVFJTZDNzQ0Evc0RJRVpnbG5RdEdWZ1hxODgwTXRhTQpJMnVCb0pIK0ZsK2tQUEk0ZEtkMXoyUzlkelYwN0R4blBxU1FwL20yQ1h0OXVXdTNTL0tXNFVPNkZJRUNXdUwwClJFMWxRWnliak5wREhQS2wvYWtTTVRjQ2dZQnFDODBXakNSakdKZWF1VEpIemFjMTBYbVdvZ1ZuV0VKZzZxekEKZlpiS280UjRlNEJnYXRZcWo1d2lYVGxtREZBVjc3T29YMUh5MEVjclVHcGpXOElRWEEwd0gzWnAzdWhCQVhEOQpjay9ZWDdzeTAvYXR5VEdOTUFHcTR6cGRoUXlZdVVzaTA1WW1jeS83ODlBaVUwZDRsZDdQcWZoSEllKzIrZm1VClBhanJLUUtCZ1FDanBqMFkrRS9IaTBTSlVLTlZQbDN1K1NOMWxMY1IxL2dNakpic1lGR3VhMU1zTFhCTlhUU2wKUWlZSGlhZFQ3QmhQRWtGZFc3dStiRndzMmFkbVcxOUJvVWIrd2d0WlQvdDduVHlvUzRMYWc0dnlhek5QWnpkUQp4NlhQcndaaW1kMFhERGl0R0xqY0xmOTkxRUZzWFNxUHpuRERmWHRKMzErb1U2T2JuSFNJdEE9PQotLS0tLUVORCBSU0EgUFJJVkFURSBLRVktLS0tLQo=") // crypto_rsa_private.key .publicKey("key2", "file:invalid").privateKey("key2", "file:invalid").publicKey("key3", "data:invalid") - .privateKey("key3", "data:invalid").build(); + .privateKey("key3", "data:invalid"); + DefaultCryptoKeyReader keyReader2; + // windows use \r\n instead of \n, so the base64 would be different + if (SystemUtils.IS_OS_WINDOWS) { + keyReader2 = auxBuilder + .publicKey("key1", + "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBQVUJMSUMgS0VZLS0tLS0NCk1JSUJJakFOQmdrcWhraUc5dzBCQVFFRkFBT0NBUThBTUlJQkNnS0NBUUVBenpUVHpzU3NaRllsV3liWnJNTncNCmFEamdxaW5JTm81eU5rSHVSQlBnMnI1NkJFYUhvVTV5K2NjRGh5eEJHQ0tQWmtUY1Fhc3ZZZ1dKM3NIUktBbE4NCmRaTkc4R3QzazJTcmZEcnJ0ajFLTDNHNk5XUkE4VHF5Umt4eGw1dnBBTWM2OVVqWDlIUHdTemxtckM3WlhtMWUNClN3WVRWNys3cXMvNjlKTEJuck1KY3NsK0l5WFVhaGhSbjh3MkZrczlKV3JpTktpMVBTZ0NQak1qZ0tCRjd5YUUNCkFUSjBHTVNNYzhGdlhXd0ZKc1dCV1FrVXdncWxFeFIxTURpVlZCdHc5UXRKQjI5SU5pME5EczJQZWI2MUR0NDkNCk5abE4va2xKQ1hJVXRCU0lxZzlvK2lSS1Z3WExIbklNMFdIVm5tUm4yTUswbmYwMy9Ed0NJVm5iNWVsVG9aNzINCjh3SURBUUFCDQotLS0tLUVORCBQVUJMSUMgS0VZLS0tLS0NCg==") // crypto_rsa_public.key + .privateKey("key1", + "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBSU0EgUFJJVkFURSBLRVktLS0tLQ0KTUlJRXBBSUJBQUtDQVFFQXp6VFR6c1NzWkZZbFd5YlpyTU53YURqZ3FpbklObzV5TmtIdVJCUGcycjU2QkVhSA0Kb1U1eStjY0RoeXhCR0NLUFprVGNRYXN2WWdXSjNzSFJLQWxOZFpORzhHdDNrMlNyZkRycnRqMUtMM0c2TldSQQ0KOFRxeVJreHhsNXZwQU1jNjlValg5SFB3U3psbXJDN1pYbTFlU3dZVFY3Kzdxcy82OUpMQm5yTUpjc2wrSXlYVQ0KYWhoUm44dzJGa3M5SldyaU5LaTFQU2dDUGpNamdLQkY3eWFFQVRKMEdNU01jOEZ2WFd3RkpzV0JXUWtVd2dxbA0KRXhSMU1EaVZWQnR3OVF0SkIyOUlOaTBORHMyUGViNjFEdDQ5TlpsTi9rbEpDWElVdEJTSXFnOW8raVJLVndYTA0KSG5JTTBXSFZubVJuMk1LMG5mMDMvRHdDSVZuYjVlbFRvWjcyOHdJREFRQUJBb0lCQUhXRnZmaVJua0dPaHNPTA0KWm56Um9Nak1NY2p4eDhnQnhRKzNGMS92Y21JL0ZNK2wvVGxsV0ZzSklKd2pZb3hBMWRxb2hkQ05PbU83Um5aYw0Kc2JlbWh4Ti94QVdLdnBpUHlaKzlmNEd1ZzR3alVmMEVicjBqa0lmRXhjeTZ0azRsc2UvN0w5bExoT2YxbDZGag0KTlJDVXNaMlZ4WlRJZjdXakh2Qm02SUNOaFhkZmdjL1RPWC9INEJCTXh5UWtrbXZTN3lRSFBtbmVrVnBDandYaQ0KUmdkUE9wQlNIVUFzdUxjM2NkTzdEelNsWEJ5K1IzVUI1YkM5N2VmU1R3eG1NZGNHVU5RaDE3Q3V3K291Mk9MSg0KbC9XeU1CSmdLUDB6cDhNSTJZQ1AwdG9FMVZWMEZXaXNpTlVkeXcybW1kc0tCUEN0Wlc0SmYvYXZSTGpDcHk4Mw0KZ3llSGk0a0NnWUVBN2ZhYzh1L1dvVWNSeGZ3YmV4eFZOSWI1anBWZ1EyMlhYVXZjT0JBMzE0NUhGSDRrRDlRcw0KT2xPTTQ4aUVYMUdxK0ZPYitUa1ZhM3llRWJ5RUhWbU54bTdacURFbEdsUG5ISGdXSmZWbzRsdGVta05ROGNRSQ0KQ2lEaFVIN0Q5aUdkNFRyTHErdThKWS9veTBkcEp5YUovR3NOUHdqVnpNaUE5a0R1STJLS1JwY0NnWUVBM3VscA0Kc1p5ODJFaWJUK1lZTnNoZkF5elBjUWMrQ1ZkY3BTM3lFVU1jTXJyR1QySUhiQWhsZHpBV3ZuaTdQRjFDdllvQg0KZW9QU0dpRUdNQXdBZmFVSTRwc1p2QVRaVGYrR1dLaXpsSDgyMGx3NHRZMk5XMnRZRndEY1o2RVBLZHE5WkNPeg0KTGZ5dzJOaExxaTJGcEZ5QXBjWGxBOTJVVUkwRmNYQ0N0QUtKMmdVQ2dZRUFzSTVtZXJWS2lOVERNaU5RZkhKVQ0KSWFuM3BUdmRkWW50WVhKMGpVQXpQb0s0NkZLRERSOStSVFJTZDNzQ0Evc0RJRVpnbG5RdEdWZ1hxODgwTXRhTQ0KSTJ1Qm9KSCtGbCtrUFBJNGRLZDF6MlM5ZHpWMDdEeG5QcVNRcC9tMkNYdDl1V3UzUy9LVzRVTzZGSUVDV3VMMA0KUkUxbFFaeWJqTnBESFBLbC9ha1NNVGNDZ1lCcUM4MFdqQ1JqR0plYXVUSkh6YWMxMFhtV29nVm5XRUpnNnF6QQ0KZlpiS280UjRlNEJnYXRZcWo1d2lYVGxtREZBVjc3T29YMUh5MEVjclVHcGpXOElRWEEwd0gzWnAzdWhCQVhEOQ0KY2svWVg3c3kwL2F0eVRHTk1BR3E0enBkaFF5WXVVc2kwNVltY3kvNzg5QWlVMGQ0bGQ3UHFmaEhJZSsyK2ZtVQ0KUGFqcktRS0JnUUNqcGowWStFL0hpMFNKVUtOVlBsM3UrU04xbExjUjEvZ01qSmJzWUZHdWExTXNMWEJOWFRTbA0KUWlZSGlhZFQ3QmhQRWtGZFc3dStiRndzMmFkbVcxOUJvVWIrd2d0WlQvdDduVHlvUzRMYWc0dnlhek5QWnpkUQ0KeDZYUHJ3WmltZDBYRERpdEdMamNMZjk5MUVGc1hTcVB6bkREZlh0SjMxK29VNk9ibkhTSXRBPT0NCi0tLS0tRU5EIFJTQSBQUklWQVRFIEtFWS0tLS0tDQo=") + .build(); + } else { + keyReader2 = auxBuilder + .publicKey("key1", + "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBQVUJMSUMgS0VZLS0tLS0KTUlJQklqQU5CZ2txaGtpRzl3MEJBUUVGQUFPQ0FROEFNSUlCQ2dLQ0FRRUF6elRUenNTc1pGWWxXeWJack1OdwphRGpncWluSU5vNXlOa0h1UkJQZzJyNTZCRWFIb1U1eStjY0RoeXhCR0NLUFprVGNRYXN2WWdXSjNzSFJLQWxOCmRaTkc4R3QzazJTcmZEcnJ0ajFLTDNHNk5XUkE4VHF5Umt4eGw1dnBBTWM2OVVqWDlIUHdTemxtckM3WlhtMWUKU3dZVFY3Kzdxcy82OUpMQm5yTUpjc2wrSXlYVWFoaFJuOHcyRmtzOUpXcmlOS2kxUFNnQ1BqTWpnS0JGN3lhRQpBVEowR01TTWM4RnZYV3dGSnNXQldRa1V3Z3FsRXhSMU1EaVZWQnR3OVF0SkIyOUlOaTBORHMyUGViNjFEdDQ5Ck5abE4va2xKQ1hJVXRCU0lxZzlvK2lSS1Z3WExIbklNMFdIVm5tUm4yTUswbmYwMy9Ed0NJVm5iNWVsVG9aNzIKOHdJREFRQUIKLS0tLS1FTkQgUFVCTElDIEtFWS0tLS0tCg==") // crypto_rsa_public.key + .privateKey("key1", + "data:application/x-pem-file;base64,LS0tLS1CRUdJTiBSU0EgUFJJVkFURSBLRVktLS0tLQpNSUlFcEFJQkFBS0NBUUVBenpUVHpzU3NaRllsV3liWnJNTndhRGpncWluSU5vNXlOa0h1UkJQZzJyNTZCRWFICm9VNXkrY2NEaHl4QkdDS1Baa1RjUWFzdllnV0ozc0hSS0FsTmRaTkc4R3QzazJTcmZEcnJ0ajFLTDNHNk5XUkEKOFRxeVJreHhsNXZwQU1jNjlValg5SFB3U3psbXJDN1pYbTFlU3dZVFY3Kzdxcy82OUpMQm5yTUpjc2wrSXlYVQphaGhSbjh3MkZrczlKV3JpTktpMVBTZ0NQak1qZ0tCRjd5YUVBVEowR01TTWM4RnZYV3dGSnNXQldRa1V3Z3FsCkV4UjFNRGlWVkJ0dzlRdEpCMjlJTmkwTkRzMlBlYjYxRHQ0OU5abE4va2xKQ1hJVXRCU0lxZzlvK2lSS1Z3WEwKSG5JTTBXSFZubVJuMk1LMG5mMDMvRHdDSVZuYjVlbFRvWjcyOHdJREFRQUJBb0lCQUhXRnZmaVJua0dPaHNPTApabnpSb01qTU1janh4OGdCeFErM0YxL3ZjbUkvRk0rbC9UbGxXRnNKSUp3allveEExZHFvaGRDTk9tTzdSblpjCnNiZW1oeE4veEFXS3ZwaVB5Wis5ZjRHdWc0d2pVZjBFYnIwamtJZkV4Y3k2dGs0bHNlLzdMOWxMaE9mMWw2RmoKTlJDVXNaMlZ4WlRJZjdXakh2Qm02SUNOaFhkZmdjL1RPWC9INEJCTXh5UWtrbXZTN3lRSFBtbmVrVnBDandYaQpSZ2RQT3BCU0hVQXN1TGMzY2RPN0R6U2xYQnkrUjNVQjViQzk3ZWZTVHd4bU1kY0dVTlFoMTdDdXcrb3UyT0xKCmwvV3lNQkpnS1AwenA4TUkyWUNQMHRvRTFWVjBGV2lzaU5VZHl3Mm1tZHNLQlBDdFpXNEpmL2F2UkxqQ3B5ODMKZ3llSGk0a0NnWUVBN2ZhYzh1L1dvVWNSeGZ3YmV4eFZOSWI1anBWZ1EyMlhYVXZjT0JBMzE0NUhGSDRrRDlRcwpPbE9NNDhpRVgxR3ErRk9iK1RrVmEzeWVFYnlFSFZtTnhtN1pxREVsR2xQbkhIZ1dKZlZvNGx0ZW1rTlE4Y1FJCkNpRGhVSDdEOWlHZDRUckxxK3U4Slkvb3kwZHBKeWFKL0dzTlB3alZ6TWlBOWtEdUkyS0tScGNDZ1lFQTN1bHAKc1p5ODJFaWJUK1lZTnNoZkF5elBjUWMrQ1ZkY3BTM3lFVU1jTXJyR1QySUhiQWhsZHpBV3ZuaTdQRjFDdllvQgplb1BTR2lFR01Bd0FmYVVJNHBzWnZBVFpUZitHV0tpemxIODIwbHc0dFkyTlcydFlGd0RjWjZFUEtkcTlaQ096CkxmeXcyTmhMcWkyRnBGeUFwY1hsQTkyVVVJMEZjWENDdEFLSjJnVUNnWUVBc0k1bWVyVktpTlRETWlOUWZISlUKSWFuM3BUdmRkWW50WVhKMGpVQXpQb0s0NkZLRERSOStSVFJTZDNzQ0Evc0RJRVpnbG5RdEdWZ1hxODgwTXRhTQpJMnVCb0pIK0ZsK2tQUEk0ZEtkMXoyUzlkelYwN0R4blBxU1FwL20yQ1h0OXVXdTNTL0tXNFVPNkZJRUNXdUwwClJFMWxRWnliak5wREhQS2wvYWtTTVRjQ2dZQnFDODBXakNSakdKZWF1VEpIemFjMTBYbVdvZ1ZuV0VKZzZxekEKZlpiS280UjRlNEJnYXRZcWo1d2lYVGxtREZBVjc3T29YMUh5MEVjclVHcGpXOElRWEEwd0gzWnAzdWhCQVhEOQpjay9ZWDdzeTAvYXR5VEdOTUFHcTR6cGRoUXlZdVVzaTA1WW1jeS83ODlBaVUwZDRsZDdQcWZoSEllKzIrZm1VClBhanJLUUtCZ1FDanBqMFkrRS9IaTBTSlVLTlZQbDN1K1NOMWxMY1IxL2dNakpic1lGR3VhMU1zTFhCTlhUU2wKUWlZSGlhZFQ3QmhQRWtGZFc3dStiRndzMmFkbVcxOUJvVWIrd2d0WlQvdDduVHlvUzRMYWc0dnlhek5QWnpkUQp4NlhQcndaaW1kMFhERGl0R0xqY0xmOTkxRUZzWFNxUHpuRERmWHRKMzErb1U2T2JuSFNJdEE9PQotLS0tLUVORCBSU0EgUFJJVkFURSBLRVktLS0tLQo=") + .build(); + } assertNotNull(keyReader2.getPublicKey("key0", null).getKey()); assertEquals(keyReader2.getPublicKey("key0", null).getKey(), Files.readAllBytes(Paths.get(ecdsaPublicKey))); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java index 5ca9d07c56558..4c2d99cc5d7bb 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/NamespaceIsolationDataImpl.java @@ -59,7 +59,7 @@ public class NamespaceIsolationDataImpl implements NamespaceIsolationData { private List primary; @ApiModelProperty( - name = "primary", + name = "secondary", value = "The list of secondary brokers for serving the list of namespaces in this isolation policy" ) private List secondary; diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java index ec55f5349110c..faefc8adcd78e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SecurityUtility.java @@ -422,12 +422,12 @@ public static X509Certificate[] loadCertificatesFromPemStream(InputStream inStre } public static PrivateKey loadPrivateKeyFromPemFile(String keyFilePath) throws KeyManagementException { - PrivateKey privateKey = null; - if (keyFilePath == null || keyFilePath.isEmpty()) { - return privateKey; + return null; } + PrivateKey privateKey; + try (FileInputStream input = new FileInputStream(keyFilePath)) { privateKey = loadPrivateKeyFromPemStream(input); } catch (IOException e) { @@ -438,12 +438,12 @@ public static PrivateKey loadPrivateKeyFromPemFile(String keyFilePath) throws Ke } public static PrivateKey loadPrivateKeyFromPemStream(InputStream inStream) throws KeyManagementException { - PrivateKey privateKey = null; - if (inStream == null) { - return privateKey; + return null; } + PrivateKey privateKey; + try (BufferedReader reader = new BufferedReader(new InputStreamReader(inStream, StandardCharsets.UTF_8))) { if (inStream.markSupported()) { inStream.reset(); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java index d7c6a71f8c36f..4f0287f19ffc7 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/WorkerUtils.java @@ -352,7 +352,9 @@ public static Producer createExclusiveProducerWithRetry(PulsarClient cli } tries++; if (tries % 6 == 0) { - log.warn("Failed to acquire exclusive producer to topic {} after {} attempts. Will retry if we are still the leader.", topic, tries); + if (log.isDebugEnabled()) { + log.debug("Failed to acquire exclusive producer to topic {} after {} attempts. Will retry if we are still the leader.", topic, tries); + } } Thread.sleep(sleepInBetweenMs); } while (isLeader.get()); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index 40ac38ecb1683..b031d1a472ad5 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -832,7 +832,7 @@ public void testUpdateSinkDifferentInputs() throws Exception { PowerMockito.when(WorkerUtils.class, "dumpToTmpFile", any()).thenCallRealMethod(); Map inputTopics = new HashMap<>(); - inputTopics.put("DifferntTopic", DEFAULT_SERDE); + inputTopics.put("DifferentTopic", DEFAULT_SERDE); testUpdateSinkMissingArguments( tenant, namespace, diff --git a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/AbstractKafkaConnectSource.java b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/AbstractKafkaConnectSource.java index 3901c5f3d176b..4612633677b1a 100644 --- a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/AbstractKafkaConnectSource.java +++ b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/AbstractKafkaConnectSource.java @@ -27,8 +27,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.kafka.connect.runtime.TaskConfig; @@ -168,6 +166,7 @@ public synchronized Record read() throws Exception { } catch (ExecutionException ex) { // log the error, continue execution log.error("execution exception while get flushFuture", ex); + throw new Exception("Flush failed", ex.getCause()); } finally { flushFuture = null; currentBatch = null; @@ -180,6 +179,12 @@ public synchronized Record read() throws Exception { public void close() { if (sourceTask != null) { sourceTask.stop(); + sourceTask = null; + } + + if (offsetStore != null) { + offsetStore.stop(); + offsetStore = null; } } @@ -187,7 +192,6 @@ public void close() { private static Map PROPERTIES = Collections.emptyMap(); private static Optional RECORD_SEQUENCE = Optional.empty(); - private static long FLUSH_TIMEOUT_MS = 60000; public abstract class AbstractKafkaSourceRecord implements Record { @Getter @@ -248,8 +252,15 @@ private void completedFlushOffset(Throwable error, Void result) { flushFuture.complete(null); } catch (InterruptedException exception) { log.warn("Flush of {} offsets interrupted, cancelling", this); + Thread.currentThread().interrupt(); + offsetWriter.cancelFlush(); + flushFuture.completeExceptionally(new Exception("Failed to commit offsets", exception)); + } catch (Throwable t) { + // SourceTask can throw unchecked ConnectException/KafkaException. + // Make sure the future is cancelled in that case + log.warn("Flush of {} offsets failed, cancelling", this); offsetWriter.cancelFlush(); - flushFuture.completeExceptionally(new Exception("Failed to commit offsets")); + flushFuture.completeExceptionally(new Exception("Failed to commit offsets", t)); } } } diff --git a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarOffsetBackingStore.java b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarOffsetBackingStore.java index b1338f837bb49..495c8b9c194c0 100644 --- a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarOffsetBackingStore.java +++ b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/PulsarOffsetBackingStore.java @@ -158,12 +158,19 @@ public void start() { @Override public void stop() { + log.info("Stopping PulsarOffsetBackingStore"); if (null != producer) { + try { + producer.flush(); + } catch (PulsarClientException pce) { + log.warn("Failed to flush the producer", pce); + } try { producer.close(); } catch (PulsarClientException e) { log.warn("Failed to close producer", e); } + producer = null; } if (null != reader) { try { @@ -171,7 +178,10 @@ public void stop() { } catch (IOException e) { log.warn("Failed to close reader", e); } + reader = null; } + + // do not close the client, it is provided by the sink context } @Override diff --git a/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/ErrFileStreamSourceTask.java b/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/ErrFileStreamSourceTask.java new file mode 100644 index 0000000000000..d17f32cfc47a0 --- /dev/null +++ b/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/ErrFileStreamSourceTask.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.pulsar.io.kafka.connect; + +import org.apache.kafka.connect.file.FileStreamSourceTask; + +public class ErrFileStreamSourceTask extends FileStreamSourceTask { + + @Override + public void commit() throws InterruptedException { + throw new org.apache.kafka.connect.errors.ConnectException("blah"); + } + +} diff --git a/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSourceErrTest.java b/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSourceErrTest.java new file mode 100644 index 0000000000000..cc04706f3ee45 --- /dev/null +++ b/pulsar-io/kafka-connect-adaptor/src/test/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSourceErrTest.java @@ -0,0 +1,145 @@ +/** + * 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.io.kafka.connect; + +import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.connect.file.FileStreamSourceConnector; +import org.apache.kafka.connect.runtime.TaskConfig; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.SourceContext; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.io.File; +import java.io.OutputStream; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +/** + * Test the implementation of {@link KafkaConnectSource}. + */ +@Slf4j +public class KafkaConnectSourceErrTest extends ProducerConsumerBase { + + private Map config = new HashMap<>(); + private String offsetTopicName; + // The topic to publish data to, for kafkaSource + private String topicName; + private KafkaConnectSource kafkaConnectSource; + private File tempFile; + private SourceContext context; + private PulsarClient client; + + @BeforeMethod + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + + config.put(TaskConfig.TASK_CLASS_CONFIG, "org.apache.pulsar.io.kafka.connect.ErrFileStreamSourceTask"); + config.put(PulsarKafkaWorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + config.put(PulsarKafkaWorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + + this.offsetTopicName = "persistent://my-property/my-ns/kafka-connect-source-offset"; + config.put(PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG, offsetTopicName); + + this.topicName = "persistent://my-property/my-ns/kafka-connect-source"; + config.put(FileStreamSourceConnector.TOPIC_CONFIG, topicName); + tempFile = File.createTempFile("some-file-name", null); + config.put(FileStreamSourceConnector.FILE_CONFIG, tempFile.getAbsoluteFile().toString()); + config.put(FileStreamSourceConnector.TASK_BATCH_SIZE_CONFIG, String.valueOf(FileStreamSourceConnector.DEFAULT_TASK_BATCH_SIZE)); + + this.context = mock(SourceContext.class); + this.client = PulsarClient.builder() + .serviceUrl(brokerUrl.toString()) + .build(); + when(context.getPulsarClient()).thenReturn(this.client); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + if (this.client != null) { + this.client.close(); + } + tempFile.delete(); + super.internalCleanup(); + } + + @Test + public void testOpenAndRead() throws Exception { + kafkaConnectSource = new KafkaConnectSource(); + kafkaConnectSource.open(config, context); + + // use FileStreamSourceConnector, each line is a record, need "\n" and end of each record. + OutputStream os = Files.newOutputStream(tempFile.toPath()); + + String line1 = "This is the first line\n"; + os.write(line1.getBytes()); + os.flush(); + log.info("write 2 lines."); + + String line2 = "This is the second line\n"; + os.write(line2.getBytes()); + os.flush(); + + log.info("finish write, will read 2 lines"); + + // Note: FileStreamSourceTask read the whole line as Value, and set Key as null. + Record> record = kafkaConnectSource.read(); + String readBack1 = new String(record.getValue().getValue()); + assertTrue(line1.contains(readBack1)); + assertNull(record.getValue().getKey()); + log.info("read line1: {}", readBack1); + record.ack(); + + record = kafkaConnectSource.read(); + String readBack2 = new String(record.getValue().getValue()); + assertTrue(line2.contains(readBack2)); + assertNull(record.getValue().getKey()); + assertTrue(record.getPartitionId().isPresent()); + assertFalse(record.getPartitionIndex().isPresent()); + log.info("read line2: {}", readBack2); + record.ack(); + + String line3 = "This is the 3rd line\n"; + os.write(line3.getBytes()); + os.flush(); + + try { + kafkaConnectSource.read(); + fail("expected exception"); + } catch (Exception e) { + log.info("got exception", e); + assertTrue(e.getCause().getCause() instanceof org.apache.kafka.connect.errors.ConnectException); + } + } +} diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index fe24f16001d80..e8090258b54fa 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -179,6 +179,27 @@ log4j-core + + + + org.gaul + modernizer-maven-plugin + ${modernizer-maven-plugin.version} + + true + 8 + + + + modernizer + + modernizer + + + + + + diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensions.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensions.java index 8f58a0938a58c..ead2fa2ef1c40 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensions.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/extensions/ProxyExtensions.java @@ -19,10 +19,10 @@ package org.apache.pulsar.proxy.extensions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; +import java.util.HashMap; +import java.util.HashSet; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.proxy.server.ProxyConfiguration; import org.apache.pulsar.proxy.server.ProxyService; @@ -108,8 +108,8 @@ public void initialize(ProxyConfiguration conf) throws Exception { } public Map>> newChannelInitializers() { - Map>> channelInitializers = Maps.newHashMap(); - Set addresses = Sets.newHashSet(); + Map>> channelInitializers = new HashMap<>(); + Set addresses = new HashSet<>(); for (Map.Entry extension : extensions.entrySet()) { Map> initializers = diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java index 9ce7a26d99c00..f152b32d62dd1 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ParserProxyHandler.java @@ -20,6 +20,7 @@ package org.apache.pulsar.proxy.server; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -33,8 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Lists; - import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.CompositeByteBuf; @@ -96,7 +95,7 @@ private void logging(Channel conn, BaseCommand.Type cmdtype, String info, List messages = Lists.newArrayList(); + List messages = new ArrayList<>(); ByteBuf buffer = (ByteBuf)(msg); try { diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java index 0573670e993ae..5bad624824a72 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConfiguration.java @@ -18,14 +18,14 @@ */ package org.apache.pulsar.proxy.server; -import com.google.common.collect.Sets; - import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.TreeSet; import java.util.stream.Collectors; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -203,7 +203,7 @@ public class ProxyConfiguration implements PulsarConfiguration { + " `super-user`, meaning they will be able to do all admin operations and publish" + " & consume from all topics" ) - private Set superUserRoles = Sets.newTreeSet(); + private Set superUserRoles = new TreeSet<>(); @FieldContext( category = CATEGORY_AUTHENTICATION, @@ -214,7 +214,7 @@ public class ProxyConfiguration implements PulsarConfiguration { category = CATEGORY_AUTHENTICATION, doc = "Authentication provider name list (a comma-separated list of class names" ) - private Set authenticationProviders = Sets.newTreeSet(); + private Set authenticationProviders = new TreeSet<>(); @FieldContext( category = CATEGORY_AUTHORIZATION, doc = "Whether authorization is enforced by the Pulsar proxy" @@ -346,14 +346,14 @@ public class ProxyConfiguration implements PulsarConfiguration { + " (a comma-separated list of protocol names).\n\n" + "Examples:- [TLSv1.3, TLSv1.2]" ) - private Set tlsProtocols = Sets.newTreeSet(); + private Set tlsProtocols = new TreeSet<>(); @FieldContext( category = CATEGORY_TLS, doc = "Specify the tls cipher the proxy will use to negotiate during TLS Handshake" + " (a comma-separated list of ciphers).\n\n" + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]" ) - private Set tlsCiphers = Sets.newTreeSet(); + private Set tlsCiphers = new TreeSet<>(); @FieldContext( category = CATEGORY_TLS, doc = "Whether client certificates are required for TLS.\n\n" @@ -449,7 +449,7 @@ public class ProxyConfiguration implements PulsarConfiguration { + "Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].\n" + " used by the Pulsar proxy to authenticate with Pulsar brokers" ) - private Set brokerClientTlsCiphers = Sets.newTreeSet(); + private Set brokerClientTlsCiphers = new TreeSet<>(); @FieldContext( category = CATEGORY_KEYSTORE_TLS, doc = "Specify the tls protocols the broker will use to negotiate during TLS handshake" @@ -457,7 +457,7 @@ public class ProxyConfiguration implements PulsarConfiguration { + "Examples:- [TLSv1.3, TLSv1.2] \n" + " used by the Pulsar proxy to authenticate with Pulsar brokers" ) - private Set brokerClientTlsProtocols = Sets.newTreeSet(); + private Set brokerClientTlsProtocols = new TreeSet<>(); /***** --- HTTP --- ****/ @@ -465,7 +465,7 @@ public class ProxyConfiguration implements PulsarConfiguration { category = CATEGORY_HTTP, doc = "Http directs to redirect to non-pulsar services" ) - private Set httpReverseProxyConfigs = Sets.newHashSet(); + private Set httpReverseProxyConfigs = new HashSet<>(); @FieldContext( minValue = 1, @@ -523,13 +523,13 @@ public class ProxyConfiguration implements PulsarConfiguration { category = CATEGORY_PLUGIN, doc = "List of proxy additional servlet to load, which is a list of proxy additional servlet names" ) - private Set proxyAdditionalServlets = Sets.newTreeSet(); + private Set proxyAdditionalServlets = new TreeSet<>(); @FieldContext( category = CATEGORY_PLUGIN, doc = "List of proxy additional servlet to load, which is a list of proxy additional servlet names" ) - private Set additionalServlets = Sets.newTreeSet(); + private Set additionalServlets = new TreeSet<>(); @FieldContext( category = CATEGORY_HTTP, @@ -579,7 +579,7 @@ public class ProxyConfiguration implements PulsarConfiguration { category = CATEGORY_PLUGIN, doc = "List of messaging protocols to load, which is a list of extension names" ) - private Set proxyExtensions = Sets.newTreeSet(); + private Set proxyExtensions = new TreeSet<>(); /***** --- WebSocket --- ****/ @FieldContext( diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java index af5b2a8ec9409..0486a8fbefe1a 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyService.java @@ -18,9 +18,8 @@ */ package org.apache.pulsar.proxy.server; -import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isBlank; -import com.google.common.collect.Maps; import com.google.common.collect.Sets; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.AdaptiveRecvByteBufAllocator; @@ -41,6 +40,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.Semaphore; @@ -131,11 +131,11 @@ public class ProxyService implements Closeable { public ProxyService(ProxyConfiguration proxyConfig, AuthenticationService authenticationService) throws Exception { - checkNotNull(proxyConfig); + requireNonNull(proxyConfig); this.proxyConfig = proxyConfig; this.timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); this.clientCnxs = Sets.newConcurrentHashSet(); - this.topicStats = Maps.newConcurrentMap(); + this.topicStats = new ConcurrentHashMap<>(); this.lookupRequestSemaphore = new AtomicReference( new Semaphore(proxyConfig.getMaxConcurrentLookupRequests(), false)); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 235927c98c5e2..565c3c5462e9e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -312,6 +312,11 @@ public ProxyConfiguration getConfig() { return config; } + @VisibleForTesting + public WebServer getServer() { + return server; + } + private static final Logger log = LoggerFactory.getLogger(ProxyServiceStarter.class); } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java index 8a9956ca493fb..c92f722c76049 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/WebServer.java @@ -18,8 +18,6 @@ */ package org.apache.pulsar.proxy.server; -import com.google.common.collect.Lists; - import io.prometheus.client.jetty.JettyStatisticsCollector; import java.io.IOException; import java.net.URI; @@ -69,8 +67,8 @@ public class WebServer { private final Server server; private final WebExecutorThreadPool webServiceExecutor; private final AuthenticationService authenticationService; - private final List servletPaths = Lists.newArrayList(); - private final List handlers = Lists.newArrayList(); + private final List servletPaths = new ArrayList<>(); + private final List handlers = new ArrayList<>(); private final ProxyConfiguration config; protected int externalServicePort; private URI serviceURI = null; @@ -84,7 +82,7 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication this.authenticationService = authenticationService; this.config = config; - List connectors = Lists.newArrayList(); + List connectors = new ArrayList<>(); HttpConfiguration http_config = new HttpConfiguration(); http_config.setOutputBufferSize(config.getHttpOutputBufferSize()); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/ProxyStats.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/ProxyStats.java index 92e4852d83b0b..f709a2571ba4e 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/ProxyStats.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/stats/ProxyStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.proxy.stats; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; @@ -34,8 +35,6 @@ import org.apache.pulsar.proxy.server.ProxyService; -import com.google.common.collect.Lists; - import io.netty.channel.Channel; import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; @@ -59,7 +58,7 @@ public class ProxyStats { @ApiOperation(value = "Proxy stats api to get info for live connections", response = List.class, responseContainer = "List") @ApiResponses(value = { @ApiResponse(code = 503, message = "Proxy service is not initialized") }) public List metrics() { - List stats = Lists.newArrayList(); + List stats = new ArrayList<>(); proxyService().getClientCnxs().forEach(cnx -> { if (cnx.getDirectProxyHandler() == null) { return; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java index e63d3aeb4cb96..706a57d677407 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java @@ -189,7 +189,7 @@ public void testTlsSyncProducerAndConsumer() throws Exception { } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java index af76bfaeb2bb4..b44f0caa90298 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.proxy.server; -import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import com.google.common.collect.Sets; @@ -195,7 +195,7 @@ public void testPartitions() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java index 9b0e9b427e56c..0bf5c5e64849d 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.proxy.server; -import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import java.util.HashMap; @@ -180,7 +180,7 @@ public void testPartitions() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 905ca2066c738..b0d53ad3df7db 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -19,8 +19,8 @@ package org.apache.pulsar.proxy.server; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import static org.testng.Assert.assertEquals; @@ -128,7 +128,7 @@ public void testProducerConsumer() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); } @@ -162,7 +162,7 @@ public void testPartitions() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); } } @@ -231,7 +231,7 @@ public void testProtocolVersionAdvertisement() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(10, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java index 3377ec266a713..bdba8d35c5515 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceStarterTest.java @@ -59,7 +59,8 @@ protected void setup() throws Exception { serviceStarter = new ProxyServiceStarter(ARGS); serviceStarter.getConfig().setBrokerServiceURL(pulsar.getBrokerServiceUrl()); serviceStarter.getConfig().setBrokerWebServiceURL(pulsar.getWebServiceAddress()); - serviceStarter.getConfig().setServicePort(Optional.of(11000)); + serviceStarter.getConfig().setWebServicePort(Optional.of(0)); + serviceStarter.getConfig().setServicePort(Optional.of(0)); serviceStarter.getConfig().setWebSocketServiceEnabled(true); serviceStarter.start(); } @@ -71,14 +72,19 @@ protected void cleanup() throws Exception { serviceStarter.close(); } + private String computeWsBasePath() { + return String.format("ws://localhost:%d/ws", serviceStarter.getServer().getListenPortHTTP().get()); + } + @Test public void testEnableWebSocketServer() throws Exception { HttpClient httpClient = new HttpClient(); WebSocketClient webSocketClient = new WebSocketClient(httpClient); webSocketClient.start(); MyWebSocket myWebSocket = new MyWebSocket(); - String webSocketUri = "ws://localhost:8080/ws/pingpong"; + String webSocketUri = computeWsBasePath() + "/pingpong"; Future sessionFuture = webSocketClient.connect(myWebSocket, URI.create(webSocketUri)); + System.out.println("uri" + webSocketUri); sessionFuture.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes())); assertTrue(myWebSocket.getResponse().contains("ping")); } @@ -86,7 +92,7 @@ public void testEnableWebSocketServer() throws Exception { @Test public void testProducer() throws Exception { @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:11000") + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:" + this.pulsar.getBrokerService().getListenPort().get()) .build(); @Cleanup @@ -105,7 +111,7 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { WebSocketClient producerWebSocketClient = new WebSocketClient(producerClient); producerWebSocketClient.start(); MyWebSocket producerSocket = new MyWebSocket(); - String produceUri = "ws://localhost:8080/ws/producer/persistent/sample/test/local/websocket-topic"; + String produceUri = computeWsBasePath() + "/producer/persistent/sample/test/local/websocket-topic"; Future producerSession = producerWebSocketClient.connect(producerSocket, URI.create(produceUri)); ProducerMessage produceRequest = new ProducerMessage(); @@ -116,7 +122,7 @@ public void testProduceAndConsumeMessageWithWebsocket() throws Exception { WebSocketClient consumerWebSocketClient = new WebSocketClient(consumerClient); consumerWebSocketClient.start(); MyWebSocket consumerSocket = new MyWebSocket(); - String consumeUri = "ws://localhost:8080/ws/consumer/persistent/sample/test/local/websocket-topic/my-sub"; + String consumeUri = computeWsBasePath() + "/consumer/persistent/sample/test/local/websocket-topic/my-sub"; Future consumerSession = consumerWebSocketClient.connect(consumerSocket, URI.create(consumeUri)); consumerSession.get().getRemote().sendPing(ByteBuffer.wrap("ping".getBytes())); producerSession.get().getRemote().sendString(ObjectMapperFactory.getThreadLocal().writeValueAsString(produceRequest)); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java index 2b1c22c22d012..55b48feed1f6f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.proxy.server; -import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; @@ -120,7 +120,7 @@ public void testConnectionsStats() throws Exception { for (int i = 0; i < totalMessages; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); } @@ -169,7 +169,7 @@ public void testTopicStats() throws Exception { for (int i = 0; i < totalMessages; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); msg = consumer2.receive(1, TimeUnit.SECONDS); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index 92f6a63d0f185..edeb0006a845c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -19,8 +19,8 @@ package org.apache.pulsar.proxy.server; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import static org.testng.Assert.assertEquals; import io.netty.channel.EventLoopGroup; @@ -149,7 +149,7 @@ public void testProducerConsumer() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); } @@ -183,7 +183,7 @@ public void testPartitions() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); } } @@ -192,8 +192,8 @@ public void testPartitions() throws Exception { **/ @Test public void testAutoCreateTopic() throws Exception{ - int defaultPartition=2; - int defaultNumPartitions=pulsar.getConfiguration().getDefaultNumPartitions(); + int defaultPartition = 2; + int defaultNumPartitions = pulsar.getConfiguration().getDefaultNumPartitions(); pulsar.getConfiguration().setAllowAutoTopicCreationType("partitioned"); pulsar.getConfiguration().setDefaultNumPartitions(defaultPartition); try { @@ -204,7 +204,7 @@ public void testAutoCreateTopic() throws Exception{ CompletableFuture> partitionNamesFuture = client.getPartitionsForTopic(topic); List partitionNames = partitionNamesFuture.get(30000, TimeUnit.MILLISECONDS); Assert.assertEquals(partitionNames.size(), defaultPartition); - }finally { + } finally { pulsar.getConfiguration().setAllowAutoTopicCreationType("non-partitioned"); pulsar.getConfiguration().setDefaultNumPartitions(defaultNumPartitions); } @@ -271,12 +271,13 @@ public void testGetSchema() throws Exception { Assert.fail("Should not have failed since can acquire LookupRequestSemaphore"); } byte[] schemaVersion = new byte[8]; - byte b = new Long(0l).byteValue(); - for (int i = 0; i<8; i++){ + byte b = Long.valueOf(0L).byteValue(); + for (int i = 0; i < 8; i++){ schemaVersion[i] = b; } SchemaInfo schemaInfo = ((PulsarClientImpl) client).getLookup() - .getSchema(TopicName.get("persistent://sample/test/local/get-schema"), schemaVersion).get().orElse(null); + .getSchema(TopicName.get("persistent://sample/test/local/get-schema"), schemaVersion) + .get().orElse(null); Assert.assertEquals(new String(schemaInfo.getSchema()), new String(schema.getSchemaInfo().getSchema())); } @@ -304,7 +305,7 @@ public void testProtocolVersionAdvertisement() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(10, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); consumer.acknowledge(msg); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java index 59beb94712c7d..d0115c5072ec1 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.proxy.server; -import static com.google.common.base.Preconditions.checkNotNull; +import static java.util.Objects.requireNonNull; import static org.mockito.Mockito.doReturn; import java.util.Optional; @@ -117,7 +117,7 @@ public void testPartitions() throws Exception { for (int i = 0; i < 10; i++) { Message msg = consumer.receive(1, TimeUnit.SECONDS); - checkNotNull(msg); + requireNonNull(msg); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index 5d05867d4fffd..43ad66195fea2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -210,7 +210,7 @@ public void testProxyAuthorization() throws Exception { } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index 14c72881b2994..eaec65059ba45 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.TimeUnit; import lombok.Cleanup; @@ -88,47 +89,47 @@ public Object[][] hostnameVerificationCodecProvider() { @DataProvider(name = "protocolsCiphersProvider") public Object[][] protocolsCiphersProviderCodecProvider() { // Test using defaults - Set ciphers_1 = Sets.newTreeSet(); - Set protocols_1 = Sets.newTreeSet(); + Set ciphers_1 = new TreeSet<>(); + Set protocols_1 = new TreeSet<>(); // Test explicitly specifying protocols defaults - Set ciphers_2 = Sets.newTreeSet(); - Set protocols_2 = Sets.newTreeSet(); + Set ciphers_2 = new TreeSet<>(); + Set protocols_2 = new TreeSet<>(); protocols_2.add("TLSv1.3"); protocols_2.add("TLSv1.2"); // Test for invalid ciphers - Set ciphers_3 = Sets.newTreeSet(); - Set protocols_3 = Sets.newTreeSet(); + Set ciphers_3 = new TreeSet<>(); + Set protocols_3 = new TreeSet<>(); ciphers_3.add("INVALID_PROTOCOL"); // Incorrect Config since TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256 was introduced in TLSv1.2 - Set ciphers_4 = Sets.newTreeSet(); - Set protocols_4 = Sets.newTreeSet(); + Set ciphers_4 = new TreeSet<>(); + Set protocols_4 = new TreeSet<>(); ciphers_4.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); protocols_4.add("TLSv1.1"); // Incorrect Config since TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256 was introduced in TLSv1.2 - Set ciphers_5 = Sets.newTreeSet(); - Set protocols_5 = Sets.newTreeSet(); + Set ciphers_5 = new TreeSet<>(); + Set protocols_5 = new TreeSet<>(); ciphers_5.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); protocols_5.add("TLSv1"); // Correct Config - Set ciphers_6 = Sets.newTreeSet(); - Set protocols_6 = Sets.newTreeSet(); + Set ciphers_6 = new TreeSet<>(); + Set protocols_6 = new TreeSet<>(); ciphers_6.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); protocols_6.add("TLSv1.2"); // In correct config - JDK 8 doesn't support TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384 - Set ciphers_7 = Sets.newTreeSet(); - Set protocols_7 = Sets.newTreeSet(); + Set ciphers_7 = new TreeSet<>(); + Set protocols_7 = new TreeSet<>(); protocols_7.add("TLSv1.2"); ciphers_7.add("TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"); // Correct config - Atlease one of the Cipher Suite is supported - Set ciphers_8 = Sets.newTreeSet(); - Set protocols_8 = Sets.newTreeSet(); + Set ciphers_8 = new TreeSet<>(); + Set protocols_8 = new TreeSet<>(); protocols_8.add("TLSv1.2"); ciphers_8.add("TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256"); ciphers_8.add("TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384"); @@ -265,7 +266,7 @@ public void testProxyAuthorization() throws Exception { } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 693e4ca5db9d6..17581feddd7f9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -185,7 +185,7 @@ public void testProxyAuthorization() throws Exception { } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); @@ -227,7 +227,7 @@ public void testUpdatePartitionNumAndReconnect() throws Exception { admin.clusters().createCluster(clusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); admin.tenants().createTenant("my-property", - new TenantInfoImpl(Sets.newHashSet(), Sets.newHashSet(clusterName))); + new TenantInfoImpl(new HashSet<>(), Sets.newHashSet(clusterName))); admin.namespaces().createNamespace(namespaceName); admin.topics().createPartitionedTopic(topicName, 2); admin.topics().grantPermission(topicName, CLIENT_ROLE, @@ -240,7 +240,7 @@ public void testUpdatePartitionNumAndReconnect() throws Exception { Producer producer = proxyClient.newProducer(Schema.BYTES) .topic(topicName).create(); final int MSG_NUM = 10; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); for (int i = 0; i < MSG_NUM; i++) { String message = "my-message-" + i; messageSet.add(message); @@ -248,7 +248,7 @@ public void testUpdatePartitionNumAndReconnect() throws Exception { } Message msg; - Set receivedMessageSet = Sets.newHashSet(); + Set receivedMessageSet = new HashSet<>(); for (int i = 0; i < MSG_NUM; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); String receivedMessage = new String(msg.getData()); @@ -347,7 +347,7 @@ public void testProxyAuthorizationWithPrefixSubscriptionAuthMode() throws Except } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java index f20401c33aebf..7c01eff480f87 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java @@ -180,7 +180,7 @@ public void testDiscoveryService() throws Exception { } Message msg = null; - Set messageSet = Sets.newHashSet(); + Set messageSet = new HashSet<>(); int count = 0; for (int i = 0; i < 10; i++) { msg = consumer.receive(5, TimeUnit.SECONDS); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java index 5d8b0ebf2fd3f..0ca5b93730585 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/proxy/socket/client/PerformanceClient.java @@ -212,8 +212,6 @@ public Arguments loadArguments(String[] args) { arguments.proxyURL += "/"; } - arguments.testTime = TimeUnit.SECONDS.toMillis(arguments.testTime); - return arguments; } @@ -295,10 +293,17 @@ public void runPerformanceTest(Arguments arguments) throws InterruptedException, executor.submit(() -> { try { RateLimiter rateLimiter = RateLimiter.create(arguments.msgRate); + long startTime = System.nanoTime(); + long testEndTime = startTime + (long) (arguments.testTime * 1e9); // Send messages on all topics/producers long totalSent = 0; while (true) { for (String topic : producersMap.keySet()) { + if (arguments.testTime > 0 && System.nanoTime() > testEndTime) { + log.info("------------- DONE (reached the maximum duration: [{} seconds] of production) --------------", arguments.testTime); + PerfClientUtils.exit(0); + } + if (arguments.numMessages > 0) { if (totalSent >= arguments.numMessages) { log.trace("------------- DONE (reached the maximum number: [{}] of production) --------------", arguments.numMessages); diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java index 52ba73e6781b8..2f571add41594 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceConsumer.java @@ -212,7 +212,7 @@ public static void main(String[] args) throws Exception { if (arguments.topic != null && arguments.topic.size() != arguments.numTopics) { // keep compatibility with the previous version if (arguments.topic.size() == 1) { - String prefixTopicName = TopicName.get(arguments.topic.get(0)).toString(); + String prefixTopicName = TopicName.get(arguments.topic.get(0)).toString().trim(); List defaultTopics = Lists.newArrayList(); for (int i = 0; i < arguments.numTopics; i++) { defaultTopics.add(String.format("%s-%d", prefixTopicName, i)); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java index a1c76d26fd2c4..d30c0844809de 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ConsumerHandler.java @@ -232,6 +232,10 @@ public void onWebSocketText(String message) { // Check and notify consumer if reached end of topic. private void handleEndOfTopic() { + if (log.isDebugEnabled()) { + log.debug("[{}/{}] Received check reach the end of topic request from {} ", consumer.getTopic(), + subscription, getRemote().getInetSocketAddress().toString()); + } try { String msg = ObjectMapperFactory.getThreadLocal().writeValueAsString( new EndOfTopicResponse(consumer.hasReachedEndOfTopic())); @@ -259,6 +263,10 @@ public void writeSuccess() { } private void handleUnsubscribe(ConsumerCommand command) throws PulsarClientException { + if (log.isDebugEnabled()) { + log.debug("[{}/{}] Received unsubscribe request from {} ", consumer.getTopic(), + subscription, getRemote().getInetSocketAddress().toString()); + } consumer.unsubscribe(); } @@ -276,6 +284,10 @@ private void handleAck(ConsumerCommand command) throws IOException { // We should have received an ack MessageId msgId = MessageId.fromByteArrayWithTopic(Base64.getDecoder().decode(command.messageId), topic.toString()); + if (log.isDebugEnabled()) { + log.debug("[{}/{}] Received ack request of message {} from {} ", consumer.getTopic(), + subscription, msgId, getRemote().getInetSocketAddress().toString()); + } consumer.acknowledgeAsync(msgId).thenAccept(consumer -> numMsgsAcked.increment()); checkResumeReceive(); } @@ -283,11 +295,19 @@ private void handleAck(ConsumerCommand command) throws IOException { private void handleNack(ConsumerCommand command) throws IOException { MessageId msgId = MessageId.fromByteArrayWithTopic(Base64.getDecoder().decode(command.messageId), topic.toString()); + if (log.isDebugEnabled()) { + log.debug("[{}/{}] Received negative ack request of message {} from {} ", consumer.getTopic(), + subscription, msgId, getRemote().getInetSocketAddress().toString()); + } consumer.negativeAcknowledge(msgId); checkResumeReceive(); } private void handlePermit(ConsumerCommand command) throws IOException { + if (log.isDebugEnabled()) { + log.debug("[{}/{}] Received {} permits request from {} ", consumer.getTopic(), + subscription, command.permitMessages, getRemote().getInetSocketAddress().toString()); + } if (command.permitMessages == null) { throw new IOException("Missing required permitMessages field for 'permit' command"); } diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java index 9552d42462cb6..9b6593d0274fe 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/ProducerHandler.java @@ -131,6 +131,10 @@ public void close() throws IOException { @Override public void onWebSocketText(String message) { + if (log.isDebugEnabled()) { + log.debug("[{}] Received new message from producer {} ", producer.getTopic(), + getRemote().getInetSocketAddress().toString()); + } ProducerMessage sendRequest; byte[] rawPayload = null; String requestContext = null; @@ -188,6 +192,10 @@ public void onWebSocketText(String message) { final long now = System.nanoTime(); builder.sendAsync().thenAccept(msgId -> { + if (log.isDebugEnabled()) { + log.debug("[{}] Success fully write the message to broker with returned message ID {} from producer {}", + producer.getTopic(), msgId, getRemote().getInetSocketAddress().toString()); + } updateSentMsgStats(msgSize, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - now)); if (isConnected()) { String messageId = Base64.getEncoder().encodeToString(msgId.toByteArray()); diff --git a/site2/docs/admin-api-topics.md b/site2/docs/admin-api-topics.md index f339c2fc4269f..c6e84a95991cb 100644 --- a/site2/docs/admin-api-topics.md +++ b/site2/docs/admin-api-topics.md @@ -811,7 +811,7 @@ $ pulsar-admin topics reset-cursor \ String topic = "persistent://my-tenant/my-namespace/my-topic"; String subName = "my-subscription"; long timestamp = 2342343L; -admin.topics().skipAllMessages(topic, subName, timestamp); +admin.topics().resetCursor(topic, subName, timestamp); ``` diff --git a/site2/docs/cookbooks-tiered-storage.md b/site2/docs/cookbooks-tiered-storage.md index 0263ec2596250..731a7cb93d5a6 100644 --- a/site2/docs/cookbooks-tiered-storage.md +++ b/site2/docs/cookbooks-tiered-storage.md @@ -306,5 +306,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/docs/functions-package.md b/site2/docs/functions-package.md index 0bdf0836995e9..6158eba2ed400 100644 --- a/site2/docs/functions-package.md +++ b/site2/docs/functions-package.md @@ -200,7 +200,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -216,7 +216,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/docs/io-develop.md b/site2/docs/io-develop.md index 2cff78b8f0cd2..3c5af1873b192 100644 --- a/site2/docs/io-develop.md +++ b/site2/docs/io-develop.md @@ -272,7 +272,7 @@ a bit of Java ClassLoader isolation. > For more information about **how NAR works**, see > [here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). -Pulsar uses the same mechanism for packaging **all** [built-in connectors](io-connectors). +Pulsar uses the same mechanism for packaging **all** [built-in connectors](io-connectors.md). The easiest approach to package a Pulsar connector is to create a NAR package using [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin). diff --git a/site2/docs/io-influxdb-sink.md b/site2/docs/io-influxdb-sink.md index fe12de76451c8..9cc69fd56278d 100644 --- a/site2/docs/io-influxdb-sink.md +++ b/site2/docs/io-influxdb-sink.md @@ -61,7 +61,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -71,7 +71,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -94,7 +93,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -103,5 +102,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/docs/io-quickstart.md b/site2/docs/io-quickstart.md index 75b48d18894ac..e354896614314 100644 --- a/site2/docs/io-quickstart.md +++ b/site2/docs/io-quickstart.md @@ -558,7 +558,7 @@ This sink connector runs as a Pulsar Function and writes the messages produced i The sink has been created successfully if the following message appears. ```bash -"Created successfully" +Created successfully ``` ### Inspect a JDBC sink @@ -682,7 +682,7 @@ $ bin/pulsar-admin sinks stop \ The sink instance has been stopped successfully if the following message disappears. ```bash -"Stopped successfully" +Stopped successfully ``` ### Restart a JDBC sink @@ -704,7 +704,7 @@ $ bin/pulsar-admin sinks restart \ The sink instance has been started successfully if the following message disappears. ```bash -"Started successfully" +Started successfully ``` > #### Tip @@ -735,7 +735,7 @@ $ bin/pulsar-admin sinks update \ The sink connector has been updated successfully if the following message disappears. ```bash -"Updated successfully" +Updated successfully ``` This example double-checks the information. @@ -794,7 +794,7 @@ $ bin/pulsar-admin sinks delete \ The sink connector has been deleted successfully if the following message appears. ```text -"Deleted successfully" +Deleted successfully ``` This example double-checks the status of the sink connector. diff --git a/site2/docs/reference-configuration.md b/site2/docs/reference-configuration.md index 72f134ae7286f..d1a917c74d0d7 100644 --- a/site2/docs/reference-configuration.md +++ b/site2/docs/reference-configuration.md @@ -482,6 +482,8 @@ You can set the log level and configuration in the [log4j2.yaml](https://github |subscribeRatePeriodPerConsumerInSecond|Rate period for {subscribeThrottlingRatePerConsumer}. By default, it is 30s.|30| | dispatchThrottlingRatePerTopicInMsg | Default messages (per second) dispatch throttling-limit for every topic. When the value is set to 0, default message dispatch throttling-limit is disabled. |0 | | dispatchThrottlingRatePerTopicInByte | Default byte (per second) dispatch throttling-limit for every topic. When the value is set to 0, default byte dispatch throttling-limit is disabled. | 0| +| dispatchThrottlingOnBatchMessageEnabled |Apply dispatch rate limiting on batch message instead individual messages with in batch message. (Default is disabled). | false| + | dispatchThrottlingRateRelativeToPublishRate | Enable dispatch rate-limiting relative to publish rate. | false | |dispatchThrottlingRatePerSubscriptionInMsg|The defaulted number of message dispatching throttling-limit for a subscription. The value of 0 disables message dispatch-throttling.|0| |dispatchThrottlingRatePerSubscriptionInByte|The default number of message-bytes dispatching throttling-limit for a subscription. The value of 0 disables message-byte dispatch-throttling.|0| diff --git a/site2/docs/reference-metrics.md b/site2/docs/reference-metrics.md index 3467931e86b98..992906bc984ae 100644 --- a/site2/docs/reference-metrics.md +++ b/site2/docs/reference-metrics.md @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -120,6 +120,7 @@ The following metrics are available for broker: - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -303,20 +304,20 @@ brk_ml_cursor_readLedgerSize(namespace="", ledger_name="", cursor_name:"")|Gauge All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -326,7 +327,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -437,6 +438,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -489,10 +519,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/docs/security-encryption.md b/site2/docs/security-encryption.md index 0ce319cce0811..2229e590914b4 100644 --- a/site2/docs/security-encryption.md +++ b/site2/docs/security-encryption.md @@ -39,63 +39,148 @@ openssl ec -in test_ecdsa_privkey.pem -pubout -outform pem -out test_ecdsa_pubke 4. Add encryption key name to producer builder: PulsarClient.newProducer().addEncryptionKey("myapp.key"). -5. Add CryptoKeyReader implementation to producer or consumer builder: PulsarClient.newProducer().cryptoKeyReader(keyReader) / PulsarClient.newConsumer().cryptoKeyReader(keyReader). +5. Configure a `CryptoKeyReader` to a producer, consumer or reader. -6. Sample producer application: + + ```java -class RawFileKeyReader implements CryptoKeyReader { - - String publicKeyFile = ""; - String privateKeyFile = ""; +PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); +String topic = "persistent://my-tenant/my-ns/my-topic"; +// RawFileKeyReader is just an example implementation that's not provided by Pulsar +CryptoKeyReader keyReader = new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem"); + +Producer producer = pulsarClient.newProducer() + .topic(topic) + .cryptoKeyReader(keyReader) + .addEncryptionKey(“myappkey”) + .create(); + +Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName("my-subscriber-name") + .cryptoKeyReader(keyReader) + .subscribe(); + +Reader reader = pulsarClient.newReader() + .topic(topic) + .startMessageId(MessageId.earliest) + .cryptoKeyReader(keyReader) + .create(); +``` - RawFileKeyReader(String pubKeyFile, String privKeyFile) { - publicKeyFile = pubKeyFile; - privateKeyFile = privKeyFile; - } + +```c++ +Client client("pulsar://localhost:6650"); +std::string topic = "persistent://my-tenant/my-ns/my-topic"; +// DefaultCryptoKeyReader is a built-in implementation that reads public key and private key from files +auto keyReader = std::make_shared("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem"); + +Producer producer; +ProducerConfiguration producerConf; +producerConf.setCryptoKeyReader(keyReader); +producerConf.addEncryptionKey("myappkey"); +client.createProducer(topic, producerConf, producer); + +Consumer consumer; +ConsumerConfiguration consumerConf; +consumerConf.setCryptoKeyReader(keyReader); +client.subscribe(topic, "my-subscriber-name", consumerConf, consumer); + +Reader reader; +ReaderConfiguration readerConf; +readerConf.setCryptoKeyReader(keyReader); +client.createReader(topic, MessageId::earliest(), readerConf, reader); +``` - @Override - public EncryptionKeyInfo getPublicKey(String keyName, Map keyMeta) { - EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); - try { - keyInfo.setKey(Files.readAllBytes(Paths.get(publicKeyFile))); - } catch (IOException e) { - System.out.println("ERROR: Failed to read public key from file " + publicKeyFile); - e.printStackTrace(); - } - return keyInfo; - } + +```python +from pulsar import Client, CryptoKeyReader + +client = Client('pulsar://localhost:6650') +topic = 'persistent://my-tenant/my-ns/my-topic' +# CryptoKeyReader is a built-in implementation that reads public key and private key from files +key_reader = CryptoKeyReader('test_ecdsa_pubkey.pem', 'test_ecdsa_privkey.pem') + +producer = client.create_producer( + topic=topic, + encryption_key='myappkey', + crypto_key_reader=key_reader +) + +consumer = client.subscribe( + topic=topic, + subscription_name='my-subscriber-name', + crypto_key_reader=key_reader +) + +reader = client.create_reader( + topic=topic, + start_message_id=MessageId.earliest, + crypto_key_reader=key_reader +) + +client.close() +``` - @Override - public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMeta) { - EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); - try { - keyInfo.setKey(Files.readAllBytes(Paths.get(privateKeyFile))); - } catch (IOException e) { - System.out.println("ERROR: Failed to read private key from file " + privateKeyFile); - e.printStackTrace(); - } - return keyInfo; - } + +```nodejs +const Pulsar = require('pulsar-client'); + +(async () => { +// Create a client +const client = new Pulsar.Client({ + serviceUrl: 'pulsar://localhost:6650', + operationTimeoutSeconds: 30, +}); + +// Create a producer +const producer = await client.createProducer({ + topic: 'persistent://public/default/my-topic', + sendTimeoutMs: 30000, + batchingEnabled: true, + publicKeyPath: "public-key.client-rsa.pem", + encryptionKey: "encryption-key" +}); + +// Create a consumer +const consumer = await client.subscribe({ + topic: 'persistent://public/default/my-topic', + subscription: 'sub1', + subscriptionType: 'Shared', + ackTimeoutMs: 10000, + privateKeyPath: "private-key.client-rsa.pem" +}); + +// Send messages +for (let i = 0; i < 10; i += 1) { + const msg = `my-message-${i}`; + producer.send({ + data: Buffer.from(msg), + }); + console.log(`Sent message: ${msg}`); } +await producer.flush(); -PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); - -Producer producer = pulsarClient.newProducer() - .topic("persistent://my-tenant/my-ns/my-topic") - .addEncryptionKey("myappkey") - .cryptoKeyReader(new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem")) - .create(); - -for (int i = 0; i < 10; i++) { - producer.send("my-message".getBytes()); +// Receive messages +for (let i = 0; i < 10; i += 1) { + const msg = await consumer.receive(); + console.log(msg.getData().toString()); + consumer.acknowledge(msg); } -producer.close(); -pulsarClient.close(); +await consumer.close(); +await producer.close(); +await client.close(); +})(); ``` -7. Sample Consumer Application: + + +6. Below is an example of a **customized** `CryptoKeyReader` implementation. + + + ```java class RawFileKeyReader implements CryptoKeyReader { @@ -131,27 +216,39 @@ class RawFileKeyReader implements CryptoKeyReader { return keyInfo; } } +``` -PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); -Consumer consumer = pulsarClient.newConsumer() - .topic("persistent://my-tenant/my-ns/my-topic") - .subscriptionName("my-subscriber-name") - .cryptoKeyReader(new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem")) - .subscribe(); -Message msg = null; - -for (int i = 0; i < 10; i++) { - msg = consumer.receive(); - // do something - System.out.println("Received: " + new String(msg.getData())); -} + +```c++ +class CustomCryptoKeyReader : public CryptoKeyReader { + public: + Result getPublicKey(const std::string& keyName, std::map& metadata, + EncryptionKeyInfo& encKeyInfo) const override { + // TODO: + return ResultOk; + } -// Acknowledge the consumption of all messages at once -consumer.acknowledgeCumulative(msg); -consumer.close(); -pulsarClient.close(); + Result getPrivateKey(const std::string& keyName, std::map& metadata, + EncryptionKeyInfo& encKeyInfo) const override { + // TODO: + return ResultOk; + } +}; + +auto keyReader = std::make_shared(/* ... */); +// TODO: create producer, consumer or reader based on keyReader here ``` +Besides, you can use the **default** implementation of `CryptoKeyReader` by specifying the paths of `private key` and `public key`. + + +Currently, **customized** `CryptoKeyReader` implementation is not supported in Python. However, you can use the **default** implementation by specifying the path of `private key` and `public key`. + + +Currently, **customized** `CryptoKeyReader` implementation is not supported in Node.JS. However, you can use the **default** implementation by specifying the path of `private key` and `public key`. + + + ## Key rotation Pulsar generates a new AES data key every 4 hours or after publishing a certain number of messages. A producer fetches the asymmetric public key every 4 hours by calling CryptoKeyReader.getPublicKey() to retrieve the latest version. diff --git a/site2/docs/tiered-storage-aliyun.md b/site2/docs/tiered-storage-aliyun.md index 5b8f07d6a44a5..c131842e9bc13 100644 --- a/site2/docs/tiered-storage-aliyun.md +++ b/site2/docs/tiered-storage-aliyun.md @@ -210,7 +210,7 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > **Tip** > diff --git a/site2/docs/tiered-storage-aws.md b/site2/docs/tiered-storage-aws.md index 2c3915a3375a8..35348e3b7dbf0 100644 --- a/site2/docs/tiered-storage-aws.md +++ b/site2/docs/tiered-storage-aws.md @@ -270,7 +270,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/docs/tiered-storage-azure.md b/site2/docs/tiered-storage-azure.md index 53816f90c001b..1c224fe00ab88 100644 --- a/site2/docs/tiered-storage-azure.md +++ b/site2/docs/tiered-storage-azure.md @@ -217,7 +217,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/docs/tiered-storage-filesystem.md b/site2/docs/tiered-storage-filesystem.md index 50274ead8e853..ba4cc6ba59210 100644 --- a/site2/docs/tiered-storage-filesystem.md +++ b/site2/docs/tiered-storage-filesystem.md @@ -257,7 +257,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/docs/tiered-storage-gcs.md b/site2/docs/tiered-storage-gcs.md index 68a9d05a342fc..02f96184c15f5 100644 --- a/site2/docs/tiered-storage-gcs.md +++ b/site2/docs/tiered-storage-gcs.md @@ -262,7 +262,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website-next/docs/adaptors-kafka.md b/site2/website-next/docs/adaptors-kafka.md index b2db0e6b579c2..aa3735730d548 100644 --- a/site2/website-next/docs/adaptors-kafka.md +++ b/site2/website-next/docs/adaptors-kafka.md @@ -1,7 +1,7 @@ --- id: adaptors-kafka title: Pulsar adaptor for Apache Kafka -sidebar_label: Kafka client wrapper +sidebar_label: "Kafka client wrapper" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/adaptors-spark.md b/site2/website-next/docs/adaptors-spark.md index 0febaad0f04c8..324aaa4637417 100644 --- a/site2/website-next/docs/adaptors-spark.md +++ b/site2/website-next/docs/adaptors-spark.md @@ -1,7 +1,7 @@ --- id: adaptors-spark title: Pulsar adaptor for Apache Spark -sidebar_label: Apache Spark +sidebar_label: "Apache Spark" --- import Tabs from '@theme/Tabs'; @@ -90,4 +90,5 @@ SparkStreamingPulsarReceiver pulsarReceiver = new SparkStreamingPulsarReceiver( pulsarConf, new AuthenticationToken("token:")); -``` \ No newline at end of file +``` + diff --git a/site2/website-next/docs/adaptors-storm.md b/site2/website-next/docs/adaptors-storm.md index 6c830d84f95f7..a5ca062edf60a 100644 --- a/site2/website-next/docs/adaptors-storm.md +++ b/site2/website-next/docs/adaptors-storm.md @@ -1,7 +1,7 @@ --- id: adaptors-storm title: Pulsar adaptor for Apache Storm -sidebar_label: Apache Storm +sidebar_label: "Apache Storm" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-brokers.md b/site2/website-next/docs/admin-api-brokers.md index 8727c670291b9..c29e6e2f2612f 100644 --- a/site2/website-next/docs/admin-api-brokers.md +++ b/site2/website-next/docs/admin-api-brokers.md @@ -1,7 +1,7 @@ --- id: admin-api-brokers title: Managing Brokers -sidebar_label: Brokers +sidebar_label: "Brokers" --- import Tabs from '@theme/Tabs'; @@ -64,6 +64,7 @@ $ pulsar-admin brokers list use ``` ``` + broker1.use.org.com:8080 ``` @@ -115,6 +116,7 @@ $ pulsar-admin brokers leader-broker ``` ``` + BrokerInfo(serviceUrl=broker1.use.org.com:8080) ``` @@ -132,6 +134,7 @@ BrokerInfo(serviceUrl=broker1.use.org.com:8080) admin.brokers().getLeaderBroker() ``` + For the detail of the code above, see [here](https://github.com/apache/pulsar/blob/master/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java#L80) @@ -178,6 +181,7 @@ $ pulsar-admin brokers namespaces use \ } ``` + @@ -191,6 +195,7 @@ $ pulsar-admin brokers namespaces use \ admin.brokers().getOwnedNamespaces(cluster,brokerUrl); ``` + @@ -245,6 +250,7 @@ $ pulsar-admin brokers update-dynamic-config --config brokerShutdownTimeoutMs -- admin.brokers().updateDynamicConfiguration(configName, configValue); ``` + @@ -290,6 +296,7 @@ brokerShutdownTimeoutMs admin.brokers().getDynamicConfigurationNames(); ``` + @@ -336,6 +343,7 @@ brokerShutdownTimeoutMs:100 admin.brokers().getAllDynamicConfigurations(); ``` + diff --git a/site2/website-next/docs/admin-api-clusters.md b/site2/website-next/docs/admin-api-clusters.md index b582db832f29f..1063279771e1d 100644 --- a/site2/website-next/docs/admin-api-clusters.md +++ b/site2/website-next/docs/admin-api-clusters.md @@ -1,7 +1,7 @@ --- id: admin-api-clusters title: Managing Clusters -sidebar_label: Clusters +sidebar_label: "Clusters" --- import Tabs from '@theme/Tabs'; @@ -82,6 +82,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().createCluster(clusterName, clusterData); ``` + @@ -172,6 +173,7 @@ $ pulsar-admin clusters get cluster-1 admin.clusters().getCluster(clusterName); ``` + @@ -227,6 +229,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().updateCluster(clusterName, clusterData); ``` + @@ -256,6 +259,7 @@ Clusters can be deleted from a Pulsar [instance](reference-terminology.md#instan Use the [`delete`](reference-pulsar-admin.md#clusters-delete) subcommand and specify the name of the cluster. ``` + $ pulsar-admin clusters delete cluster-1 ``` @@ -273,6 +277,7 @@ $ pulsar-admin clusters delete cluster-1 admin.clusters().deleteCluster(clusterName); ``` + @@ -322,6 +327,7 @@ cluster-2 admin.clusters().getClusters(); ``` + @@ -351,6 +357,7 @@ Peer clusters can be configured for a given cluster in a Pulsar [instance](refer Use the [`update-peer-clusters`](reference-pulsar-admin.md#clusters-update-peer-clusters) subcommand and specify the list of peer-cluster names. ``` + $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 ``` @@ -368,6 +375,7 @@ $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 admin.clusters().updatePeerClusterNames(clusterName, peerClusterList); ``` + \ No newline at end of file diff --git a/site2/website-next/docs/admin-api-functions.md b/site2/website-next/docs/admin-api-functions.md index 91e90a91d6108..40976ecae903d 100644 --- a/site2/website-next/docs/admin-api-functions.md +++ b/site2/website-next/docs/admin-api-functions.md @@ -1,7 +1,7 @@ --- id: admin-api-functions title: Manage Functions -sidebar_label: Functions +sidebar_label: "Functions" --- import Tabs from '@theme/Tabs'; @@ -100,6 +100,7 @@ functionConfig.setOutput(sinkTopic); admin.functions().createFunction(functionConfig, fileName); ``` + @@ -163,6 +164,7 @@ updateOptions.setUpdateAuthData(updateAuthData); admin.functions().updateFunction(functionConfig, userCodeFile, updateOptions); ``` + @@ -214,6 +216,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -266,6 +269,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName); ``` + @@ -319,6 +323,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -371,6 +376,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName); ``` + @@ -424,6 +430,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -476,6 +483,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName); ``` + @@ -527,6 +535,7 @@ $ pulsar-admin functions list \ admin.functions().getFunctions(tenant, namespace); ``` + @@ -562,7 +571,7 @@ Use the [`delete`](reference-pulsar-admin.md#functions-delete) subcommand. $ pulsar-admin functions delete \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -579,6 +588,7 @@ $ pulsar-admin functions delete \ admin.functions().deleteFunction(tenant, namespace, functionName); ``` + @@ -614,7 +624,7 @@ Use the [`get`](reference-pulsar-admin.md#functions-get) subcommand. $ pulsar-admin functions get \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -631,6 +641,7 @@ $ pulsar-admin functions get \ admin.functions().getFunction(tenant, namespace, functionName); ``` + @@ -683,6 +694,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -718,7 +730,7 @@ Use the [`status`](reference-pulsar-admin.md#functions-status) subcommand. $ pulsar-admin functions status \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -735,6 +747,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName); ``` + @@ -787,6 +800,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -822,7 +836,7 @@ Use the [`stats`](reference-pulsar-admin.md#functions-stats) subcommand. $ pulsar-admin functions stats \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -839,6 +853,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName); ``` + @@ -880,6 +895,7 @@ $ pulsar-admin functions trigger \ # or --trigger-file (the path of trigger file) ``` + @@ -893,6 +909,7 @@ $ pulsar-admin functions trigger \ admin.functions().triggerFunction(tenant, namespace, functionName, topic, triggerValue, triggerFile); ``` + @@ -929,7 +946,7 @@ $ pulsar-admin functions putstate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" + --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" ``` @@ -948,6 +965,7 @@ FunctionState stateRepr = ObjectMapperFactory.getThreadLocal().readValue(state, admin.functions().putFunctionState(tenant, namespace, functionName, stateRepr); ``` + @@ -984,7 +1002,7 @@ $ pulsar-admin functions querystate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --key (the key of state) + --key (the key of state) ``` @@ -1001,6 +1019,7 @@ $ pulsar-admin functions querystate \ admin.functions().getFunctionState(tenant, namespace, functionName, key); ``` + \ No newline at end of file diff --git a/site2/website-next/docs/admin-api-namespaces.md b/site2/website-next/docs/admin-api-namespaces.md index a21f94edb7571..d3aebb8c5e7ae 100644 --- a/site2/website-next/docs/admin-api-namespaces.md +++ b/site2/website-next/docs/admin-api-namespaces.md @@ -1,7 +1,7 @@ --- id: admin-api-namespaces title: Managing Namespaces -sidebar_label: Namespaces +sidebar_label: "Namespaces" --- import Tabs from '@theme/Tabs'; @@ -62,6 +62,7 @@ $ pulsar-admin namespaces create test-tenant/test-namespace ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace|operation/createNamespace?version=@pulsar:version_number@} ``` @@ -74,6 +75,7 @@ $ pulsar-admin namespaces create test-tenant/test-namespace admin.namespaces().createNamespace(namespace); ``` + @@ -133,6 +135,7 @@ $ pulsar-admin namespaces policies test-tenant/test-namespace ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace|operation/getPolicies?version=@pulsar:version_number@} ``` @@ -145,6 +148,7 @@ $ pulsar-admin namespaces policies test-tenant/test-namespace admin.namespaces().getPolicies(namespace); ``` + @@ -185,6 +189,7 @@ test-tenant/ns2 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant|operation/getTenantNamespaces?version=@pulsar:version_number@} ``` @@ -197,6 +202,7 @@ test-tenant/ns2 admin.namespaces().getNamespaces(tenant); ``` + @@ -235,6 +241,7 @@ $ pulsar-admin namespaces delete test-tenant/ns1 ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace|operation/deleteNamespace?version=@pulsar:version_number@} ``` @@ -247,6 +254,7 @@ $ pulsar-admin namespaces delete test-tenant/ns1 admin.namespaces().deleteNamespace(namespace); ``` + @@ -276,6 +284,7 @@ You can set replication clusters for a namespace to enable Pulsar to internally ``` + $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ --clusters cl1 @@ -285,6 +294,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replication|operation/setNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -297,6 +307,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ admin.namespaces().setNamespaceReplicationClusters(namespace, clusters); ``` + @@ -324,11 +335,13 @@ You can get the list of replication clusters for a given namespace. ``` + $ pulsar-admin namespaces get-clusters test-tenant/cl1/ns1 ``` ``` + cl2 ``` @@ -337,6 +350,7 @@ cl2 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replication|operation/getNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -349,6 +363,7 @@ cl2 admin.namespaces().getNamespaceReplicationClusters(namespace) ``` + @@ -386,6 +401,7 @@ Backlog quota restriction can be taken care by defining restriction of backlog-q ``` + $ pulsar-admin namespaces set-backlog-quota --limit 10G --limitTime 36000 --policy producer_request_hold test-tenant/ns1 ``` @@ -394,6 +410,7 @@ $ pulsar-admin namespaces set-backlog-quota --limit 10G --limitTime 36000 --poli ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/setBacklogQuota?version=@pulsar:version_number@} ``` @@ -406,6 +423,7 @@ $ pulsar-admin namespaces set-backlog-quota --limit 10G --limitTime 36000 --poli admin.namespaces().setBacklogQuota(namespace, new BacklogQuota(limit, limitTime, policy)) ``` + @@ -433,6 +451,7 @@ You can get a configured backlog quota for a given namespace. ``` + $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` @@ -452,6 +471,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/backlogQuotaMap|operation/getBacklogQuotaMap?version=@pulsar:version_number@} ``` @@ -464,6 +484,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 admin.namespaces().getBacklogQuotaMap(namespace); ``` + @@ -491,6 +512,7 @@ You can remove backlog quota policies for a given namespace. ``` + $ pulsar-admin namespaces remove-backlog-quota test-tenant/ns1 ``` @@ -499,6 +521,7 @@ $ pulsar-admin namespaces remove-backlog-quota test-tenant/ns1 ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/removeBacklogQuota?version=@pulsar:version_number@} ``` @@ -511,6 +534,7 @@ $ pulsar-admin namespaces remove-backlog-quota test-tenant/ns1 admin.namespaces().removeBacklogQuota(namespace, backlogQuotaType) ``` + @@ -548,6 +572,7 @@ Persistence policies allow users to configure persistency-level for all topic me ``` + $ pulsar-admin namespaces set-persistence --bookkeeper-ack-quorum 2 --bookkeeper-ensemble 3 --bookkeeper-write-quorum 2 --ml-mark-delete-max-rate 0 test-tenant/ns1 ``` @@ -556,6 +581,7 @@ $ pulsar-admin namespaces set-persistence --bookkeeper-ack-quorum 2 --bookkeeper ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/setPersistence?version=@pulsar:version_number@} ``` @@ -568,6 +594,7 @@ $ pulsar-admin namespaces set-persistence --bookkeeper-ack-quorum 2 --bookkeeper admin.namespaces().setPersistence(namespace,new PersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum,bookkeeperAckQuorum,managedLedgerMaxMarkDeleteRate)) ``` + @@ -595,6 +622,7 @@ You can get the configured persistence policies of a given namespace. ``` + $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` @@ -614,6 +642,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/getPersistence?version=@pulsar:version_number@} ``` @@ -626,6 +655,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 admin.namespaces().getPersistence(namespace) ``` + @@ -655,6 +685,7 @@ The namespace bundle is a virtual group of topics which belong to the same names ``` + $ pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` @@ -663,6 +694,7 @@ $ pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/unload|operation/unloadNamespaceBundle?version=@pulsar:version_number@} ``` @@ -675,6 +707,7 @@ $ pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff test-tenant/ns1 admin.namespaces().unloadNamespaceBundle(namespace, bundle) ``` + @@ -702,6 +735,7 @@ One namespace bundle can contain multiple topics but can be served by only one b ``` + $ pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` @@ -710,6 +744,7 @@ $ pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff test-tenan ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/split|operation/splitNamespaceBundle?version=@pulsar:version_number@} ``` @@ -722,6 +757,7 @@ $ pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff test-tenan admin.namespaces().splitNamespaceBundle(namespace, bundle) ``` + @@ -751,6 +787,7 @@ You can configure the time to live (in seconds) duration for messages. In the ex ``` + $ pulsar-admin namespaces set-message-ttl --messageTTL 100 test-tenant/ns1 ``` @@ -759,6 +796,7 @@ $ pulsar-admin namespaces set-message-ttl --messageTTL 100 test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/setNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -771,6 +809,7 @@ $ pulsar-admin namespaces set-message-ttl --messageTTL 100 test-tenant/ns1 admin.namespaces().setNamespaceMessageTTL(namespace, messageTTL) ``` + @@ -798,11 +837,13 @@ When the message-ttl for a namespace is set, you can use the command below to ge ``` + $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` ``` + 100 ``` @@ -811,11 +852,13 @@ $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/getNamespaceMessageTTL?version=@pulsar:version_number@} ``` ``` + 100 ``` @@ -830,6 +873,7 @@ admin.namespaces().getNamespaceMessageTTL(namespace) ``` ``` + 100 ``` @@ -861,6 +905,7 @@ Remove a message TTL of the configured namespace. ``` + $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` @@ -869,6 +914,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/removeNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -881,6 +927,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + @@ -911,6 +958,7 @@ It clears all message backlog for all the topics that belong to a specific names ``` + $ pulsar-admin namespaces clear-backlog --sub my-subscription test-tenant/ns1 ``` @@ -919,6 +967,7 @@ $ pulsar-admin namespaces clear-backlog --sub my-subscription test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/clearBacklog|operation/clearNamespaceBacklogForSubscription?version=@pulsar:version_number@} ``` @@ -931,6 +980,7 @@ $ pulsar-admin namespaces clear-backlog --sub my-subscription test-tenant/ns1 admin.namespaces().clearNamespaceBacklogForSubscription(namespace, subscription) ``` + @@ -958,6 +1008,7 @@ It clears all message backlog for all the topics that belong to a specific Names ``` + $ pulsar-admin namespaces clear-backlog --bundle 0x00000000_0xffffffff --sub my-subscription test-tenant/ns1 ``` @@ -966,6 +1017,7 @@ $ pulsar-admin namespaces clear-backlog --bundle 0x00000000_0xffffffff --sub m ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/{bundle}/clearBacklog|operation?version=@pulsar:version_number@/clearNamespaceBundleBacklogForSubscription} ``` @@ -978,6 +1030,7 @@ $ pulsar-admin namespaces clear-backlog --bundle 0x00000000_0xffffffff --sub m admin.namespaces().clearNamespaceBundleBacklogForSubscription(namespace, bundle, subscription) ``` + @@ -1007,6 +1060,7 @@ Each namespace contains multiple topics and the retention size (storage size) of ``` + $ pulsar-admin namespaces set-retention --size 100 --time 10 test-tenant/ns1 ``` @@ -1015,6 +1069,7 @@ $ pulsar-admin namespaces set-retention --size 100 --time 10 test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/retention|operation/setRetention?version=@pulsar:version_number@} ``` @@ -1027,6 +1082,7 @@ $ pulsar-admin namespaces set-retention --size 100 --time 10 test-tenant/ns1 admin.namespaces().setRetention(namespace, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)) ``` + @@ -1054,6 +1110,7 @@ It shows retention information of a given namespace. ``` + $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` @@ -1071,6 +1128,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/retention|operation/getRetention?version=@pulsar:version_number@} ``` @@ -1083,6 +1141,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 admin.namespaces().getRetention(namespace) ``` + @@ -1099,9 +1158,7 @@ disables the throttling. :::note - If neither `clusterDispatchRate` nor `topicDispatchRate` is configured, dispatch throttling is disabled. - - If `topicDispatchRate` is not configured, `clusterDispatchRate` takes effect. - - If `topicDispatchRate` is configured, `topicDispatchRate` takes effect. ::: @@ -1125,6 +1182,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1136,6 +1194,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1148,6 +1207,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ admin.namespaces().setDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1175,6 +1235,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` @@ -1193,6 +1254,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1205,6 +1267,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 admin.namespaces().getDispatchRate(namespace) ``` + @@ -1237,6 +1300,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1248,6 +1312,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1260,6 +1325,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ admin.namespaces().setSubscriptionDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1287,6 +1353,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` @@ -1305,6 +1372,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1317,6 +1385,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 admin.namespaces().getSubscriptionDispatchRate(namespace) ``` + @@ -1349,6 +1418,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1360,6 +1430,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1372,6 +1443,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ admin.namespaces().setReplicatorDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1399,6 +1471,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` @@ -1417,6 +1490,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1429,6 +1503,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 admin.namespaces().getReplicatorDispatchRate(namespace) ``` + @@ -1458,6 +1533,7 @@ It shows configured `deduplicationSnapshotInterval` for a namespace (Each topic ``` + $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1466,6 +1542,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1478,6 +1555,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().getDeduplicationSnapshotInterval(namespace) ``` + @@ -1506,7 +1584,8 @@ Set configured `deduplicationSnapshotInterval` for a namespace. Each topic under ``` -$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 + +$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 ``` @@ -1514,6 +1593,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1534,6 +1614,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- admin.namespaces().setDeduplicationSnapshotInterval(namespace, 1000) ``` + @@ -1561,6 +1642,7 @@ Remove configured `deduplicationSnapshotInterval` of a namespace (Each topic und ``` + $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1569,6 +1651,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1581,6 +1664,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().removeDeduplicationSnapshotInterval(namespace) ``` + @@ -1625,6 +1709,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/unload|operation/unloadNamespace?version=@pulsar:version_number@} ``` @@ -1637,6 +1722,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns admin.namespaces().unload(namespace) ``` + diff --git a/site2/website-next/docs/admin-api-non-partitioned-topics.md b/site2/website-next/docs/admin-api-non-partitioned-topics.md index 23043df3547e3..1461ec6b92e4c 100644 --- a/site2/website-next/docs/admin-api-non-partitioned-topics.md +++ b/site2/website-next/docs/admin-api-non-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-partitioned-topics title: Managing non-partitioned topics -sidebar_label: Non-partitioned topics +sidebar_label: "Non-partitioned topics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-non-persistent-topics.md b/site2/website-next/docs/admin-api-non-persistent-topics.md index 389e70cacd8c1..a9f7e12040cbe 100644 --- a/site2/website-next/docs/admin-api-non-persistent-topics.md +++ b/site2/website-next/docs/admin-api-non-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-persistent-topics title: Managing non-persistent topics -sidebar_label: Non-Persistent topics +sidebar_label: "Non-Persistent topics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-overview.md b/site2/website-next/docs/admin-api-overview.md index b23d5e4ce1ca8..d1e2fb6f27094 100644 --- a/site2/website-next/docs/admin-api-overview.md +++ b/site2/website-next/docs/admin-api-overview.md @@ -1,7 +1,7 @@ --- id: admin-api-overview title: Pulsar admin interface -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -14,11 +14,11 @@ You can interact with the admin interface via: - The `pulsar-admin` CLI tool, which is available in the `bin` folder of your Pulsar installation: - ```shell - - bin/pulsar-admin - - ``` + ```shell + + bin/pulsar-admin + + ``` > **Important** > @@ -121,6 +121,7 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + @@ -141,12 +142,10 @@ Kubernetes requires a name that can be used as a DNS subdomain name as defined i - Dots (.) - Replace beginning and ending non-alphanumeric characters with 0 - + :::tip - - If you get an error in translating Pulsar object names into Kubernetes resource labels (for example, you may have a naming collision if your Pulsar object name is too long) or want to customize the translating rules, see [customize Kubernetes runtime](https://pulsar.apache.org/docs/en/next/functions-runtime/#customize-kubernetes-runtime). - - For how to configure Kubernetes runtime, see [here](https://pulsar.apache.org/docs/en/next/functions-runtime/#configure-kubernetes-runtime). ::: diff --git a/site2/website-next/docs/admin-api-packages.md b/site2/website-next/docs/admin-api-packages.md index 2935ceff5612c..0bb5f9739bc3b 100644 --- a/site2/website-next/docs/admin-api-packages.md +++ b/site2/website-next/docs/admin-api-packages.md @@ -1,7 +1,7 @@ --- id: admin-api-packages title: Manage packages -sidebar_label: Packages +sidebar_label: "Packages" --- import Tabs from '@theme/Tabs'; @@ -114,6 +114,7 @@ bin/pulsar-admin packages upload functions://public/default/example@v0.1 --path + Upload a package to the package management service synchronously. ```java @@ -121,6 +122,7 @@ Upload a package to the package management service synchronously. void upload(PackageMetadata metadata, String packageName, String path) throws PulsarAdminException; ``` + Upload a package to the package management service asynchronously. ```java @@ -128,6 +130,7 @@ Upload a package to the package management service asynchronously. CompletableFuture uploadAsync(PackageMetadata metadata, String packageName, String path); ``` + @@ -166,6 +169,7 @@ bin/pulsar-admin packages download functions://public/default/example@v0.1 --pat + Download a package to the package management service synchronously. ```java @@ -181,6 +185,7 @@ Download a package to the package management service asynchronously. CompletableFuture downloadAsync(String packageName, String path); ``` + @@ -218,6 +223,7 @@ bin/pulsar-admin packages list --type function public/default + List all versions of a package synchronously. ```java @@ -225,6 +231,7 @@ List all versions of a package synchronously. List listPackageVersions(String packageName) throws PulsarAdminException; ``` + List all versions of a package asynchronously. ```java @@ -232,6 +239,7 @@ List all versions of a package asynchronously. CompletableFuture> listPackageVersionsAsync(String packageName); ``` + @@ -269,6 +277,7 @@ bin/pulsar-admin packages list --type function public/default + List all the packages with the given type in a namespace synchronously. ```java @@ -276,6 +285,7 @@ List all the packages with the given type in a namespace synchronously. List listPackages(String type, String namespace) throws PulsarAdminException; ``` + List all the packages with the given type in a namespace asynchronously. ```java @@ -283,6 +293,7 @@ List all the packages with the given type in a namespace asynchronously. CompletableFuture> listPackagesAsync(String type, String namespace); ``` + @@ -321,6 +332,7 @@ bin/pulsar-admin packages get-metadata function://public/default/test@v1 + Get the metadata of a package synchronously. ```java @@ -328,6 +340,7 @@ Get the metadata of a package synchronously. PackageMetadata getMetadata(String packageName) throws PulsarAdminException; ``` + Get the metadata of a package asynchronously. ```java @@ -335,6 +348,7 @@ Get the metadata of a package asynchronously. CompletableFuture getMetadataAsync(String packageName); ``` + @@ -372,6 +386,7 @@ bin/pulsar-admin packages update-metadata function://public/default/example@v0.1 + Update a package metadata information synchronously. ```java @@ -379,6 +394,7 @@ Update a package metadata information synchronously. void updateMetadata(String packageName, PackageMetadata metadata) throws PulsarAdminException; ``` + Update a package metadata information asynchronously. ```java @@ -386,6 +402,7 @@ Update a package metadata information asynchronously. CompletableFuture updateMetadataAsync(String packageName, PackageMetadata metadata); ``` + @@ -410,6 +427,7 @@ You can delete a specified package with its package name in the following ways. } ]}> + The following command example deletes a package of version 0.1. ```shell @@ -425,6 +443,7 @@ bin/pulsar-admin packages delete functions://public/default/example@v0.1 + Delete a specified package synchronously. ```java @@ -432,6 +451,7 @@ Delete a specified package synchronously. void delete(String packageName) throws PulsarAdminException; ``` + Delete a specified package asynchronously. ```java @@ -439,6 +459,7 @@ Delete a specified package asynchronously. CompletableFuture deleteAsync(String packageName); ``` + diff --git a/site2/website-next/docs/admin-api-partitioned-topics.md b/site2/website-next/docs/admin-api-partitioned-topics.md index eda1b6972006b..6359aa368788d 100644 --- a/site2/website-next/docs/admin-api-partitioned-topics.md +++ b/site2/website-next/docs/admin-api-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-partitioned-topics title: Managing partitioned topics -sidebar_label: Partitioned topics +sidebar_label: "Partitioned topics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-permissions.md b/site2/website-next/docs/admin-api-permissions.md index c5465a46e36ac..7d315eec97c6d 100644 --- a/site2/website-next/docs/admin-api-permissions.md +++ b/site2/website-next/docs/admin-api-permissions.md @@ -1,7 +1,7 @@ --- id: admin-api-permissions title: Managing permissions -sidebar_label: Permissions +sidebar_label: "Permissions" --- import Tabs from '@theme/Tabs'; @@ -105,6 +105,7 @@ Roles `my.1.role`, `my.2.role`, `my.foo.role`, `my.bar.role`, etc. **cannot** pr admin.namespaces().grantPermissionOnNamespace(namespace, role, getAuthActions(actions)); ``` + @@ -141,7 +142,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 "produce", "consume" ] -} +} ``` @@ -158,6 +159,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 admin.namespaces().getPermissions(namespace); ``` + @@ -206,6 +208,7 @@ $ pulsar-admin namespaces revoke-permission test-tenant/ns1 \ admin.namespaces().revokePermissionsOnNamespace(namespace, role); ``` + \ No newline at end of file diff --git a/site2/website-next/docs/admin-api-persistent-topics.md b/site2/website-next/docs/admin-api-persistent-topics.md index cdec59d479db7..2b7317c7640e7 100644 --- a/site2/website-next/docs/admin-api-persistent-topics.md +++ b/site2/website-next/docs/admin-api-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-persistent-topics title: Managing persistent topics -sidebar_label: Persistent topics +sidebar_label: "Persistent topics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-schemas.md b/site2/website-next/docs/admin-api-schemas.md index bc368edebd3fd..9d10c08cfc147 100644 --- a/site2/website-next/docs/admin-api-schemas.md +++ b/site2/website-next/docs/admin-api-schemas.md @@ -1,7 +1,7 @@ --- id: admin-api-schemas title: Managing Schemas -sidebar_label: Schemas +sidebar_label: "Schemas" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/admin-api-tenants.md b/site2/website-next/docs/admin-api-tenants.md index 9d7696a9f7ffc..543183d456bb2 100644 --- a/site2/website-next/docs/admin-api-tenants.md +++ b/site2/website-next/docs/admin-api-tenants.md @@ -1,7 +1,7 @@ --- id: admin-api-tenants title: Managing Tenants -sidebar_label: Tenants +sidebar_label: "Tenants" --- import Tabs from '@theme/Tabs'; @@ -70,6 +70,7 @@ my-tenant-2 admin.tenants().getTenants(); ``` + @@ -115,6 +116,7 @@ $ pulsar-admin tenants create my-tenant \ -r role1 ``` + @@ -172,6 +174,7 @@ $ pulsar-admin tenants get my-tenant } ``` + @@ -233,6 +236,7 @@ $ pulsar-admin tenants delete my-tenant admin.Tenants().deleteTenant(tenantName); ``` + diff --git a/site2/website-next/docs/admin-api-topics.md b/site2/website-next/docs/admin-api-topics.md index a73f328f81d33..ffd020257a2f3 100644 --- a/site2/website-next/docs/admin-api-topics.md +++ b/site2/website-next/docs/admin-api-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-topics title: Manage topics -sidebar_label: Topics +sidebar_label: "Topics" --- import Tabs from '@theme/Tabs'; @@ -33,6 +33,7 @@ Non-persistent topics are used in applications that only consume real-time publi non-persistent://tenant/namespace/topic ``` + ## Manage topic resources Whether it is persistent or non-persistent topic, you can obtain the topic resources through `pulsar-admin` tool, REST API and Java. @@ -86,6 +87,7 @@ String namespace = "my-tenant/my-namespace"; admin.topics().getList(namespace); ``` + @@ -461,6 +463,10 @@ You can check the following statistics of a given non-partitioned topic. - **replicated**: Mark that the subscription state is kept in sync across different regions. + - **allowOutOfOrderDelivery**: Whether out of order delivery is allowed on the Key_Shared subscription. + + - **keySharedMode**: Whether the Key_Shared subscription mode is AUTO_SPLIT or STICKY. + - **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. @@ -607,6 +613,7 @@ The following is an example of a topic status. "connectedSince" : "2021-06-09T17:22:45.353+08:00", "clientVersion" : "2.9.0-SNAPSHOT" } ], + "allowOutOfOrderDelivery": false, "consumersAfterMarkDeletePosition" : { }, "nonContiguousDeletedMessagesRanges" : 0, "nonContiguousDeletedMessagesRangesSerializedSize" : 0, @@ -621,6 +628,7 @@ The following is an example of a topic status. } ``` + To get the status of a topic, you can use the following ways. + When you create non-partitioned topics with the [`create`](reference-pulsar-admin.md#create-3) command, you need to specify the topic name as an argument. ```shell @@ -1497,6 +1507,7 @@ $ bin/pulsar-admin topics create \ persistent://my-tenant/my-namespace/my-topic ``` + :::note When you create a non-partitioned topic with the suffix '-partition-' followed by numeric value like 'xyz-topic-partition-x' for the topic name, if a partitioned topic with same suffix 'xyz-topic-partition-y' exists, then the numeric value(x) for the non-partitioned topic must be larger than the number of partitions(y) of the partitioned topic. Otherwise, you cannot create such a non-partitioned topic. @@ -1652,6 +1663,7 @@ You can check the current statistics of a given topic. The following is an examp } ``` + You can check the current statistics of a given topic and its connected producers and consumers in the following ways. @@ -1725,6 +1738,7 @@ You can create partitioned topics in the following ways. } ]}> + When you create partitioned topics with the [`create-partitioned-topic`](reference-pulsar-admin.md#create-partitioned-topic) command, you need to specify the topic name as an argument and the number of partitions using the `-p` or `--partitions` flag. @@ -1783,6 +1797,7 @@ When topic auto-creation is disabled, and you have a partitioned topic without a } ]}> + You can create missed partitions with the [`create-missed-partitions`](reference-pulsar-admin.md#create-missed-partitions) command and specify the topic name as an argument. ```shell @@ -1836,6 +1851,7 @@ Field | Description } ]}> + You can check the number of partitions in a partitioned topic with the [`get-partitioned-topic-metadata`](reference-pulsar-admin.md#get-partitioned-topic-metadata) subcommand. ```shell @@ -1890,6 +1906,7 @@ Producers and consumers can find the newly created partitions automatically. } ]}> + You can update partitioned topics with the [`update-partitioned-topic`](reference-pulsar-admin.md#update-partitioned-topic) command. ```shell @@ -2346,4 +2363,5 @@ If a message has a key, it supersedes the round robin routing policy. The follow return signSafeMod(PARTITION_INDEX_UPDATER.getAndIncrement(this), topicMetadata.numPartitions()); } -``` +``` + diff --git a/site2/website-next/docs/administration-dashboard.md b/site2/website-next/docs/administration-dashboard.md index 521a0b740279b..46977ede0d613 100644 --- a/site2/website-next/docs/administration-dashboard.md +++ b/site2/website-next/docs/administration-dashboard.md @@ -1,16 +1,15 @@ --- id: administration-dashboard title: Pulsar dashboard -sidebar_label: Dashboard +sidebar_label: "Dashboard" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; - :::note -Pulsar dashboard is deprecated. We recommend you use [Pulsar Manager](administration-pulsar-manager.md) to manage and monitor the stats of your topics. +Pulsar dashboard is deprecated. We recommend you use [Pulsar Manager](administration-pulsar-manager) to manage and monitor the stats of your topics. ::: @@ -25,28 +24,36 @@ You can use the [Django](https://www.djangoproject.com) web app to render the co The easiest way to use the dashboard is to run it inside a [Docker](https://www.docker.com/products/docker) container. ```shell + $ SERVICE_URL=http://broker.example.com:8080/ $ docker run -p 80:80 \ -e SERVICE_URL=$SERVICE_URL \ apachepulsar/pulsar-dashboard:@pulsar:version@ + ``` You can find the {@inject: github:Dockerfile:/dashboard/Dockerfile} in the `dashboard` directory and build an image from scratch as well: ```shell + $ docker build -t apachepulsar/pulsar-dashboard dashboard + ``` If token authentication is enabled: > Provided token should have super-user access. + ```shell + $ SERVICE_URL=http://broker.example.com:8080/ $ JWT_TOKEN=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiIxMjM0NTY3ODkwIiwibmFtZSI6IkpvaG4gRG9lIiwiaWF0IjoxNTE2MjM5MDIyfQ.SflKxwRJSMeKKF2QT4fwpMeJf36POk6yJV_adQssw5c $ docker run -p 80:80 \ -e SERVICE_URL=$SERVICE_URL \ -e JWT_TOKEN=$JWT_TOKEN \ apachepulsar/pulsar-dashboard + ``` + You need to specify only one service URL for a Pulsar cluster. Internally, the collector figures out all the existing clusters and the brokers from where it needs to pull the metrics. If you connect the dashboard to Pulsar running in standalone mode, the URL is `http://:8080` by default. `` is the IP address or hostname of the machine that runs Pulsar standalone. The IP address or hostname should be accessible from the running dashboard in the docker instance. @@ -61,7 +68,9 @@ Similarly, given the Pulsar standalone advertises itself with localhost by defau explicitly set the advertise address to the host IP address. For example: ```shell + $ bin/pulsar standalone --advertised-address 1.2.3.4 + ``` ### Known issues diff --git a/site2/website-next/docs/administration-geo.md b/site2/website-next/docs/administration-geo.md index 82c61b797647c..3e6023e12920c 100644 --- a/site2/website-next/docs/administration-geo.md +++ b/site2/website-next/docs/administration-geo.md @@ -1,7 +1,7 @@ --- id: administration-geo title: Pulsar geo-replication -sidebar_label: Geo-replication +sidebar_label: "Geo-replication" --- import Tabs from '@theme/Tabs'; @@ -64,28 +64,32 @@ Suppose that you have 3 replication clusters: `us-west`, `us-cent`, and `us-east Run the following command on `us-west`. ```shell + $ bin/pulsar-admin clusters create \ --broker-url pulsar://: \ --url http://: \ us-east -``` -:::tip +``` + :::tip -If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). + - If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). + - Different clusters may have different authentications. You can use the authentication flag `--auth-plugin` and `--auth-parameters` together to set cluster authentication, which overrides `brokerClientAuthenticationPlugin` and `brokerClientAuthenticationParameters` if `authenticationEnabled` sets to `true` in `broker.conf` and `standalone.conf`. For more information, see [authentication and authorization](concepts-authentication). -::: + ::: 2. Configure the connection from `us-west` to `us-cent`. Run the following command on `us-west`. ```shell + $ bin/pulsar-admin clusters create \ --broker-url pulsar://: \ --url http://: \ us-cent + ``` 3. Run similar commands on `us-east` and `us-cent` to create connections among clusters. @@ -97,9 +101,11 @@ To replicate to a cluster, the tenant needs permission to use that cluster. You Specify all the intended clusters when you create a tenant: ```shell + $ bin/pulsar-admin tenants create my-tenant \ --admin-roles my-admin-role \ --allowed-clusters us-west,us-east,us-cent + ``` To update permissions of an existing tenant, use `update` instead of `create`. @@ -109,14 +115,18 @@ To update permissions of an existing tenant, use `update` instead of `create`. You can create a namespace with the following command sample. ```shell + $ bin/pulsar-admin namespaces create my-tenant/my-namespace + ``` Initially, the namespace is not assigned to any cluster. You can assign the namespace to clusters using the `set-clusters` subcommand: ```shell + $ bin/pulsar-admin namespaces set-clusters my-tenant/my-namespace \ --clusters us-west,us-east,us-cent + ``` You can change the replication clusters for a namespace at any time, without disruption to ongoing traffic. Replication channels are immediately set up or stopped in all clusters as soon as the configuration changes. @@ -129,9 +139,10 @@ Once you create a geo-replication namespace, any topics that producers or consum By default, messages are replicated to all clusters configured for the namespace. You can restrict replication selectively by specifying a replication list for a message, and then that message is replicated only to the subset in the replication list. -The following is an example for the [Java API](client-libraries-java.md). Note the use of the `setReplicationClusters` method when you construct the {@inject: javadoc:Message:/client/org/apache/pulsar/client/api/Message} object: +The following is an example for the [Java API](client-libraries-java). Note the use of the `setReplicationClusters` method when you construct the {@inject: javadoc:Message:/client/org/apache/pulsar/client/api/Message} object: ```java + List restrictReplicationTo = Arrays.asList( "us-west", "us-east" @@ -145,14 +156,17 @@ producer.newMessage() .value("my-payload".getBytes()) .setReplicationClusters(restrictReplicationTo) .send(); + ``` #### Topic stats -Topic-specific statistics for geo-replication topics are available via the [`pulsar-admin`](reference-pulsar-admin.md) tool and {@inject: rest:REST:/} API: +Topic-specific statistics for geo-replication topics are available via the [`pulsar-admin`](reference-pulsar-admin) tool and {@inject: rest:REST:/} API: ```shell + $ bin/pulsar-admin persistent stats persistent://my-tenant/my-namespace/my-topic + ``` Each cluster reports its own local stats, including the incoming and outgoing replication rates and backlogs. @@ -182,11 +196,13 @@ In case of failover, a consumer can restart consuming from the failure point in Replicated subscription is disabled by default. You can enable replicated subscription when creating a consumer. ```java + Consumer consumer = client.newConsumer(Schema.STRING) .topic("my-topic") .subscriptionName("my-subscription") .replicateSubscriptionState(true) .subscribe(); + ``` ### Advantages diff --git a/site2/website-next/docs/administration-isolation.md b/site2/website-next/docs/administration-isolation.md index 65947cdac417a..a6e93ab84612e 100644 --- a/site2/website-next/docs/administration-isolation.md +++ b/site2/website-next/docs/administration-isolation.md @@ -1,7 +1,7 @@ --- id: administration-isolation title: Pulsar isolation -sidebar_label: Pulsar isolation +sidebar_label: "Pulsar isolation" --- import Tabs from '@theme/Tabs'; @@ -38,7 +38,9 @@ You can set a namespace isolation policy for a cluster using one of the followin ``` + pulsar-admin ns-isolation-policy set options + ``` For more information about the command `pulsar-admin ns-isolation-policy set options`, see [here](https://pulsar.apache.org/tools/pulsar-admin/). @@ -46,11 +48,13 @@ For more information about the command `pulsar-admin ns-isolation-policy set opt **Example** ```shell + bin/pulsar-admin ns-isolation-policy set \ --auto-failover-policy-type min_available \ --auto-failover-policy-params min_limit=1,usage_threshold=80 \ --namespaces my-tenant/my-namespace \ --primary 10.193.216.* my-cluster policy-name + ``` @@ -93,7 +97,9 @@ You can set a bookie affinity group using one of the following methods. ``` + pulsar-admin namespaces set-bookie-affinity-group options + ``` For more information about the command `pulsar-admin namespaces set-bookie-affinity-group options`, see [here](https://pulsar.apache.org/tools/pulsar-admin/). @@ -101,6 +107,7 @@ For more information about the command `pulsar-admin namespaces set-bookie-affin **Example** ```shell + bin/pulsar-admin bookies set-bookie-rack \ --bookie 127.0.0.1:3181 \ --hostname 127.0.0.1:3181 \ @@ -109,6 +116,7 @@ bin/pulsar-admin bookies set-bookie-rack \ bin/pulsar-admin namespaces set-bookie-affinity-group public/default \ --primary-group group-bookie1 + ``` diff --git a/site2/website-next/docs/administration-load-balance.md b/site2/website-next/docs/administration-load-balance.md index f0cd0492c33a2..26d1861b8c63a 100644 --- a/site2/website-next/docs/administration-load-balance.md +++ b/site2/website-next/docs/administration-load-balance.md @@ -1,7 +1,7 @@ --- id: administration-load-balance title: Pulsar load balance -sidebar_label: Load balance +sidebar_label: "Load balance" --- import Tabs from '@theme/Tabs'; @@ -49,15 +49,19 @@ Each bundle is independent of the others and thus is independently assigned to d When you create a new namespace, the new namespace sets to use the default number of bundles. You can set this in `conf/broker.conf`: ```properties + # When a namespace is created without specifying the number of bundle, this # value will be used as the default defaultNumberOfNamespaceBundles=4 + ``` You can either change the system default, or override it when you create a new namespace: ```shell + $ bin/pulsar-admin namespaces create my-tenant/my-namespace --clusters us-west --bundles 16 + ``` With this command, you create a namespace with 16 initial bundles. Therefore the topics for this namespaces can immediately be spread across up to 16 brokers. @@ -77,13 +81,17 @@ Unloading is the mechanism that the load-manager uses to perform the load sheddi Unloading a topic has no effect on the assignment, but just closes and reopens the particular topic: ```shell + pulsar-admin topics unload persistent://tenant/namespace/topic + ``` To unload all topics for a namespace and trigger reassignments: ```shell + pulsar-admin namespaces unload tenant/namespace + ``` ### Split namespace bundles @@ -95,6 +103,7 @@ The splitting is based on some tunable thresholds. Any existing bundle that exce You can split namespace bundles in two ways, by setting `supportedNamespaceBundleSplitAlgorithms` to `range_equally_divide` or `topic_count_equally_divide` in `broker.conf` file. The former splits the bundle into two parts with the same hash range size; the latter splits the bundle into two parts with the same number of topics. You can also configure other parameters for namespace bundles. ```properties + # enable/disable namespace bundle auto split loadBalancerAutoBundleSplitEnabled=true @@ -115,6 +124,7 @@ loadBalancerNamespaceBundleMaxBandwidthMbytes=100 # maximum number of bundles in a namespace (for auto-split) loadBalancerNamespaceMaximumBundles=128 + ``` ### Shed load automatically @@ -130,19 +140,23 @@ Given the selection of bundles to offload is based on traffic (as a proxy measur The automatic load shedding is enabled by default and you can disable the automatic load shedding with this setting: ```properties + # Enable/disable automatic bundle unloading for load-shedding loadBalancerSheddingEnabled=true + ``` Additional settings that apply to shedding: ```properties + # Load shedding interval. Broker periodically checks whether some traffic should be offload from # some over-loaded broker to other under-loaded brokers loadBalancerSheddingIntervalMinutes=1 # Prevent the same topics to be shed and moved to other brokers more that once within this timeframe loadBalancerSheddingGracePeriodMinutes=30 + ``` #### Broker overload thresholds @@ -152,8 +166,10 @@ The determinations of when a broker is overloaded is based on threshold of CPU, By default, overload threshold is set at 85%: ```properties + # Usage threshold to determine a broker as over-loaded loadBalancerBrokerOverloadedThresholdPercentage=85 + ``` Pulsar gathers the usage stats from the system metrics. @@ -165,6 +181,7 @@ Because of the incorrect max speed, the Pulsar load manager might think the brok You can use the following setting to correct the max NIC speed: ```properties + # Override the auto-detection of the network interfaces max speed. # This option is useful in some environments (eg: EC2 VMs) where the max speed # reported by Linux is not reflecting the real bandwidth available to the broker. @@ -173,6 +190,7 @@ You can use the following setting to correct the max NIC speed: # with the right value here. The configured value can be a double (eg: 0.8) and that # can be used to trigger load-shedding even before hitting on NIC limits. loadBalancerOverrideBrokerNicSpeedGbps= + ``` When the value is empty, Pulsar uses the value that the OS reports. diff --git a/site2/website-next/docs/administration-proxy.md b/site2/website-next/docs/administration-proxy.md index f1c38d71c2981..e27c7d47a19c0 100644 --- a/site2/website-next/docs/administration-proxy.md +++ b/site2/website-next/docs/administration-proxy.md @@ -1,7 +1,7 @@ --- id: administration-proxy title: Pulsar proxy -sidebar_label: Pulsar proxy +sidebar_label: "Pulsar proxy" --- import Tabs from '@theme/Tabs'; @@ -17,9 +17,12 @@ Before using the proxy, you need to configure it with the brokers addresses in t ### Use service discovery Pulsar uses [ZooKeeper](https://zookeeper.apache.org) for service discovery. To connect the proxy to ZooKeeper, specify the following in `conf/proxy.conf`. + ```properties + zookeeperServers=zk-0,zk-1,zk-2 configurationStoreServers=zk-0:2184,zk-remote:2184 + ``` > To use service discovery, you need to open the network ACLs, so the proxy can connects to the ZooKeeper nodes through the ZooKeeper client port (port `2181`) and the configuration store client port (port `2184`). @@ -35,16 +38,21 @@ Proxy authorization requires access to ZooKeeper, so if you use these broker URL You can configure the broker URLs in `conf/proxy.conf` as follows. ```properties + brokerServiceURL=pulsar://brokers.example.com:6650 brokerWebServiceURL=http://brokers.example.com:8080 functionWorkerWebServiceURL=http://function-workers.example.com:8080 + ``` If you use TLS, configure the broker URLs in the following way: + ```properties + brokerServiceURLTLS=pulsar+ssl://brokers.example.com:6651 brokerWebServiceURLTLS=https://brokers.example.com:8443 functionWorkerWebServiceURL=https://function-workers.example.com:8443 + ``` The hostname in the URLs provided should be a DNS entry which points to multiple brokers or a virtual IP address, which is backed by multiple broker IP addresses, so that the proxy does not lose connectivity to Pulsar cluster if a single broker becomes unavailable. @@ -58,8 +66,10 @@ Note that if you do not use functions, you do not need to configure `functionWor To start the proxy: ```bash + $ cd /path/to/pulsar/directory $ bin/pulsar proxy + ``` > You can run multiple instances of the Pulsar proxy in a cluster. diff --git a/site2/website-next/docs/administration-pulsar-manager.md b/site2/website-next/docs/administration-pulsar-manager.md index 98615c63c9eae..80cd52aaf1618 100644 --- a/site2/website-next/docs/administration-pulsar-manager.md +++ b/site2/website-next/docs/administration-pulsar-manager.md @@ -1,7 +1,7 @@ --- id: administration-pulsar-manager title: Pulsar Manager -sidebar_label: Pulsar Manager +sidebar_label: "Pulsar Manager" --- import Tabs from '@theme/Tabs'; @@ -12,7 +12,7 @@ Pulsar Manager is a web-based GUI management and monitoring tool that helps admi :::note -If you are monitoring your current stats with [Pulsar dashboard](administration-dashboard.md), we recommend you use Pulsar Manager instead. Pulsar dashboard is deprecated. +If you are monitoring your current stats with [Pulsar dashboard](administration-dashboard), we recommend you use Pulsar Manager instead. Pulsar dashboard is deprecated. ::: @@ -21,11 +21,13 @@ If you are monitoring your current stats with [Pulsar dashboard](administration- The easiest way to use the Pulsar Manager is to run it inside a [Docker](https://www.docker.com/products/docker) container. ```shell + docker pull apachepulsar/pulsar-manager:v0.2.0 docker run -it \ -p 9527:9527 -p 7750:7750 \ -e SPRING_CONFIGURATION_FILE=/pulsar-manager/pulsar-manager/application.properties \ apachepulsar/pulsar-manager:v0.2.0 + ``` * `SPRING_CONFIGURATION_FILE`: Default configuration file for spring. @@ -33,18 +35,21 @@ docker run -it \ ### Set administrator account and password ```shell -CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) -curl \ - -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ - -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ - -H "Content-Type: application/json" \ - -X PUT http://localhost:7750/pulsar-manager/users/superuser \ - -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' -``` + + CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) + curl \ + -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ + -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ + -H "Content-Type: application/json" \ + -X PUT http://localhost:7750/pulsar-manager/users/superuser \ + -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' + + ``` You can find the docker image in the [Docker Hub](https://github.com/apache/pulsar-manager/tree/master/docker) directory and build an image from the source code as well: ``` + git clone https://github.com/apache/pulsar-manager cd pulsar-manager/front-end npm install --save @@ -53,6 +58,7 @@ cd .. ./gradlew build -x test cd .. docker build -f docker/Dockerfile --build-arg BUILD_DATE=`date -u +"%Y-%m-%dT%H:%M:%SZ"` --build-arg VCS_REF=`latest` --build-arg VERSION=`latest` -t apachepulsar/pulsar-manager . + ``` ### Use custom databases @@ -64,16 +70,20 @@ If you have a large amount of data, you can use a custom database. The following 2. Modify the [configuration file](https://github.com/apache/pulsar-manager/blob/master/src/main/resources/application.properties) and add PostgreSQL configuration. ``` + spring.datasource.driver-class-name=org.postgresql.Driver spring.datasource.url=jdbc:postgresql://127.0.0.1:5432/pulsar_manager spring.datasource.username=postgres spring.datasource.password=postgres + ``` 3. Compile to generate a new executable jar package. ``` + ./gradlew build -x test + ``` ### Enable JWT authentication @@ -95,14 +105,17 @@ If you want to enable JWT authentication, use one of the following methods. * Method 1: use command-line tool ``` + wget https://dist.apache.org/repos/dist/release/pulsar/pulsar-manager/apache-pulsar-manager-0.2.0/apache-pulsar-manager-0.2.0-bin.tar.gz tar -zxvf apache-pulsar-manager-0.2.0-bin.tar.gz cd pulsar-manager tar -zxvf pulsar-manager.tar cd pulsar-manager cp -r ../dist ui -./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key +./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key + ``` + Firstly, [set the administrator account and password](#set-administrator-account-and-password) Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. @@ -110,6 +123,7 @@ Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. * Method 2: configure the application.properties file ``` + backend.jwt.token=token jwt.broker.token.mode=PRIVATE @@ -119,13 +133,16 @@ jwt.broker.private.key=file:///path/broker-private.key or jwt.broker.token.mode=SECRET jwt.broker.secret.key=file:///path/broker-secret.key + ``` * Method 3: use Docker and enable token authentication. ``` + export JWT_TOKEN="your-token" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -v $PWD:/data apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --secret-key` or `bin/pulsar tokens create --private-key` command. @@ -140,10 +157,12 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 4: use Docker and turn on **token authentication** and **token management** by private key and public key. ``` + export JWT_TOKEN="your-token" export PRIVATE_KEY="file:///pulsar-manager/secret/my-private.key" export PUBLIC_KEY="file:///pulsar-manager/secret/my-public.key" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -e PRIVATE_KEY=$PRIVATE_KEY -e PUBLIC_KEY=$PUBLIC_KEY -v $PWD:/data -v $PWD/secret:/pulsar-manager/secret apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --private-key` command. @@ -161,9 +180,11 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 5: use Docker and turn on **token authentication** and **token management** by secret key. ``` + export JWT_TOKEN="your-token" export SECRET_KEY="file:///pulsar-manager/secret/my-secret.key" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -e SECRET_KEY=$SECRET_KEY -v $PWD:/data -v $PWD/secret:/pulsar-manager/secret apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --secret-key` command. @@ -177,7 +198,7 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * `PASSWORD`: the password of PostgreSQL. * `LOG_LEVEL`: the level of log. -* For more information about backend configurations, see [here](https://github.com/apache/pulsar-manager/blob/master/src/README.md). +* For more information about backend configurations, see [here](https://github.com/apache/pulsar-manager/blob/master/src/README). * For more information about frontend configurations, see [here](https://github.com/apache/pulsar-manager/tree/master/front-end). ## Log in diff --git a/site2/website-next/docs/administration-stats.md b/site2/website-next/docs/administration-stats.md index f777408be8876..2ef0f233cc6f4 100644 --- a/site2/website-next/docs/administration-stats.md +++ b/site2/website-next/docs/administration-stats.md @@ -1,7 +1,7 @@ --- id: administration-stats title: Pulsar stats -sidebar_label: Pulsar statistics +sidebar_label: "Pulsar statistics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/administration-upgrade.md b/site2/website-next/docs/administration-upgrade.md index b83e906d91001..683c904d4fc5c 100644 --- a/site2/website-next/docs/administration-upgrade.md +++ b/site2/website-next/docs/administration-upgrade.md @@ -1,7 +1,7 @@ --- id: administration-upgrade title: Upgrade Guide -sidebar_label: Upgrade +sidebar_label: "Upgrade" --- import Tabs from '@theme/Tabs'; @@ -34,15 +34,24 @@ To upgrade an Apache Pulsar cluster, follow the upgrade sequence. 2. Upgrade bookies - Canary test: test an upgraded version in one or a small set of bookies. - Rolling upgrade: - - a. Disable `autorecovery` with the following command. - ```shell - bin/bookkeeper shell autorecovery -disable - ``` - - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. - - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. - ```shell - bin/bookkeeper shell autorecovery -enable - ``` + - a. Disable `autorecovery` with the following command. + + ```shell + + bin/bookkeeper shell autorecovery -disable + + ``` + + + - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. + - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. + + ```shell + + bin/bookkeeper shell autorecovery -enable + + ``` + 3. Upgrade brokers - Canary test: test an upgraded version in one or a small set of brokers. - Rolling upgrade: rollout the upgraded version to all brokers in the cluster after you determine that a version is safe after canary. @@ -89,13 +98,21 @@ To upgrade bookie to a new version, complete the following steps: 1. Stop a bookie. 2. Upgrade the binary and configuration files. 3. Start the bookie in `ReadOnly` mode to verify if the bookie of this new version runs well for read workload. + ```shell + bin/pulsar bookie --readOnly + ``` + 4. When the bookie runs successfully in `ReadOnly` mode, stop the bookie and restart it in `Write/Read` mode. + ```shell + bin/pulsar bookie + ``` + 5. Observe and make sure the cluster serves both write and read traffic. #### Canary rollback diff --git a/site2/website-next/docs/administration-zk-bk.md b/site2/website-next/docs/administration-zk-bk.md index 5efb7710541d6..1f173c0a9f211 100644 --- a/site2/website-next/docs/administration-zk-bk.md +++ b/site2/website-next/docs/administration-zk-bk.md @@ -1,7 +1,7 @@ --- id: administration-zk-bk title: ZooKeeper and BookKeeper administration -sidebar_label: ZooKeeper and BookKeeper +sidebar_label: "ZooKeeper and BookKeeper" --- import Tabs from '@theme/Tabs'; @@ -34,9 +34,11 @@ To deploy a Pulsar instance, you need to stand up one local ZooKeeper cluster *p To begin, add all ZooKeeper servers to the quorum configuration specified in the [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) file. Add a `server.N` line for each node in the cluster to the configuration, where `N` is the number of the ZooKeeper node. The following is an example for a three-node cluster: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` On each host, you need to specify the node ID in `myid` file of each node, which is in `data/zookeeper` folder of each server by default (you can change the file location via the [`dataDir`](reference-configuration.md#zookeeper-dataDir) parameter). @@ -47,8 +49,10 @@ On each host, you need to specify the node ID in `myid` file of each node, which On a ZooKeeper server at `zk1.us-west.example.com`, for example, you can set the `myid` value like this: ```shell + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com` the command is `echo 2 > data/zookeeper/myid` and so on. @@ -56,7 +60,9 @@ On `zk2.us-west.example.com` the command is `echo 2 > data/zookeeper/myid` and s Once you add each server to the `zookeeper.conf` configuration and each server has the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start zookeeper + ``` ### Deploy configuration store @@ -69,13 +75,15 @@ If you deploy a [single-cluster](#single-cluster-pulsar-instance) instance, you If your Pulsar instance consists of just one cluster, then you can deploy a configuration store on the same machines as the local ZooKeeper quorum but run on different TCP ports. -To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorom uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: +To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorum uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 server.3=zk3.us-west.example.com:2185:2186 + ``` As before, create the `myid` files for each server on `data/global-zookeeper/myid`. @@ -91,7 +99,9 @@ Again, given the very low expected load on the configuration store servers, you For example, you can assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also you can assume, each cluster has its own local ZK servers named such as ``` + zk[1-3].${CLUSTER}.example.com + ``` In this scenario you want to pick the quorum participants from few clusters and let all the others be ZK observers. For example, to form a 7 servers quorum, you can pick 3 servers from `us-west`, 2 from `us-central` and 2 from `us-east`. @@ -101,6 +111,7 @@ This guarantees that writes to configuration store is possible even if one of th The ZK configuration in all the servers looks like: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 @@ -117,12 +128,15 @@ server.12=zk3.eu-central.example.com:2185:2186:observer server.13=zk1.ap-south.example.com:2185:2186:observer server.14=zk2.ap-south.example.com:2185:2186:observer server.15=zk3.ap-south.example.com:2185:2186:observer + ``` Additionally, ZK observers need to have: ```properties + peerType=observer + ``` ##### Start the service @@ -130,7 +144,9 @@ peerType=observer Once your configuration store configuration is in place, you can start up the service using [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) ```shell + $ bin/pulsar-daemon start configuration-store + ``` ### ZooKeeper configuration @@ -162,7 +178,7 @@ The [`conf/global-zookeeper.conf`](reference-configuration.md#configuration-stor BookKeeper stores all durable message in Pulsar. BookKeeper is a distributed [write-ahead log](https://en.wikipedia.org/wiki/Write-ahead_logging) WAL system that guarantees read consistency of independent message logs calls ledgers. Individual BookKeeper servers are also called *bookies*. -> To manage message persistence, retention, and expiry in Pulsar, refer to [cookbook](cookbooks-retention-expiry.md). +> To manage message persistence, retention, and expiry in Pulsar, refer to [cookbook](cookbooks-retention-expiry). ### Hardware requirements @@ -183,6 +199,7 @@ You can configure BookKeeper bookies using the [`conf/bookkeeper.conf`](referenc The minimum configuration changes required in `conf/bookkeeper.conf` are as follows: ```properties + # Change to point to journal disk mount point journalDirectory=data/bookkeeper/journal @@ -191,6 +208,7 @@ ledgerDirectories=data/bookkeeper/ledgers # Point to local ZK quorum zkServers=zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181 + ``` To change the ZooKeeper root path that BookKeeper uses, use `zkLedgersRootPath=/MY-PREFIX/ledgers` instead of `zkServers=localhost:2181/MY-PREFIX`. @@ -208,19 +226,25 @@ You can start a bookie in the foreground or as a background daemon. To start a bookie in the foreground, use the [`bookkeeper`](reference-cli-tools.md#bookkeeper) CLI tool: ```bash + $ bin/bookkeeper bookie + ``` To start a bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` You can verify whether the bookie works properly with the `bookiesanity` command for the [BookKeeper shell](reference-cli-tools.md#bookkeeper-shell): ```shell + $ bin/bookkeeper shell bookiesanity + ``` When you use this command, you create a new ledger on the local bookie, write a few entries, read them back and finally delete the ledger. @@ -245,9 +269,9 @@ And then you can decommission bookies safely. To decommission bookies, complete 3. Run the decommission command. - If you have logged in to the node to be decommissioned, you do not need to provide `-bookieid`. - If you are running the decommission command for the target bookie node from another bookie node, you should mention the target bookie ID in the arguments for `-bookieid` - `$ bin/bookkeeper shell decommissionbookie` - or - `$ bin/bookkeeper shell decommissionbookie -bookieid ` + `$ bin/bookkeeper shell decommissionbookie` + or + `$ bin/bookkeeper shell decommissionbookie -bookieid ` 4. Validate that no ledgers are on the decommissioned bookie. `$ bin/bookkeeper shell listledgers -bookieid ` @@ -255,8 +279,10 @@ And then you can decommission bookies safely. To decommission bookies, complete You can run the following command to check if the bookie you have decommissioned is listed in the bookies list: ```bash + ./bookkeeper shell listbookies -rw -h ./bookkeeper shell listbookies -ro -h + ``` ## BookKeeper persistence policies @@ -278,7 +304,7 @@ Use the [`set-persistence`](reference-pulsar-admin.md#namespaces-set-persistence Flag | Description | Default :----|:------------|:------- -`-a`, `--bookkeeper-ack-quorom` | The number of acks (guaranteed copies) to wait on for each entry | 0 +`-a`, `--bookkeeper-ack-quorum` | The number of acks (guaranteed copies) to wait on for each entry | 0 `-e`, `--bookkeeper-ensemble` | The number of [bookies](reference-terminology.md#bookie) to use for topics in the namespace | 0 `-w`, `--bookkeeper-write-quorum` | The number of writes to make for each entry | 0 `-r`, `--ml-mark-delete-max-rate` | Throttling rate for mark-delete operations (0 means no throttle) | 0 @@ -286,9 +312,11 @@ Flag | Description | Default The following is an example: ```shell + $ pulsar-admin namespaces set-persistence my-tenant/my-ns \ - --bookkeeper-ack-quorom 3 \ + --bookkeeper-ack-quorum 3 \ --bookeeper-ensemble 2 + ``` #### REST API @@ -298,6 +326,7 @@ $ pulsar-admin namespaces set-persistence my-tenant/my-ns \ #### Java ```java + int bkEnsemble = 2; int bkQuorum = 3; int bkAckQuorum = 2; @@ -305,6 +334,7 @@ double markDeleteRate = 0.7; PersistencePolicies policies = new PersistencePolicies(ensemble, quorum, ackQuorum, markDeleteRate); admin.namespaces().setPersistence(namespace, policies); + ``` ### List persistence policies @@ -318,6 +348,7 @@ Use the [`get-persistence`](reference-pulsar-admin.md#namespaces-get-persistence The following is an example: ```shell + $ pulsar-admin namespaces get-persistence my-tenant/my-ns { "bookkeeperEnsemble": 1, @@ -325,6 +356,7 @@ $ pulsar-admin namespaces get-persistence my-tenant/my-ns "bookkeeperAckQuorum", 1, "managedLedgerMaxMarkDeleteRate": 0 } + ``` #### REST API @@ -334,7 +366,9 @@ $ pulsar-admin namespaces get-persistence my-tenant/my-ns #### Java ```java + PersistencePolicies policies = admin.namespaces().getPersistence(namespace); + ``` ## How Pulsar uses ZooKeeper and BookKeeper diff --git a/site2/website-next/docs/client-libraries-cgo.md b/site2/website-next/docs/client-libraries-cgo.md index 54607a2a00fef..dc9e5852da60e 100644 --- a/site2/website-next/docs/client-libraries-cgo.md +++ b/site2/website-next/docs/client-libraries-cgo.md @@ -1,7 +1,7 @@ --- id: client-libraries-cgo title: Pulsar CGo client -sidebar_label: CGo(deprecated) +sidebar_label: "CGo(deprecated)" --- import Tabs from '@theme/Tabs'; @@ -578,4 +578,5 @@ fmt.Println(s.ID) // output: 100 fmt.Println(s.Name) // output: pulsar defer consumer.Close() -``` \ No newline at end of file +``` + diff --git a/site2/website-next/docs/client-libraries-cpp.md b/site2/website-next/docs/client-libraries-cpp.md index bf578fe5fa0fa..b84c7a09980a4 100644 --- a/site2/website-next/docs/client-libraries-cpp.md +++ b/site2/website-next/docs/client-libraries-cpp.md @@ -1,7 +1,7 @@ --- id: client-libraries-cpp title: Pulsar C++ client -sidebar_label: C++ +sidebar_label: "C++" --- import Tabs from '@theme/Tabs'; @@ -14,7 +14,7 @@ All the methods in producer, consumer, and reader of a C++ client are thread-saf ## Supported platforms -Pulsar C++ client is supported on **Linux** and **MacOS** platforms. +Pulsar C++ client is supported on **Linux** ,**MacOS** and **Windows** platforms. [Doxygen](http://www.doxygen.nl/)-generated API docs for the C++ client are available [here](/api/cpp). @@ -127,6 +127,7 @@ These libraries rely on some other libraries. If you want to get detailed versio g++ --std=c++11 PulsarTest.cpp -o test /usr/lib/libpulsarwithdeps.a -lssl -lcrypto -ldl -lpthread -I/usr/local/ssl/include -L/usr/local/ssl/lib ``` + The `libpulsarwithdeps.a` does not include library openssl related libraries `libssl` and `libcrypto`, because these two libraries are related to security. It is more reasonable and easier to use the versions provided by the local system to handle security issues and upgrade libraries. ### Install RPM @@ -305,9 +306,15 @@ cmake --build ./build --config Release ``` +> **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. + 4. Client libraries are available in the following places. ``` + ${PULSAR_HOME}/pulsar-client-cpp/build/lib/Release/pulsar.lib ${PULSAR_HOME}/pulsar-client-cpp/build/lib/Release/pulsar.dll @@ -344,8 +351,8 @@ pulsar+ssl://pulsar.us-west.example.com:6651 ## Create a consumer To use Pulsar as a consumer, you need to create a consumer on the C++ client. There are two main ways of using the consumer: -- Blocking style: synchronously calling `receive(msg)`. -- Non-blocking (event based) style: using a message listener. +- [Blocking style](#blocking-example): synchronously calling `receive(msg)`. +- [Non-blocking](#consumer-with-a-message-listener) (event based) style: using a message listener. ### Blocking example @@ -354,6 +361,7 @@ The benefit of this approach is that it is the simplest code. Simply keeps calli This example starts a subscription at the earliest offset and consumes 100 messages. ```c++ + #include using namespace pulsar; @@ -392,11 +400,12 @@ int main() { ### Consumer with a message listener -We can avoid the need to run a loop with blocking calls with an event based style by using a message listener which is invoked for each message that is received. +You can avoid running a loop with blocking calls with an event based style by using a message listener which is invoked for each message that is received. This example starts a subscription at the earliest offset and consumes 100 messages. ```c++ + #include #include #include @@ -444,14 +453,15 @@ int main() { ## Create a producer To use Pulsar as a producer, you need to create a producer on the C++ client. There are two main ways of using a producer: -- Blocking style where each call to `send` waits for an ack from the broker. -- Non-blocking asynchronous style where `sendAsync` is called instead of `send` and a callback is supplied for when the ack is received from the broker. +- [Blocking style](#simple-blocking-example) : each call to `send` waits for an ack from the broker. +- [Non-blocking asynchronous style](#non-blocking-example) : `sendAsync` is called instead of `send` and a callback is supplied for when the ack is received from the broker. ### Simple blocking example This example sends 100 messages using the blocking style. While simple, it does not produce high throughput as it waits for each ack to come back before sending the next message. ```c++ + #include #include @@ -499,6 +509,7 @@ The producer configuration `blockIfQueueFull` is useful here to avoid `ResultPro Without this configuration, the result code `ResultProducerQueueIsFull` is passed to the callback. You must decide how to deal with that (retry, discard etc). ```c++ + #include #include @@ -565,6 +576,7 @@ With our example above, that reduces the number of internal producers spread out Note that there can be extra latency for the first message sent. If you set a low send timeout, this timeout could be reached if the initial connection handshake is slow to complete. ```c++ + ProducerConfiguration producerConf; producerConf.setPartitionsRoutingMode(ProducerConfiguration::UseSinglePartition); producerConf.setLazyStartPartitionedProducers(true); @@ -598,68 +610,73 @@ schema, see [Pulsar schema](schema-get-started). - The following example shows how to create a producer with an Avro schema. - ```cpp - - static const std::string exampleSchema = - "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," - "\"fields\":[{\"name\":\"a\",\"type\":\"int\"},{\"name\":\"b\",\"type\":\"int\"}]}"; - Producer producer; - ProducerConfiguration producerConf; - producerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); - client.createProducer("topic-avro", producerConf, producer); - - ``` + ```cpp + + static const std::string exampleSchema = + "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," + "\"fields\":[{\"name\":\"a\",\"type\":\"int\"},{\"name\":\"b\",\"type\":\"int\"}]}"; + Producer producer; + ProducerConfiguration producerConf; + producerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); + client.createProducer("topic-avro", producerConf, producer); + + ``` - The following example shows how to create a consumer with an Avro schema. - ```cpp - - static const std::string exampleSchema = - "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," - "\"fields\":[{\"name\":\"a\",\"type\":\"int\"},{\"name\":\"b\",\"type\":\"int\"}]}"; - ConsumerConfiguration consumerConf; - Consumer consumer; - consumerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); - client.subscribe("topic-avro", "sub-2", consumerConf, consumer) - - ``` + ```cpp + + static const std::string exampleSchema = + "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," + "\"fields\":[{\"name\":\"a\",\"type\":\"int\"},{\"name\":\"b\",\"type\":\"int\"}]}"; + ConsumerConfiguration consumerConf; + Consumer consumer; + consumerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); + client.subscribe("topic-avro", "sub-2", consumerConf, consumer) + + ``` ### ProtobufNative schema The following example shows how to create a producer and a consumer with a ProtobufNative schema. ​ -1. Generate the `User` class using Protobuf3. +1. Generate the `User` class using Protobuf3. -:::note + :::note -You need to use Protobuf3 or later versions. + You need to use Protobuf3 or later versions. + + ::: -::: ​ ```protobuf - + syntax = "proto3"; message User { string name = 1; int32 age = 2; } - + ``` + ​ 2. Include the `ProtobufNativeSchema.h` in your source code. Ensure the Protobuf dependency has been added to your project. ​ ```c++ + #include - + ``` + ​ 3. Create a producer to send a `User` instance. ​ ```c++ + ProducerConfiguration producerConf; producerConf.setSchema(createProtobufNativeSchema(User::GetDescriptor())); Producer producer; @@ -670,13 +687,15 @@ You need to use Protobuf3 or later versions. std::string content; user.SerializeToString(&content); producer.send(MessageBuilder().setContent(content).build()); - + ``` + ​ 4. Create a consumer to receive a `User` instance. ​ ```c++ + ConsumerConfiguration consumerConf; consumerConf.setSchema(createProtobufNativeSchema(User::GetDescriptor())); consumerConf.setSubscriptionInitialPosition(InitialPositionEarliest); @@ -686,5 +705,6 @@ You need to use Protobuf3 or later versions. consumer.receive(msg); User user2; user2.ParseFromArray(msg.getData(), msg.getLength()); - + ``` + diff --git a/site2/website-next/docs/client-libraries-dotnet.md b/site2/website-next/docs/client-libraries-dotnet.md index 8d0e7ad9bb1f9..ff608e0c83de8 100644 --- a/site2/website-next/docs/client-libraries-dotnet.md +++ b/site2/website-next/docs/client-libraries-dotnet.md @@ -1,7 +1,7 @@ --- id: client-libraries-dotnet title: Pulsar C# client -sidebar_label: C# +sidebar_label: "C#" --- import Tabs from '@theme/Tabs'; @@ -30,10 +30,11 @@ To install the Pulsar C# client library, following these steps: 3. Create the project using the following command. - ``` - dotnet new console - - ``` + ``` + + dotnet new console + + ``` 4. Use `dotnet run` to test that the app has been created properly. @@ -41,20 +42,21 @@ To install the Pulsar C# client library, following these steps: 1. Use the following command to install the `DotPulsar` package. - ``` - dotnet add package DotPulsar - - ``` + ``` + + dotnet add package DotPulsar + + ``` 2. After the command completes, open the `.csproj` file to see the added reference. - ```xml - - - - - - ``` + ```xml + + + + + + ``` ## Client @@ -65,6 +67,7 @@ This section describes some configuration examples for the Pulsar C# client. This example shows how to create a Pulsar C# client connected to localhost. ```c# + var client = PulsarClient.Builder().Build(); ``` @@ -82,20 +85,22 @@ This section describes how to create a producer. - Create a producer by using the builder. - ```c# - var producer = client.NewProducer() - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var producer = client.NewProducer() + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a producer without using the builder. - ```c# - var options = new ProducerOptions("persistent://public/default/mytopic"); - var producer = client.CreateProducer(options); - - ``` + ```c# + + var options = new ProducerOptions("persistent://public/default/mytopic"); + var producer = client.CreateProducer(options); + + ``` ### Create consumer @@ -103,21 +108,23 @@ This section describes how to create a consumer. - Create a consumer by using the builder. - ```c# - var consumer = client.NewConsumer() - .SubscriptionName("MySubscription") - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var consumer = client.NewConsumer() + .SubscriptionName("MySubscription") + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a consumer without using the builder. - ```c# - var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); - var consumer = client.CreateConsumer(options); - - ``` + ```c# + + var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); + var consumer = client.CreateConsumer(options); + + ``` ### Create reader @@ -125,21 +132,23 @@ This section describes how to create a reader. - Create a reader by using the builder. - ```c# - var reader = client.NewReader() - .StartMessageId(MessageId.Earliest) - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var reader = client.NewReader() + .StartMessageId(MessageId.Earliest) + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a reader without using the builder. - ```c# - var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); - var reader = client.CreateReader(options); - - ``` + ```c# + + var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); + var reader = client.CreateReader(options); + + ``` ### Configure encryption policies @@ -153,6 +162,7 @@ The Pulsar C# client supports four kinds of encryption policies: This example shows how to set the `EnforceUnencrypted` encryption policy. ```c# + var client = PulsarClient.Builder() .ConnectionSecurity(EncryptionPolicy.EnforceEncrypted) .Build(); @@ -167,20 +177,22 @@ If you have followed [Authentication using TLS](security-tls-authentication), yo 1. Create an unencrypted and password-less pfx file. - ```c# - openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: - - ``` + ```c# + + openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: + + ``` 2. Use the admin.pfx file to create an X509Certificate2 and pass it to the Pulsar C# client. - ```c# - var clientCertificate = new X509Certificate2("admin.pfx"); - var client = PulsarClient.Builder() - .AuthenticateUsingClientCertificate(clientCertificate) - .Build(); - - ``` + ```c# + + var clientCertificate = new X509Certificate2("admin.pfx"); + var client = PulsarClient.Builder() + .AuthenticateUsingClientCertificate(clientCertificate) + .Build(); + + ``` ## Producer @@ -191,6 +203,7 @@ A producer is a process that attaches to a topic and publishes messages to a Pul This example shows how to send data. ```c# + var data = Encoding.UTF8.GetBytes("Hello World"); await producer.Send(data); @@ -200,23 +213,25 @@ await producer.Send(data); - Send messages with customized metadata by using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var messageId = await producer.NewMessage() - .Property("SomeKey", "SomeValue") - .Send(data); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var messageId = await producer.NewMessage() + .Property("SomeKey", "SomeValue") + .Send(data); + + ``` - Send messages with customized metadata without using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var metadata = new MessageMetadata(); - metadata["SomeKey"] = "SomeValue"; - var messageId = await producer.Send(metadata, data)); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var metadata = new MessageMetadata(); + metadata["SomeKey"] = "SomeValue"; + var messageId = await producer.Send(metadata, data)); + + ``` ## Consumer @@ -227,6 +242,7 @@ A consumer is a process that attaches to a topic through a subscription and then This example shows how a consumer receives messages from a topic. ```c# + await foreach (var message in consumer.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -240,26 +256,29 @@ Messages can be acknowledged individually or cumulatively. For details about mes - Acknowledge messages individually. - ```c# - await foreach (var message in consumer.Messages()) - { - Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); - } - - ``` + ```c# + + await foreach (var message in consumer.Messages()) + { + Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); + } + + ``` - Acknowledge messages cumulatively. - ```c# - await consumer.AcknowledgeCumulative(message); - - ``` + ```c# + + await consumer.AcknowledgeCumulative(message); + + ``` ### Unsubscribe from topics This example shows how a consumer unsubscribes from a topic. ```c# + await consumer.Unsubscribe(); ``` @@ -275,6 +294,7 @@ A reader is actually just a consumer without a cursor. This means that Pulsar do This example shows how a reader receives messages. ```c# + await foreach (var message in reader.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -300,6 +320,7 @@ The following table lists states available for the producer. This example shows how to monitor the producer state. ```c# + private static async ValueTask Monitor(IProducer producer, CancellationToken cancellationToken) { var state = ProducerState.Disconnected; @@ -342,6 +363,7 @@ The following table lists states available for the consumer. This example shows how to monitor the consumer state. ```c# + private static async ValueTask Monitor(IConsumer consumer, CancellationToken cancellationToken) { var state = ConsumerState.Disconnected; @@ -385,6 +407,7 @@ The following table lists states available for the reader. This example shows how to monitor the reader state. ```c# + private static async ValueTask Monitor(IReader reader, CancellationToken cancellationToken) { var state = ReaderState.Disconnected; @@ -411,3 +434,4 @@ private static async ValueTask Monitor(IReader reader, CancellationToken cancell } ``` + diff --git a/site2/website-next/docs/client-libraries-go.md b/site2/website-next/docs/client-libraries-go.md index 4583034900215..ab07c4cde3bc8 100644 --- a/site2/website-next/docs/client-libraries-go.md +++ b/site2/website-next/docs/client-libraries-go.md @@ -1,7 +1,7 @@ --- id: client-libraries-go title: Pulsar Go client -sidebar_label: Go +sidebar_label: "Go" --- import Tabs from '@theme/Tabs'; @@ -51,6 +51,7 @@ pulsar://localhost:6650 If you have multiple brokers, you can set the URL as below. ``` + pulsar://localhost:6550,localhost:6651,localhost:6652 ``` @@ -102,6 +103,7 @@ func main() { If you have multiple brokers, you can initiate a client object as below. ``` + import ( "log" "time" @@ -882,3 +884,4 @@ client, err := pulsar.NewClient(pulsar.ClientOptions{ }) ``` + diff --git a/site2/website-next/docs/client-libraries-java.md b/site2/website-next/docs/client-libraries-java.md index b74423a6a8dc4..8c4fda1465652 100644 --- a/site2/website-next/docs/client-libraries-java.md +++ b/site2/website-next/docs/client-libraries-java.md @@ -1,7 +1,7 @@ --- id: client-libraries-java title: Pulsar Java client -sidebar_label: Java +sidebar_label: "Java" --- import Tabs from '@theme/Tabs'; @@ -178,22 +178,29 @@ stringProducer.send("My message"); ``` > Make sure that you close your producers, consumers, and clients when you do not need them. -> ```java +> ```java +> > producer.close(); > consumer.close(); > client.close(); +> +> > ``` + > > Close operations can also be asynchronous: -> ```java +> ```java +> > producer.closeAsync() > .thenRun(() -> System.out.println("Producer closed")) > .exceptionally((ex) -> { > System.err.println("Failed to close producer: " + ex); > return null; > }); +> +> > ``` ### Configure producer @@ -306,7 +313,7 @@ while (true) { } ``` - + If you don't want to block your main thread and rather listen constantly for new messages, consider using a `MessageListener`. ```java @@ -405,9 +412,7 @@ consumer.acknowledge(messages) :::note - Batch receive policy limits the number and bytes of messages in a single batch. You can specify a timeout to wait for enough messages. - The batch receive is completed if any of the following condition is met: enough number of messages, bytes of messages, wait timeout. ```java @@ -423,6 +428,7 @@ Consumer consumer = client.newConsumer() .subscribe(); ``` + The default batch receive policy is: ```java @@ -484,7 +490,6 @@ pulsarClient.newConsumer() :::note - By default, the `subscriptionTopicsMode` of the consumer is `PersistentOnly`. Available options of `subscriptionTopicsMode` are `PersistentOnly`, `NonPersistentOnly`, and `AllTopics`. ::: @@ -581,7 +586,6 @@ Only the first consumer is allowed to the subscription, other consumers receive :::note - If topic is a partitioned topic, the first consumer subscribes to all partitioned topics, other consumers are not assigned with partitions and receive an error. ::: @@ -605,8 +609,6 @@ Consumer consumer2 = client.newConsumer() //conumser1 is the active consumer, consumer2 is the standby consumer. //consumer1 receives 5 messages and then crashes, consumer2 takes over as an active consumer. - - ``` Multiple consumers can attach to the same subscription, yet only the first consumer is active, and others are standby. When the active consumer is disconnected, messages will be dispatched to one of standby consumers, and the standby consumer then becomes active consumer. @@ -614,6 +616,7 @@ Multiple consumers can attach to the same subscription, yet only the first consu If the first active consumer is disconnected after receiving 5 messages, the standby consumer becomes active consumer. Consumer1 will receive: ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -625,6 +628,7 @@ If the first active consumer is disconnected after receiving 5 messages, the sta consumer2 will receive: ``` + ("key-2", "message-2-3") ("key-3", "message-3-1") ("key-3", "message-3-2") @@ -635,7 +639,6 @@ consumer2 will receive: :::note - If a topic is a partitioned topic, each partition has only one active consumer, messages of one partition are distributed to only one consumer, and messages of multiple partitions are distributed to multiple consumers. ::: @@ -666,6 +669,7 @@ In shared subscription mode, multiple consumers can attach to the same subscript If a broker dispatches only one message at a time, consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-3") ("key-2", "message-2-2") @@ -677,6 +681,7 @@ If a broker dispatches only one message at a time, consumer1 receives the follow consumer2 receives the following information. ``` + ("key-1", "message-1-2") ("key-2", "message-2-1") ("key-2", "message-2-3") @@ -713,6 +718,7 @@ Just like in the `Shared` subscription, all consumers in the `Key_Shared` subscr consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -724,6 +730,7 @@ consumer1 receives the following information. consumer2 receives the following information. ``` + ("key-2", "message-2-1") ("key-2", "message-2-2") ("key-2", "message-2-3") @@ -742,6 +749,7 @@ Producer producer = client.newProducer() .create(); ``` + Or the producer can disable batching. ```java @@ -755,7 +763,6 @@ Producer producer = client.newProducer() :::note - If the message key is not specified, messages without key are dispatched to one consumer in order by default. ::: @@ -877,62 +884,61 @@ The following schema formats are currently available for Java: * No schema or the byte array schema (which can be applied using `Schema.BYTES`): ```java - + Producer bytesProducer = client.newProducer(Schema.BYTES) - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` Or, equivalently: ```java - + Producer bytesProducer = client.newProducer() - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` * `String` for normal UTF-8-encoded string data. Apply the schema using `Schema.STRING`: ```java - + Producer stringProducer = client.newProducer(Schema.STRING) - .topic("some-string-topic") - .create(); - + .topic("some-string-topic") + .create(); + ``` * Create JSON schemas for POJOs using `Schema.JSON`. The following is an example. ```java - + Producer pojoProducer = client.newProducer(Schema.JSON(MyPojo.class)) - .topic("some-pojo-topic") - .create(); - + .topic("some-pojo-topic") + .create(); + ``` * Generate Protobuf schemas using `Schema.PROTOBUF`. The following example shows how to create the Protobuf schema and use it to instantiate a new producer: ```java - + Producer protobufProducer = client.newProducer(Schema.PROTOBUF(MyProtobuf.class)) - .topic("some-protobuf-topic") - .create(); - + .topic("some-protobuf-topic") + .create(); + ``` * Define Avro schemas with `Schema.AVRO`. The following code snippet demonstrates how to create and use Avro schema. - ```java - + Producer avroProducer = client.newProducer(Schema.AVRO(MyAvro.class)) - .topic("some-avro-topic") - .create(); - + .topic("some-avro-topic") + .create(); + ``` ### ProtobufNativeSchema example @@ -1033,3 +1039,4 @@ PulsarClient client = PulsarClient.builder() .build(); ``` + diff --git a/site2/website-next/docs/client-libraries-node.md b/site2/website-next/docs/client-libraries-node.md index d16a4840df7c0..42a19275e02c1 100644 --- a/site2/website-next/docs/client-libraries-node.md +++ b/site2/website-next/docs/client-libraries-node.md @@ -1,7 +1,7 @@ --- id: client-libraries-node title: The Pulsar Node.js client -sidebar_label: Node.js +sidebar_label: "Node.js" --- import Tabs from '@theme/Tabs'; @@ -46,7 +46,6 @@ $ npm install pulsar-client :::note - Also, this library works only in Node.js 10.x or later because it uses the [`node-addon-api`](https://github.com/nodejs/node-addon-api) module to wrap the C++ library. ::: @@ -125,7 +124,7 @@ Here is an example: ```JavaScript const producer = await client.createProducer({ - topic: 'my-topic', + topic: 'my-topic', // or 'my-tenant/my-namespace/my-topic' to specify topic's tenant and namespace }); await producer.send({ @@ -147,7 +146,7 @@ Pulsar Node.js producers have the following methods available: | Method | Description | Return type | | :----- | :---------- | :---------- | | `send(Object)` | Publishes a [message](#messages) to the producer's topic. When the message is successfully acknowledged by the Pulsar broker, or an error is thrown, the Promise object whose result is the message ID runs executor function. | `Promise` | -| `flush()` | Sends message from send queue to Pulser broker. When the message is successfully acknowledged by the Pulsar broker, or an error is thrown, the Promise object runs executor function. | `Promise` | +| `flush()` | Sends message from send queue to Pulsar broker. When the message is successfully acknowledged by the Pulsar broker, or an error is thrown, the Promise object runs executor function. | `Promise` | | `close()` | Closes the producer and releases all resources allocated to it. Once `close()` is called, no more messages are accepted from the publisher. This method returns a Promise object. It runs the executor function when all pending publish requests are persisted by Pulsar. If an error is thrown, no pending writes are retried. | `Promise` | | `getProducerName()` | Getter method of the producer name. | `string` | | `getTopic()` | Getter method of the name of the topic. | `string` | @@ -479,6 +478,7 @@ The following static methods are available for the message id object: If you want to use the end-to-end encryption feature in the Node.js client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -496,148 +496,151 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** + **Input** - ```shell - - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem - - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://localhost:6650', - operationTimeoutSeconds: 30, - }); - - // Create a producer - const producer = await client.createProducer({ - topic: 'persistent://public/default/my-topic', - sendTimeoutMs: 30000, - batchingEnabled: true, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem", - encryptionKey: "encryption-key" - }); - - console.log(producer.ProducerConfig) - // Send messages - for (let i = 0; i < 10; i += 1) { - const msg = `my-message-${i}`; - producer.send({ - data: Buffer.from(msg), - }); - console.log(`Sent message: ${msg}`); - } - await producer.flush(); - - await producer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://localhost:6650', + operationTimeoutSeconds: 30, + }); + + // Create a producer + const producer = await client.createProducer({ + topic: 'persistent://public/default/my-topic', + sendTimeoutMs: 30000, + batchingEnabled: true, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem", + encryptionKey: "encryption-key" + }); + + console.log(producer.ProducerConfig) + // Send messages + for (let i = 0; i < 10; i += 1) { + const msg = `my-message-${i}`; + producer.send({ + data: Buffer.from(msg), + }); + console.log(`Sent message: ${msg}`); + } + await producer.flush(); + + await producer.close(); + await client.close(); + })(); + + ``` 3. Create a consumer to receive encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://172.25.0.3:6650', - operationTimeoutSeconds: 30 - }); - - // Create a consumer - const consumer = await client.subscribe({ - topic: 'persistent://public/default/my-topic', - subscription: 'sub1', - subscriptionType: 'Shared', - ackTimeoutMs: 10000, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem" - }); - - console.log(consumer) - // Receive messages - for (let i = 0; i < 10; i += 1) { - const msg = await consumer.receive(); - console.log(msg.getData().toString()); - consumer.acknowledge(msg); - } - - await consumer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://172.25.0.3:6650', + operationTimeoutSeconds: 30 + }); + + // Create a consumer + const consumer = await client.subscribe({ + topic: 'persistent://public/default/my-topic', + subscription: 'sub1', + subscriptionType: 'Shared', + ackTimeoutMs: 10000, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem" + }); + + console.log(consumer) + // Receive messages + for (let i = 0; i < 10; i += 1) { + const msg = await consumer.receive(); + console.log(msg.getData().toString()); + consumer.acknowledge(msg); + } + + await consumer.close(); + await client.close(); + })(); + + ``` 4. Run the consumer to receive encrypted messages. - **Input** + **Input** - ```shell - - node consumer.js - - ``` + ```shell + + node consumer.js + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - node producer.js - - ``` - - Now you can see the producer sends messages and the consumer receives messages successfully. - - **Output** - - This is from the producer side. - - ``` - Sent message: my-message-0 - Sent message: my-message-1 - Sent message: my-message-2 - Sent message: my-message-3 - Sent message: my-message-4 - Sent message: my-message-5 - Sent message: my-message-6 - Sent message: my-message-7 - Sent message: my-message-8 - Sent message: my-message-9 - - ``` - - This is from the consumer side. - - ``` - my-message-0 - my-message-1 - my-message-2 - my-message-3 - my-message-4 - my-message-5 - my-message-6 - my-message-7 - my-message-8 - my-message-9 + **Input** + + ```shell + + node producer.js + + ``` + + Now you can see the producer sends messages and the consumer receives messages successfully. + + **Output** + + This is from the producer side. + + ``` + + Sent message: my-message-0 + Sent message: my-message-1 + Sent message: my-message-2 + Sent message: my-message-3 + Sent message: my-message-4 + Sent message: my-message-5 + Sent message: my-message-6 + Sent message: my-message-7 + Sent message: my-message-8 + Sent message: my-message-9 + + ``` + + This is from the consumer side. + + ``` + + my-message-0 + my-message-1 + my-message-2 + my-message-3 + my-message-4 + my-message-5 + my-message-6 + my-message-7 + my-message-8 + my-message-9 + + ``` - ``` \ No newline at end of file diff --git a/site2/website-next/docs/client-libraries-python.md b/site2/website-next/docs/client-libraries-python.md index e696ed7b2105d..bf2b09284cfce 100644 --- a/site2/website-next/docs/client-libraries-python.md +++ b/site2/website-next/docs/client-libraries-python.md @@ -1,7 +1,7 @@ --- id: client-libraries-python title: Pulsar Python client -sidebar_label: Python +sidebar_label: "Python" --- import Tabs from '@theme/Tabs'; @@ -164,6 +164,7 @@ while True: # No acknowledgment ``` + ### Multi-topic subscriptions In addition to subscribing a consumer to a single Pulsar topic, you can also subscribe to multiple topics simultaneously. To use multi-topic subscriptions, you can supply a regular expression (regex) or a `List` of topics. If you select topics via regex, all topics must be within the same Pulsar namespace. @@ -355,6 +356,7 @@ class NamespaceDemo(Record): The schema definition is like this. ``` + { 'name': 'NamespaceDemo', 'namespace': 'xxx.xxx.xxx', 'type': 'record', 'fields': [ {'name': 'x', 'type': ['null', 'string']}, @@ -373,6 +375,7 @@ The schema definition is like this. To use the end-to-end encryption feature in the Python client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -390,89 +393,92 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** - - ```shell + **Input** - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem - - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** - - ```python + **Input** - import pulsar + ```python + + import pulsar - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) - producer.send('encryption message'.encode('utf8')) - print('sent message') - producer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) + producer.send('encryption message'.encode('utf8')) + print('sent message') + producer.close() + client.close() + + ``` 3. Create a consumer to receive encrypted messages. - **Input** + **Input** - ```python + ```python + + import pulsar - import pulsar - - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) - msg = consumer.receive() - print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) - consumer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) + msg = consumer.receive() + print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) + consumer.close() + client.close() + + ``` 4. Run the consumer to receive encrypted messages. - **Input** - - ```shell - - python consumer.py + **Input** - ``` + ```shell + + python consumer.py + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - python producer.py - - ``` + **Input** - Now you can see the producer sends messages and the consumer receives messages successfully. + ```shell + + python producer.py + + ``` - **Output** + Now you can see the producer sends messages and the consumer receives messages successfully. - This is from the producer side. + **Output** - ``` - sent message + This is from the producer side. - ``` + ``` + + sent message + + ``` - This is from the consumer side. + This is from the consumer side. - ``` - Received msg 'encryption message' id = '(0,0,-1,-1)' + ``` + + Received msg 'encryption message' id = '(0,0,-1,-1)' + + ``` - ``` diff --git a/site2/website-next/docs/client-libraries-websocket.md b/site2/website-next/docs/client-libraries-websocket.md index 0e47df5694b68..c789002dafb34 100644 --- a/site2/website-next/docs/client-libraries-websocket.md +++ b/site2/website-next/docs/client-libraries-websocket.md @@ -1,7 +1,7 @@ --- id: client-libraries-websocket title: Pulsar WebSocket API -sidebar_label: WebSocket +sidebar_label: "WebSocket" --- import Tabs from '@theme/Tabs'; @@ -98,7 +98,7 @@ The producer endpoint requires you to specify a tenant, namespace, and topic in ```http -ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic +ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic ``` @@ -151,6 +151,7 @@ Key | Type | Required? | Explanation } ``` + ##### Example failure response ```json @@ -626,3 +627,4 @@ ws.on('message', function(message) { }); ``` + diff --git a/site2/website-next/docs/concepts-architecture-overview.md b/site2/website-next/docs/concepts-architecture-overview.md index c76fa3c2a3480..470c616c49152 100644 --- a/site2/website-next/docs/concepts-architecture-overview.md +++ b/site2/website-next/docs/concepts-architecture-overview.md @@ -1,10 +1,14 @@ --- id: concepts-architecture-overview title: Architecture Overview -sidebar_label: Architecture +sidebar_label: "Architecture" --- -At the highest level, a Pulsar instance is composed of one or more Pulsar clusters. Clusters within an instance can [replicate](concepts-replication.md) data amongst themselves. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +At the highest level, a Pulsar instance is composed of one or more Pulsar clusters. Clusters within an instance can [replicate](concepts-replication) data amongst themselves. In a Pulsar cluster: @@ -16,20 +20,20 @@ The diagram below provides an illustration of a Pulsar cluster: ![Pulsar architecture diagram](/assets/pulsar-system-architecture.png) -At the broader instance level, an instance-wide ZooKeeper cluster called the configuration store handles coordination tasks involving multiple clusters, for example [geo-replication](concepts-replication.md). +At the broader instance level, an instance-wide ZooKeeper cluster called the configuration store handles coordination tasks involving multiple clusters, for example [geo-replication](concepts-replication). ## Brokers The Pulsar message broker is a stateless component that's primarily responsible for running two other components: * An HTTP server that exposes a {@inject: rest:REST:/} API for both administrative tasks and [topic lookup](concepts-clients.md#client-setup-phase) for producers and consumers. The producers connect to the brokers to publish messages and the consumers connect to the brokers to consume the messages. -* A dispatcher, which is an asynchronous TCP server over a custom [binary protocol](developing-binary-protocol.md) used for all data transfers +* A dispatcher, which is an asynchronous TCP server over a custom [binary protocol](developing-binary-protocol) used for all data transfers Messages are typically dispatched out of a [managed ledger](#managed-ledgers) cache for the sake of performance, *unless* the backlog exceeds the cache size. If the backlog grows too large for the cache, the broker will start reading entries from BookKeeper. -Finally, to support geo-replication on global topics, the broker manages replicators that tail the entries published in the local region and republish them to the remote region using the Pulsar [Java client library](client-libraries-java.md). +Finally, to support geo-replication on global topics, the broker manages replicators that tail the entries published in the local region and republish them to the remote region using the Pulsar [Java client library](client-libraries-java). -> For a guide to managing Pulsar brokers, see the [brokers](admin-api-brokers.md) guide. +> For a guide to managing Pulsar brokers, see the [brokers](admin-api-brokers) guide. ## Clusters @@ -39,9 +43,9 @@ A Pulsar instance consists of one or more Pulsar *clusters*. Clusters, in turn, * A ZooKeeper quorum used for cluster-level configuration and coordination * An ensemble of bookies used for [persistent storage](#persistent-storage) of messages -Clusters can replicate amongst themselves using [geo-replication](concepts-replication.md). +Clusters can replicate amongst themselves using [geo-replication](concepts-replication). -> For a guide to managing Pulsar clusters, see the [clusters](admin-api-clusters.md) guide. +> For a guide to managing Pulsar clusters, see the [clusters](admin-api-clusters) guide. ## Metadata store @@ -78,7 +82,9 @@ In addition to message data, *cursors* are also persistently stored in BookKeepe At the moment, Pulsar supports persistent message storage. This accounts for the `persistent` in all topic names. Here's an example: ```http + persistent://my-tenant/my-namespace/my-topic + ``` > Pulsar also supports ephemeral ([non-persistent](concepts-messaging.md#non-persistent-topics)) message storage. @@ -125,23 +131,25 @@ The **Pulsar proxy** provides a solution to this problem by acting as a single g Architecturally, the Pulsar proxy gets all the information it requires from ZooKeeper. When starting the proxy on a machine, you only need to provide ZooKeeper connection strings for the cluster-specific and instance-wide configuration store clusters. Here's an example: ```bash + $ bin/pulsar proxy \ --zookeeper-servers zk-0,zk-1,zk-2 \ --configuration-store-servers zk-0,zk-1,zk-2 + ``` > #### Pulsar proxy docs -> For documentation on using the Pulsar proxy, see the [Pulsar proxy admin documentation](administration-proxy.md). +> For documentation on using the Pulsar proxy, see the [Pulsar proxy admin documentation](administration-proxy). Some important things to know about the Pulsar proxy: * Connecting clients don't need to provide *any* specific configuration to use the Pulsar proxy. You won't need to update the client configuration for existing applications beyond updating the IP used for the service URL (for example if you're running a load balancer over the Pulsar proxy). -* [TLS encryption](security-tls-transport.md) and [authentication](security-tls-authentication.md) is supported by the Pulsar proxy +* [TLS encryption](security-tls-transport.md) and [authentication](security-tls-authentication) is supported by the Pulsar proxy ## Service discovery -[Clients](getting-started-clients.md) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. +[Clients](getting-started-clients) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. You can use your own service discovery system if you'd like. If you use your own system, there is just one requirement: when a client performs an HTTP request to an endpoint, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to *some* active broker in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. @@ -149,13 +157,19 @@ The diagram below illustrates Pulsar service discovery: ![alt-text](/assets/pulsar-service-discovery.png) -In this diagram, the Pulsar cluster is addressable via a single DNS name: `pulsar-cluster.acme.com`. A [Python client](client-libraries-python.md), for example, could access this Pulsar cluster like this: +In this diagram, the Pulsar cluster is addressable via a single DNS name: `pulsar-cluster.acme.com`. A [Python client](client-libraries-python), for example, could access this Pulsar cluster like this: ```python + from pulsar import Client client = Client('pulsar://pulsar-cluster.acme.com:6650') + ``` -> **Note** -> In Pulsar, each topic is handled by only one broker. Initial requests from a client to read, update or delete a topic are sent to a broker that may not be the topic owner. If the broker cannot handle the request for this topic, it redirects the request to the appropriate broker. +:::note + +In Pulsar, each topic is handled by only one broker. Initial requests from a client to read, update or delete a topic are sent to a broker that may not be the topic owner. If the broker cannot handle the request for this topic, it redirects the request to the appropriate broker. + +::: + diff --git a/site2/website-next/docs/concepts-authentication.md b/site2/website-next/docs/concepts-authentication.md index ae1232dd6d650..724b7104cb022 100644 --- a/site2/website-next/docs/concepts-authentication.md +++ b/site2/website-next/docs/concepts-authentication.md @@ -1,8 +1,12 @@ --- id: concepts-authentication title: Authentication and Authorization -sidebar_label: Authentication and Authorization +sidebar_label: "Authentication and Authorization" --- -Pulsar supports a pluggable [authentication](security-overview.md) mechanism which can be configured at the proxy and/or the broker. Pulsar also supports a pluggable [authorization](security-authorization.md) mechanism. These mechanisms work together to identify the client and its access rights on topics, namespaces and tenants. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar supports a pluggable [authentication](security-overview.md) mechanism which can be configured at the proxy and/or the broker. Pulsar also supports a pluggable [authorization](security-authorization) mechanism. These mechanisms work together to identify the client and its access rights on topics, namespaces and tenants. diff --git a/site2/website-next/docs/concepts-clients.md b/site2/website-next/docs/concepts-clients.md index 9fe6d46fd4583..d8084fadb9a7d 100644 --- a/site2/website-next/docs/concepts-clients.md +++ b/site2/website-next/docs/concepts-clients.md @@ -1,15 +1,19 @@ --- id: concepts-clients title: Pulsar Clients -sidebar_label: Clients +sidebar_label: "Clients" --- -Pulsar exposes a client API with language bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md), [C++](client-libraries-cpp.md) and [C#](client-libraries-dotnet.md). The client API optimizes and encapsulates Pulsar's client-broker communication protocol and exposes a simple and intuitive API for use by applications. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar exposes a client API with language bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md), [C++](client-libraries-cpp.md) and [C#](client-libraries-dotnet). The client API optimizes and encapsulates Pulsar's client-broker communication protocol and exposes a simple and intuitive API for use by applications. Under the hood, the current official Pulsar client libraries support transparent reconnection and/or connection failover to brokers, queuing of messages until acknowledged by the broker, and heuristics such as connection retries with backoff. > **Custom client libraries** -> If you'd like to create your own client library, we recommend consulting the documentation on Pulsar's custom [binary protocol](developing-binary-protocol.md). +> If you'd like to create your own client library, we recommend consulting the documentation on Pulsar's custom [binary protocol](developing-binary-protocol). ## Client setup phase @@ -37,7 +41,7 @@ Internally, the reader interface is implemented as a consumer using an exclusive [ **IMPORTANT** ] -Unlike subscription/consumer, readers are non-durable in nature and does not prevent data in a topic from being deleted, thus it is ***strongly*** advised that [data retention](cookbooks-retention-expiry.md) be configured. If data retention for a topic is not configured for an adequate amount of time, messages that the reader has not yet read might be deleted . This causes the readers to essentially skip messages. Configuring the data retention for a topic guarantees the reader with a certain duration to read a message. +Unlike subscription/consumer, readers are non-durable in nature and does not prevent data in a topic from being deleted, thus it is ***strongly*** advised that [data retention](cookbooks-retention-expiry) be configured. If data retention for a topic is not configured for an adequate amount of time, messages that the reader has not yet read might be deleted . This causes the readers to essentially skip messages. Configuring the data retention for a topic guarantees the reader with a certain duration to read a message. Please also note that a reader can have a "backlog", but the metric is only used for users to know how behind the reader is. The metric is not considered for any backlog quota calculations. @@ -46,6 +50,7 @@ Please also note that a reader can have a "backlog", but the metric is only used Here's a Java example that begins reading from the earliest available message on a topic: ```java + import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Reader; @@ -61,24 +66,30 @@ while (true) { // Process the message } + ``` To create a reader that reads from the latest available message: ```java + Reader reader = pulsarClient.newReader() .topic(topic) .startMessageId(MessageId.latest) .create(); + ``` To create a reader that reads from some message between the earliest and the latest: ```java + byte[] msgIdBytes = // Some byte array MessageId id = MessageId.fromByteArray(msgIdBytes); Reader reader = pulsarClient.newReader() .topic(topic) .startMessageId(id) .create(); + ``` + diff --git a/site2/website-next/docs/concepts-messaging.md b/site2/website-next/docs/concepts-messaging.md index 1da3297c115f2..8665c564a8fd4 100644 --- a/site2/website-next/docs/concepts-messaging.md +++ b/site2/website-next/docs/concepts-messaging.md @@ -1,9 +1,13 @@ --- id: concepts-messaging title: Messaging -sidebar_label: Messaging +sidebar_label: "Messaging" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar is built on the [publish-subscribe](https://en.wikipedia.org/wiki/Publish%E2%80%93subscribe_pattern) pattern (often abbreviated to pub-sub). In this pattern, [producers](#producers) publish messages to [topics](#topics); [consumers](#consumers) [subscribe](#subscription-modes) to those topics, process incoming messages, and send [acknowledgements](#acknowledgement) to the broker when processing is finished. When a subscription is created, Pulsar [retains](concepts-architecture-overview.md#persistent-storage) all messages, even if the consumer is disconnected. The retained messages are discarded only when a consumer acknowledges that all these messages are processed successfully. @@ -16,8 +20,8 @@ Messages are the basic "unit" of Pulsar. The following table lists the component Component | Description :---------|:------- -Value / data payload | The data carried by the message. All Pulsar messages contain raw bytes, although message data can also conform to data [schemas](schema-get-started.md). -Key | Messages are optionally tagged with keys, which is useful for things like [topic compaction](concepts-topic-compaction.md). +Value / data payload | The data carried by the message. All Pulsar messages contain raw bytes, although message data can also conform to data [schemas](schema-get-started). +Key | Messages are optionally tagged with keys, which is useful for things like [topic compaction](concepts-topic-compaction). Properties | An optional key/value map of user-defined properties. Producer name | The name of the producer who produces the message. If you do not specify a producer name, the default name is used. Sequence ID | Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID of the message is its order in that sequence. @@ -29,18 +33,23 @@ The default size of a message is 5 MB. You can configure the max size of a messa - In the `broker.conf` file. - ```bash - # The max size of a message (in bytes). - maxMessageSize=5242880 - ``` + ```bash + + # The max size of a message (in bytes). + maxMessageSize=5242880 + + ``` - In the `bookkeeper.conf` file. - ```bash - # The max size of the netty frame (in bytes). Any messages received larger than this value are rejected. The default value is 5 MB. - nettyMaxFrameSizeBytes=5253120 - ``` -> For more information on Pulsar messages, see Pulsar [binary protocol](developing-binary-protocol.md). + ```bash + + # The max size of the netty frame (in bytes). Any messages received larger than this value are rejected. The default value is 5 MB. + nettyMaxFrameSizeBytes=5253120 + + ``` + +> For more information on Pulsar messages, see Pulsar [binary protocol](developing-binary-protocol). ## Producers @@ -65,11 +74,12 @@ You can have different types of access modes on topics for producers. `Exclusive`|Only one producer can publish on a topic.

If there is already a producer connected, other producers trying to publish on this topic get errors immediately.

The “old” producer is evicted and a “new” producer is selected to be the next exclusive producer if the “old” producer experiences a network partition with the broker. `WaitForExclusive`|If there is already a producer connected, the producer creation is pending (rather than timing out) until the producer gets the `Exclusive` access.

The producer that succeeds in becoming the exclusive one is treated as the leader. Consequently, if you want to implement the leader election scheme for your application, you can use this access mode. -> **Note** -> -> Once an application creates a producer with `Exclusive` or `WaitForExclusive` access mode successfully, the instance of this application is guaranteed to be the **only writer** to the topic. Any other producers trying to produce messages on this topic will either get errors immediately or have to wait until they get the `Exclusive` access. -> -> For more information, see [PIP 68: Exclusive Producer](https://github.com/apache/pulsar/wiki/PIP-68:-Exclusive-Producer). +:::note + +Once an application creates a producer with `Exclusive` or `WaitForExclusive` access mode successfully, the instance of this application is guaranteed to be the **only writer** to the topic. Any other producers trying to produce messages on this topic will either get errors immediately or have to wait until they get the `Exclusive` access. +For more information, see [PIP 68: Exclusive Producer](https://github.com/apache/pulsar/wiki/PIP-68:-Exclusive-Producer). + +::: You can set producer access mode through Java Client API. For more information, see `ProducerAccessMode` in [ProducerBuilder.java](https://github.com/apache/pulsar/blob/fc5768ca3bbf92815d142fe30e6bfad70a1b4fc6/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ProducerBuilder.java) file. @@ -136,7 +146,7 @@ Messages are received from [brokers](reference-terminology.md#broker) either syn ### Listeners -Client libraries provide listener implementation for consumers. For example, the [Java client](client-libraries-java.md) provides a {@inject: javadoc:MesssageListener:/client/org/apache/pulsar/client/api/MessageListener} interface. In this interface, the `received` method is called whenever a new message is received. +Client libraries provide listener implementation for consumers. For example, the [Java client](client-libraries-java) provides a {@inject: javadoc:MesssageListener:/client/org/apache/pulsar/client/api/MessageListener} interface. In this interface, the `received` method is called whenever a new message is received. ### Acknowledgement @@ -152,16 +162,24 @@ Messages can be acknowledged in one of the following two ways: If you want to acknowledge messages individually, you can use the following API. ```java + consumer.acknowledge(msg); + ``` + If you want to acknowledge messages cumulatively, you can use the following API. + ```java + consumer.acknowledgeCumulative(msg); + ``` +:::note + +Cumulative acknowledgement cannot be used in the [shared subscription mode](#subscription-modes), because the shared subscription mode involves multiple consumers who have access to the same subscription. In the shared subscription mode, messages are acknowledged individually. -> **Note** -> Cumulative acknowledgement cannot be used in the [shared subscription mode](#subscription-modes), because the shared subscription mode involves multiple consumers who have access to the same subscription. In the shared subscription mode, messages are acknowledged individually. +::: ### Negative acknowledgement @@ -178,20 +196,28 @@ Be aware that negative acknowledgments on ordered subscription types, such as Ex If you want to acknowledge messages negatively, you can use the following API. ```java + //With calling this api, messages are negatively acknowledged consumer.negativeAcknowledge(msg); + ``` -> **Note** -> If batching is enabled, all messages in one batch are redelivered to the consumer. +:::note + +If batching is enabled, all messages in one batch are redelivered to the consumer. + +::: ### Acknowledgement timeout If a message is not consumed successfully, and you want the broker to redeliver this message automatically, then you can enable automatic redelivery mechanism for unacknowledged messages. With automatic redelivery enabled, the client tracks the unacknowledged messages within the entire `acktimeout` time range, and sends a `redeliver unacknowledged messages` request to the broker automatically when the acknowledgement timeout is specified. -> **Note** -> - If batching is enabled, all messages in one batch are redelivered to the consumer. -> - The negative acknowledgement is preferable over the acknowledgement timeout, since negative acknowledgement controls the redelivery of individual messages more precisely and avoids invalid redeliveries when the message processing time exceeds the acknowledgement timeout. +:::note + +- If batching is enabled, all messages in one batch are redelivered to the consumer. +- The negative acknowledgement is preferable over the acknowledgement timeout, since negative acknowledgement controls the redelivery of individual messages more precisely and avoids invalid redeliveries when the message processing time exceeds the acknowledgement timeout. + +::: ### Dead letter topic @@ -200,6 +226,7 @@ Dead letter topic enables you to consume new messages when some messages cannot The following example shows how to enable dead letter topic in a Java client using the default dead letter topic: ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -208,16 +235,21 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .maxRedeliverCount(maxRedeliveryCount) .build()) .subscribe(); - + ``` + The default dead letter topic uses this format: + ``` + --DLQ + ``` If you want to specify the name of the dead letter topic, use this Java client example: ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -227,13 +259,16 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .deadLetterTopic("your-topic-name") .build()) .subscribe(); - + ``` Dead letter topic depends on message redelivery. Messages are redelivered either due to [acknowledgement timeout](#acknowledgement-timeout) or [negative acknowledgement](#negative-acknowledgement). If you are going to use negative acknowledgement on a message, make sure it is negatively acknowledged before the acknowledgement timeout. -> **Note** -> Currently, dead letter topic is enabled in the Shared and Key_Shared subscription modes. +:::note + +Currently, dead letter topic is enabled in the Shared and Key_Shared subscription modes. + +::: ### Retry letter topic @@ -244,6 +279,7 @@ By default, automatic retry is disabled. You can set `enableRetry` to `true` to This example shows how to consume messages from a retry letter topic. ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -256,22 +292,25 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .build()) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); + ``` If you want to put messages into a retrial queue, you can use the following API. ```java -consumer.reconsumeLater(msg,3,TimeUnit.SECONDS); -``` +consumer.reconsumeLater(msg,3,TimeUnit.SECONDS); +``` ## Topics As in other pub-sub systems, topics in Pulsar are named channels for transmitting messages from producers to consumers. Topic names are URLs that have a well-defined structure: ```http + {persistent|non-persistent}://tenant/namespace/topic + ``` Topic name component | Description @@ -353,28 +392,57 @@ The key-based batching aims at resolving the above-mentioned issues. This batchi Below are examples of enabling the key-based batching under the Key_Shared subscription mode, with `client` being the Pulsar client that you created. - - + + + ``` + Producer producer = client.newProducer() .topic("my-topic") .batcherBuilder(BatcherBuilder.KEY_BASED) .create(); + ``` - + + + ``` + ProducerConfiguration producerConfig; producerConfig.setBatchingType(ProducerConfiguration::BatchingType::KeyBasedBatching); Producer producer; client.createProducer("my-topic", producerConfig, producer); + ``` - + + + ``` + producer = client.create_producer(topic='my-topic', batching_type=pulsar.BatchingType.KeyBased) + ``` - + + + + > **Limitations of Key_Shared mode** > When you use Key_Shared mode, be aware that: @@ -398,6 +466,7 @@ When subscribing to multiple topics, the Pulsar client automatically makes a cal The following are multi-topic subscription examples for Java. ```java + import java.util.regex.Pattern; import org.apache.pulsar.client.api.Consumer; @@ -418,6 +487,7 @@ Consumer someTopicsConsumer = pulsarClient.newConsumer() .topicsPattern(someTopicsInNamespace) .subscriptionName("subscription-1") .subscribe(); + ``` For code examples, see [Java](client-libraries-java.md#multi-topic-subscriptions). @@ -440,7 +510,7 @@ Decisions about routing and subscription modes can be made separately in most ca There is no difference between partitioned topics and normal topics in terms of how subscription modes work, as partitioning only determines what happens between when a message is published by a producer and processed and acknowledged by a consumer. -Partitioned topics need to be explicitly created via the [admin API](admin-api-overview.md). The number of partitions can be specified when creating the topic. +Partitioned topics need to be explicitly created via the [admin API](admin-api-overview). The number of partitions can be specified when creating the topic. ### Routing modes @@ -452,7 +522,7 @@ Mode | Description :--------|:------------ `RoundRobinPartition` | If no key is provided, the producer will publish messages across all partitions in round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it's set to the same boundary of batching delay, to ensure batching is effective. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. This is the default mode. `SinglePartition` | If no key is provided, the producer will randomly pick one single partition and publish all the messages into that partition. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. -`CustomPartition` | Use custom message router implementation that will be called to determine the partition for a particular message. User can create a custom routing mode by using the [Java client](client-libraries-java.md) and implementing the {@inject: javadoc:MessageRouter:/client/org/apache/pulsar/client/api/MessageRouter} interface. +`CustomPartition` | Use custom message router implementation that will be called to determine the partition for a particular message. User can create a custom routing mode by using the [Java client](client-libraries-java) and implementing the {@inject: javadoc:MessageRouter:/client/org/apache/pulsar/client/api/MessageRouter} interface. ### Ordering guarantee @@ -485,10 +555,12 @@ Pulsar also, however, supports **non-persistent topics**, which are topics on wh Non-persistent topics have names of this form (note the `non-persistent` in the name): ```http + non-persistent://tenant/namespace/topic + ``` -> For more info on using non-persistent topics, see the [Non-persistent messaging cookbook](cookbooks-non-persistent.md). +> For more info on using non-persistent topics, see the [Non-persistent messaging cookbook](cookbooks-non-persistent). In non-persistent topics, brokers immediately deliver messages to all connected subscribers *without persisting them* in [BookKeeper](concepts-architecture-overview.md#persistent-storage). If a subscriber is disconnected, the broker will not be able to deliver those in-transit messages, and subscribers will never be able to receive those messages again. Eliminating the persistent storage step makes messaging on non-persistent topics slightly faster than on persistent topics in some cases, but with the caveat that some of the core benefits of Pulsar are lost. @@ -507,6 +579,7 @@ Producers and consumers can connect to non-persistent topics in the same way as Here's an example [Java consumer](client-libraries-java.md#consumers) for a non-persistent topic: ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .build(); @@ -517,14 +590,17 @@ Consumer consumer = client.newConsumer() .topic(npTopic) .subscriptionName(subscriptionName) .subscribe(); + ``` Here's an example [Java producer](client-libraries-java.md#producer) for the same non-persistent topic: ```java + Producer producer = client.newProducer() .topic(npTopic) .create(); + ``` ## Message retention and expiry @@ -539,15 +615,15 @@ Pulsar has two features, however, that enable you to override this default behav * Message **retention** enables you to store messages that have been acknowledged by a consumer * Message **expiry** enables you to set a time to live (TTL) for messages that have not yet been acknowledged -> All message retention and expiry is managed at the [namespace](#namespaces) level. For a how-to, see the [Message retention and expiry](cookbooks-retention-expiry.md) cookbook. +> All message retention and expiry is managed at the [namespace](#namespaces) level. For a how-to, see the [Message retention and expiry](cookbooks-retention-expiry) cookbook. The diagram below illustrates both concepts: ![Message retention and expiry](/assets/retention-expiry.png) -With message retention, shown at the top, a retention policy applied to all topics in a namespace dictates that some messages are durably stored in Pulsar even though they've already been acknowledged. Acknowledged messages that are not covered by the retention policy are deleted. Without a retention policy, *all* of the acknowledged messages would be deleted. +With message retention, shown at the top, a retention policy applied to all topics in a namespace dictates that some messages are durably stored in Pulsar even though they've already been acknowledged. Acknowledged messages that are not covered by the retention policy are deleted. Without a retention policy, *all* of the acknowledged messages would be deleted. -With message expiry, shown at the bottom, some messages are deleted, even though they haven't been acknowledged, because they've expired according to the TTL applied to the namespace (for example because a TTL of 5 minutes has been applied and the messages haven't been acknowledged but are 10 minutes old). +With message expiry, shown at the bottom, some messages are deleted, even though they haven't been acknowledged, because they've expired according to the TTL applied to the namespace (for example because a TTL of 5 minutes has been applied and the messages haven't been acknowledged but are 10 minutes old). ## Message deduplication @@ -562,12 +638,12 @@ Message deduplication is disabled in the scenario shown at the top. Here, a prod In the second scenario at the bottom, the producer publishes message 1, which is received by the broker and persisted, as in the first scenario. When the producer attempts to publish the message again, however, the broker knows that it has already seen message 1 and thus does not persist the message. -> Message deduplication is handled at the namespace level or the topic level. For more instructions, see the [message deduplication cookbook](cookbooks-deduplication.md). +> Message deduplication is handled at the namespace level or the topic level. For more instructions, see the [message deduplication cookbook](cookbooks-deduplication). ### Producer idempotency -The other available approach to message deduplication is to ensure that each message is *only produced once*. This approach is typically called **producer idempotency**. The drawback of this approach is that it defers the work of message deduplication to the application. In Pulsar, this is handled at the [broker](reference-terminology.md#broker) level, so you do not need to modify your Pulsar client code. Instead, you only need to make administrative changes. For details, see [Managing message deduplication](cookbooks-deduplication.md). +The other available approach to message deduplication is to ensure that each message is *only produced once*. This approach is typically called **producer idempotency**. The drawback of this approach is that it defers the work of message deduplication to the application. In Pulsar, this is handled at the [broker](reference-terminology.md#broker) level, so you do not need to modify your Pulsar client code. Instead, you only need to make administrative changes. For details, see [Managing message deduplication](cookbooks-deduplication). ### Deduplication and effectively-once semantics @@ -590,6 +666,7 @@ A broker saves a message without any check. When a consumer consumes a message, Delayed message delivery is enabled by default. You can change it in the broker configuration file as below: ``` + # Whether to enable the delayed delivery for messages. # If disabled, messages are immediately delivered and there is no tracking overhead. delayedDeliveryEnabled=true @@ -598,11 +675,16 @@ delayedDeliveryEnabled=true # affecting the accuracy of the delivery time compared to the scheduled time. # Default is 1 second. delayedDeliveryTickTimeMillis=1000 + ``` ### Producer The following is an example of delayed message delivery for a producer in Java: + ```java + // message to be delivered at the configured delay interval producer.newMessage().deliverAfter(3L, TimeUnit.Minute).value("Hello Pulsar!").send(); + ``` + diff --git a/site2/website-next/docs/concepts-multi-tenancy.md b/site2/website-next/docs/concepts-multi-tenancy.md index 06d724ceb6741..a831305c59062 100644 --- a/site2/website-next/docs/concepts-multi-tenancy.md +++ b/site2/website-next/docs/concepts-multi-tenancy.md @@ -1,15 +1,21 @@ --- id: concepts-multi-tenancy title: Multi Tenancy -sidebar_label: Multi Tenancy +sidebar_label: "Multi Tenancy" --- -Pulsar was created from the ground up as a multi-tenant system. To support multi-tenancy, Pulsar has a concept of tenants. Tenants can be spread across clusters and can each have their own [authentication and authorization](security-overview.md) scheme applied to them. They are also the administrative unit at which storage quotas, [message TTL](cookbooks-retention-expiry.md#time-to-live-ttl), and isolation policies can be managed. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar was created from the ground up as a multi-tenant system. To support multi-tenancy, Pulsar has a concept of tenants. Tenants can be spread across clusters and can each have their own [authentication and authorization](security-overview) scheme applied to them. They are also the administrative unit at which storage quotas, [message TTL](cookbooks-retention-expiry.md#time-to-live-ttl), and isolation policies can be managed. The multi-tenant nature of Pulsar is reflected mostly visibly in topic URLs, which have this structure: ```http + persistent://tenant/namespace/topic + ``` As you can see, the tenant is the most basic unit of categorization for topics (more fundamental than the namespace and topic name). @@ -18,7 +24,7 @@ As you can see, the tenant is the most basic unit of categorization for topics ( To each tenant in a Pulsar instance you can assign: -* An [authorization](security-authorization.md) scheme +* An [authorization](security-authorization) scheme * The set of [clusters](reference-terminology.md#cluster) to which the tenant's configuration applies ## Namespaces @@ -26,16 +32,18 @@ To each tenant in a Pulsar instance you can assign: Tenants and namespaces are two key concepts of Pulsar to support multi-tenancy. * Pulsar is provisioned for specified tenants with appropriate capacity allocated to the tenant. -* A namespace is the administrative unit nomenclature within a tenant. The configuration policies set on a namespace apply to all the topics created in that namespace. A tenant may create multiple namespaces via self-administration using the REST API and the [`pulsar-admin`](reference-pulsar-admin.md) CLI tool. For instance, a tenant with different applications can create a separate namespace for each application. +* A namespace is the administrative unit nomenclature within a tenant. The configuration policies set on a namespace apply to all the topics created in that namespace. A tenant may create multiple namespaces via self-administration using the REST API and the [`pulsar-admin`](reference-pulsar-admin) CLI tool. For instance, a tenant with different applications can create a separate namespace for each application. Names for topics in the same namespace will look like this: ```http + persistent://tenant/app1/topic-1 persistent://tenant/app1/topic-2 persistent://tenant/app1/topic-3 + ``` ### Namespace change events and topic-level policies diff --git a/site2/website-next/docs/concepts-multiple-advertised-listeners.md b/site2/website-next/docs/concepts-multiple-advertised-listeners.md index 8ba0db9880ed6..d9767b505a2fc 100644 --- a/site2/website-next/docs/concepts-multiple-advertised-listeners.md +++ b/site2/website-next/docs/concepts-multiple-advertised-listeners.md @@ -1,9 +1,13 @@ --- id: concepts-multiple-advertised-listeners title: Multiple advertised listeners -sidebar_label: Multiple advertised listeners +sidebar_label: "Multiple advertised listeners" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + When a Pulsar cluster is deployed in the production environment, it may require to expose multiple advertised addresses for the broker. For example, when you deploy a Pulsar cluster in Kubernetes and want other clients, which are not in the same Kubernetes cluster, to connect to the Pulsar cluster, you need to assign a broker URL to external clients. But clients in the same Kubernetes cluster can still connect to the Pulsar cluster through the internal network of Kubernetes. ## Advertised listeners @@ -24,15 +28,20 @@ This example shows how a Pulsar client uses multiple advertised listeners. 1. Configure multiple advertised listeners in the broker configuration file. ```shell + advertisedListeners={listenerName}:pulsar://xxxx:6650, {listenerName}:pulsar+ssl://xxxx:6651 + ``` 2. Specify the listener name for the client. ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://xxxx:6650") .listenerName("external") .build(); -``` \ No newline at end of file + +``` + diff --git a/site2/website-next/docs/concepts-overview.md b/site2/website-next/docs/concepts-overview.md index e30443eac1555..501f3fd367a77 100644 --- a/site2/website-next/docs/concepts-overview.md +++ b/site2/website-next/docs/concepts-overview.md @@ -1,30 +1,34 @@ --- id: concepts-overview title: Pulsar Overview -sidebar_label: Overview +sidebar_label: "Overview" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar is a multi-tenant, high-performance solution for server-to-server messaging. Pulsar was originally developed by Yahoo, it is under the stewardship of the [Apache Software Foundation](https://www.apache.org/). Key features of Pulsar are listed below: -* Native support for multiple clusters in a Pulsar instance, with seamless [geo-replication](administration-geo.md) of messages across clusters. +* Native support for multiple clusters in a Pulsar instance, with seamless [geo-replication](administration-geo) of messages across clusters. * Very low publish and end-to-end latency. * Seamless scalability to over a million topics. -* A simple [client API](concepts-clients.md) with bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) and [C++](client-libraries-cpp.md). +* A simple [client API](concepts-clients.md) with bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) and [C++](client-libraries-cpp). * Multiple [subscription modes](concepts-messaging.md#subscription-modes) ([exclusive](concepts-messaging.md#exclusive), [shared](concepts-messaging.md#shared), and [failover](concepts-messaging.md#failover)) for topics. * Guaranteed message delivery with [persistent message storage](concepts-architecture-overview.md#persistent-storage) provided by [Apache BookKeeper](http://bookkeeper.apache.org/). -* A serverless light-weight computing framework [Pulsar Functions](functions-overview.md) offers the capability for stream-native data processing. -* A serverless connector framework [Pulsar IO](io-overview.md), which is built on Pulsar Functions, makes it easier to move data in and out of Apache Pulsar. -* [Tiered Storage](concepts-tiered-storage.md) offloads data from hot/warm storage to cold/longterm storage (such as S3 and GCS) when the data is aging out. +* A serverless light-weight computing framework [Pulsar Functions](functions-overview) offers the capability for stream-native data processing. +* A serverless connector framework [Pulsar IO](io-overview), which is built on Pulsar Functions, makes it easier to move data in and out of Apache Pulsar. +* [Tiered Storage](concepts-tiered-storage) offloads data from hot/warm storage to cold/longterm storage (such as S3 and GCS) when the data is aging out. ## Contents -- [Messaging Concepts](concepts-messaging.md) -- [Architecture Overview](concepts-architecture-overview.md) -- [Pulsar Clients](concepts-clients.md) -- [Geo Replication](concepts-replication.md) -- [Multi Tenancy](concepts-multi-tenancy.md) -- [Authentication and Authorization](concepts-authentication.md) -- [Topic Compaction](concepts-topic-compaction.md) -- [Tiered Storage](concepts-tiered-storage.md) +- [Messaging Concepts](concepts-messaging) +- [Architecture Overview](concepts-architecture-overview) +- [Pulsar Clients](concepts-clients) +- [Geo Replication](concepts-replication) +- [Multi Tenancy](concepts-multi-tenancy) +- [Authentication and Authorization](concepts-authentication) +- [Topic Compaction](concepts-topic-compaction) +- [Tiered Storage](concepts-tiered-storage) diff --git a/site2/website-next/docs/concepts-proxy-sni-routing.md b/site2/website-next/docs/concepts-proxy-sni-routing.md index 3574b541cea1b..efed49da32e5d 100644 --- a/site2/website-next/docs/concepts-proxy-sni-routing.md +++ b/site2/website-next/docs/concepts-proxy-sni-routing.md @@ -1,9 +1,13 @@ --- id: concepts-proxy-sni-routing title: Proxy support with SNI routing -sidebar_label: Proxy support with SNI routing +sidebar_label: "Proxy support with SNI routing" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + A proxy server is an intermediary server that forwards requests from multiple clients to different servers across the Internet. The proxy server acts as a "traffic cop" in both forward and reverse proxy scenarios, and benefits your system such as load balancing, performance, security, auto-scaling, and so on. The proxy in Pulsar acts as a reverse proxy, and creates a gateway in front of brokers. Proxies such as Apache Traffic Server (ATS), HAProxy, Nginx, and Envoy are not supported in Pulsar. These proxy-servers support **SNI routing**. SNI routing is used to route traffic to a destination without terminating the SSL connection. Layer 4 routing provides greater transparency because the outbound connection is determined by examining the destination address in the client TCP packets. @@ -31,6 +35,7 @@ To configure the `records.config` files, complete the following steps. The following is an example. ``` + # PROXY TLS PORT CONFIG proxy.config.http.server_ports STRING 4443:ssl 4080 # PROXY CERTS FILE PATH @@ -41,6 +46,7 @@ CONFIG proxy.config.ssl.client.cert.filename STRING /proxy-key.pem # The range of origin server ports that can be used for tunneling via CONNECT. # Traffic Server allows tunnels only to the specified ports. Supports both wildcards (*) and ranges (e.g. 0-1023). CONFIG proxy.config.http.connect_ports STRING 4443 6651 + ``` The [ssl_server_name](https://docs.trafficserver.apache.org/en/8.0.x/admin-guide/files/ssl_server_name.yaml.en.html) file is used to configure TLS connection handling for inbound and outbound connections. The configuration is determined by the SNI values provided by the inbound connection. The file consists of a set of configuration items, and each is identified by an SNI value (`fqdn`). When an inbound TLS connection is made, the SNI value from the TLS negotiation is matched with the items specified in this file. If the values match, the values specified in that item override the default values. @@ -48,6 +54,7 @@ The [ssl_server_name](https://docs.trafficserver.apache.org/en/8.0.x/admin-guide The following example shows mapping of the inbound SNI hostname coming from the client, and the actual broker service URL where request should be redirected. For example, if the client sends the SNI header `pulsar-broker1`, the proxy creates a TLS tunnel by redirecting request to the `pulsar-broker1:6651` service URL. ``` + server_config = { { fqdn = 'pulsar-broker-vip', @@ -65,6 +72,7 @@ server_config = { tunnel_route = 'pulsar-broker2:6651' }, } + ``` After you configure the `ssl_server_name.config` and `records.config` files, the ATS-proxy server handles SNI routing and creates TCP tunnel between the client and the broker. @@ -72,11 +80,27 @@ After you configure the `ssl_server_name.config` and `records.config` files, the ### Configure Pulsar-client with SNI routing ATS SNI-routing works only with TLS. You need to enable TLS for the ATS proxy and brokers first, configure the SNI routing protocol, and then connect Pulsar clients to brokers through ATS proxy. Pulsar clients support SNI routing by connecting to the proxy, and sending the target broker URL to the SNI header. This process is processed internally. You only need to configure the following proxy configuration initially when you create a Pulsar client to use the SNI routing protocol. - + - + ```java + String brokerServiceUrl = “pulsar+ssl://pulsar-broker-vip:6651/”; String proxyUrl = “pulsar+ssl://ats-proxy:443”; ClientBuilder clientBuilder = PulsarClient.builder() @@ -87,17 +111,20 @@ ClientBuilder clientBuilder = PulsarClient.builder() .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI) .operationTimeout(1000, TimeUnit.MILLISECONDS); -Map authParams = new HashMap<>(); +Map authParams = new HashMap(); authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH); authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH); clientBuilder.authentication(AuthenticationTls.class.getName(), authParams); PulsarClient pulsarClient = clientBuilder.build(); + ``` - + + ```c++ + ClientConfiguration config = ClientConfiguration(); config.setUseTls(true); config.setTlsTrustCertsFilePath("/path/to/cacert.pem"); @@ -106,11 +133,14 @@ config.setAuth(pulsar::AuthTls::create( "/path/to/client-cert.pem", "/path/to/client-key.pem");); Client client("pulsar+ssl://ats-proxy:443", config); + ``` - + + ```python + from pulsar import Client, AuthenticationTLS auth = AuthenticationTLS("/path/to/my-role.cert.pem", "/path/to/my-role.key-pk8.pem") @@ -118,9 +148,12 @@ client = Client("pulsar+ssl://ats-proxy:443", tls_trust_certs_file_path="/path/to/ca.cert.pem", tls_allow_insecure_connection=False, authentication=auth) + ``` - + + + ### Pulsar geo-replication with SNI routing You can use the ATS proxy for geo-replication. Pulsar brokers can connect to brokers in geo-replication by using SNI routing. To enable SNI routing for broker connection cross clusters, you need to configure SNI proxy URL to the cluster metadata. If you have configured SNI proxy URL in the cluster metadata, you can connect to broker cross clusters through the proxy over SNI routing. @@ -132,19 +165,24 @@ In this example, a Pulsar cluster is deployed into two separate regions, `us-wes (a) Configure the cluster metadata for `us-east` with `us-east` broker service URL and `us-east` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://east-broker-vip:6651 \ --url http://east-broker-vip:8080 \ --proxy-protocol SNI \ --proxy-url pulsar+ssl://east-ats-proxy:443 + ``` (b) Configure the cluster metadata for `us-west` with `us-west` broker service URL and `us-west` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://west-broker-vip:6651 \ --url http://west-broker-vip:8080 \ --proxy-protocol SNI \ --proxy-url pulsar+ssl://west-ats-proxy:443 + ``` + diff --git a/site2/website-next/docs/concepts-replication.md b/site2/website-next/docs/concepts-replication.md index 3d1c823919177..95444fe59e394 100644 --- a/site2/website-next/docs/concepts-replication.md +++ b/site2/website-next/docs/concepts-replication.md @@ -1,8 +1,12 @@ --- id: concepts-replication title: Geo Replication -sidebar_label: Geo Replication +sidebar_label: "Geo Replication" --- -Pulsar enables messages to be produced and consumed in different geo-locations. For instance, your application may be publishing data in one region or market and you would like to process it for consumption in other regions or markets. [Geo-replication](administration-geo.md) in Pulsar enables you to do that. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar enables messages to be produced and consumed in different geo-locations. For instance, your application may be publishing data in one region or market and you would like to process it for consumption in other regions or markets. [Geo-replication](administration-geo) in Pulsar enables you to do that. diff --git a/site2/website-next/docs/concepts-schema-registry.md b/site2/website-next/docs/concepts-schema-registry.md index b405c42430116..dfe639b0d8e7a 100644 --- a/site2/website-next/docs/concepts-schema-registry.md +++ b/site2/website-next/docs/concepts-schema-registry.md @@ -1,5 +1,9 @@ --- id: concepts-schema-registry title: Schema Registry -sidebar_label: Schema Registry +sidebar_label: "Schema Registry" --- + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + diff --git a/site2/website-next/docs/concepts-tiered-storage.md b/site2/website-next/docs/concepts-tiered-storage.md index 144828761f89e..25f0808428275 100644 --- a/site2/website-next/docs/concepts-tiered-storage.md +++ b/site2/website-next/docs/concepts-tiered-storage.md @@ -1,9 +1,13 @@ --- id: concepts-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar's segment oriented architecture allows for topic backlogs to grow very large, effectively without limit. However, this can become expensive over time. One way to alleviate this cost is to use Tiered Storage. With tiered storage, older messages in the backlog can be moved from BookKeeper to a cheaper storage mechanism, while still allowing clients to access the backlog as if nothing had changed. @@ -14,4 +18,4 @@ One way to alleviate this cost is to use Tiered Storage. With tiered storage, ol Pulsar currently supports S3, Google Cloud Storage (GCS), and filesystem for [long term store](https://pulsar.apache.org/docs/en/cookbooks-tiered-storage/). Offloading to long term storage triggered via a Rest API or command line interface. The user passes in the amount of topic data they wish to retain on BookKeeper, and the broker will copy the backlog data to long term storage. The original data will then be deleted from BookKeeper after a configured delay (4 hours by default). -> For a guide for setting up tiered storage, see the [Tiered storage cookbook](cookbooks-tiered-storage.md). +> For a guide for setting up tiered storage, see the [Tiered storage cookbook](cookbooks-tiered-storage). diff --git a/site2/website-next/docs/concepts-topic-compaction.md b/site2/website-next/docs/concepts-topic-compaction.md index 96f013615211b..2c56fb16eceae 100644 --- a/site2/website-next/docs/concepts-topic-compaction.md +++ b/site2/website-next/docs/concepts-topic-compaction.md @@ -1,12 +1,16 @@ --- id: concepts-topic-compaction title: Topic Compaction -sidebar_label: Topic Compaction +sidebar_label: "Topic Compaction" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar was built with highly scalable [persistent storage](concepts-architecture-overview.md#persistent-storage) of message data as a primary objective. Pulsar topics enable you to persistently store as many unacknowledged messages as you need while preserving message ordering. By default, Pulsar stores *all* unacknowledged/unprocessed messages produced on a topic. Accumulating many unacknowledged messages on a topic is necessary for many Pulsar use cases but it can also be very time intensive for Pulsar consumers to "rewind" through the entire log of messages. -> For a more practical guide to topic compaction, see the [Topic compaction cookbook](cookbooks-compaction.md). +> For a more practical guide to topic compaction, see the [Topic compaction cookbook](cookbooks-compaction). For some use cases consumers don't need a complete "image" of the topic log. They may only need a few values to construct a more "shallow" image of the log, perhaps even just the most recent value. For these kinds of use cases Pulsar offers **topic compaction**. When you run compaction on a topic, Pulsar goes through a topic's backlog and removes messages that are *obscured* by later messages, i.e. it goes through the topic on a per-key basis and leaves only the most recent message associated with that key. @@ -14,7 +18,7 @@ Pulsar's topic compaction feature: * Allows for faster "rewind" through topic logs * Applies only to [persistent topics](concepts-architecture-overview.md#persistent-storage) -* Triggered automatically when the backlog reaches a certain size or can be triggered manually via the command line. See the [Topic compaction cookbook](cookbooks-compaction.md) +* Triggered automatically when the backlog reaches a certain size or can be triggered manually via the command line. See the [Topic compaction cookbook](cookbooks-compaction) * Is conceptually and operationally distinct from [retention and expiry](concepts-messaging.md#message-retention-and-expiry). Topic compaction *does*, however, respect retention. If retention has removed a message from the message backlog of a topic, the message will also not be readable from the compacted topic ledger. > #### Topic compaction example: the stock ticker @@ -23,7 +27,7 @@ Pulsar's topic compaction feature: ## How topic compaction works -When topic compaction is triggered [via the CLI](cookbooks-compaction.md), Pulsar will iterate over the entire topic from beginning to end. For each key that it encounters the compaction routine will keep a record of the latest occurrence of that key. +When topic compaction is triggered [via the CLI](cookbooks-compaction), Pulsar will iterate over the entire topic from beginning to end. For each key that it encounters the compaction routine will keep a record of the latest occurrence of that key. After that, the broker will create a new [BookKeeper ledger](concepts-architecture-overview.md#ledgers) and make a second iteration through each message on the topic. For each message, if the key matches the latest occurrence of that key, then the key's data payload, message ID, and metadata will be written to the newly created ledger. If the key doesn't match the latest then the message will be skipped and left alone. If any given message has an empty payload, it will be skipped and considered deleted (akin to the concept of [tombstones](https://en.wikipedia.org/wiki/Tombstone_(data_store)) in key-value databases). At the end of this second iteration through the topic, the newly created BookKeeper ledger is closed and two things are written to the topic's metadata: the ID of the BookKeeper ledger and the message ID of the last compacted message (this is known as the **compaction horizon** of the topic). Once this metadata is written compaction is complete. diff --git a/site2/website-next/docs/concepts-transactions.md b/site2/website-next/docs/concepts-transactions.md index ab8818faed0a6..471b7f5cc5953 100644 --- a/site2/website-next/docs/concepts-transactions.md +++ b/site2/website-next/docs/concepts-transactions.md @@ -1,9 +1,13 @@ --- id: transactions title: Transactions -sidebar_label: Overview +sidebar_label: "Overview" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Transactional semantics enable event streaming applications to consume, process, and produce messages in one atomic operation. In Pulsar, a producer or consumer can work with messages across multiple topics and partitions and ensure those messages are processed as a single unit. The following concepts help you understand Pulsar transactions. diff --git a/site2/website-next/docs/cookbooks-compaction.md b/site2/website-next/docs/cookbooks-compaction.md index 9714d831bb172..dd845370d9018 100644 --- a/site2/website-next/docs/cookbooks-compaction.md +++ b/site2/website-next/docs/cookbooks-compaction.md @@ -1,7 +1,7 @@ --- id: cookbooks-compaction title: Topic compaction -sidebar_label: Topic compaction +sidebar_label: "Topic compaction" --- import Tabs from '@theme/Tabs'; @@ -142,3 +142,4 @@ Message msg = MessageBuilder.create() compactedTopicProducer.send(msg); ``` + diff --git a/site2/website-next/docs/cookbooks-deduplication.md b/site2/website-next/docs/cookbooks-deduplication.md index e95f5d68e5784..5b6d5231c86cd 100644 --- a/site2/website-next/docs/cookbooks-deduplication.md +++ b/site2/website-next/docs/cookbooks-deduplication.md @@ -1,7 +1,7 @@ --- id: cookbooks-deduplication title: Message deduplication -sidebar_label: Message deduplication +sidebar_label: "Message deduplication " --- import Tabs from '@theme/Tabs'; @@ -127,6 +127,7 @@ producer = client.create_producer( send_timeout_millis=0) ``` + @@ -151,6 +152,7 @@ Producer producer; Result result = client.createProducer(topic, producerConfig, producer); ``` + \ No newline at end of file diff --git a/site2/website-next/docs/cookbooks-encryption.md b/site2/website-next/docs/cookbooks-encryption.md index aa00d4826a399..0ab0d9daf4e9f 100644 --- a/site2/website-next/docs/cookbooks-encryption.md +++ b/site2/website-next/docs/cookbooks-encryption.md @@ -1,7 +1,7 @@ --- id: cookbooks-encryption title: Pulsar Encryption -sidebar_label: Encryption +sidebar_label: "Encryption " --- import Tabs from '@theme/Tabs'; @@ -38,6 +38,7 @@ openssl ecparam -name secp521r1 -genkey -param_enc explicit -out test_ecdsa_priv openssl ec -in test_ecdsa_privkey.pem -pubout -outform pkcs8 -out test_ecdsa_pubkey.pem ``` + 2. Add the public and private key to the key management and configure your producers to retrieve public keys and consumers clients to retrieve private keys. 3. Implement CryptoKeyReader::getPublicKey() interface from producer and CryptoKeyReader::getPrivateKey() interface from consumer, which will be invoked by Pulsar client to load the key. 4. Add encryption key to producer configuration: conf.addEncryptionKey("myapp.key") @@ -95,6 +96,7 @@ for (int i = 0; i < 10; i++) { pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -170,6 +172,7 @@ conf.addEncryptionKey("myapp.messagekey1"); conf.addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypting encrypted messages at the consumer application: Consumers require access one of the private keys to decrypt messages produced by the producer. If you would like to receive encrypted messages, create a public/private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/docs/cookbooks-message-queue.md b/site2/website-next/docs/cookbooks-message-queue.md index 7f83c4eedaf0a..b2f7895f0d2b5 100644 --- a/site2/website-next/docs/cookbooks-message-queue.md +++ b/site2/website-next/docs/cookbooks-message-queue.md @@ -1,7 +1,7 @@ --- id: cookbooks-message-queue title: Using Pulsar as a message queue -sidebar_label: Message queue +sidebar_label: "Message queue" --- import Tabs from '@theme/Tabs'; @@ -127,3 +127,4 @@ if err != nil { } ``` + diff --git a/site2/website-next/docs/cookbooks-non-persistent.md b/site2/website-next/docs/cookbooks-non-persistent.md index fe380886cec7f..983958c882728 100644 --- a/site2/website-next/docs/cookbooks-non-persistent.md +++ b/site2/website-next/docs/cookbooks-non-persistent.md @@ -1,7 +1,7 @@ --- id: cookbooks-non-persistent title: Non-persistent messaging -sidebar_label: Non-persistent messaging +sidebar_label: "Non-persistent messaging" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/cookbooks-partitioned.md b/site2/website-next/docs/cookbooks-partitioned.md index a720823f1fc13..244b7aaaf4353 100644 --- a/site2/website-next/docs/cookbooks-partitioned.md +++ b/site2/website-next/docs/cookbooks-partitioned.md @@ -1,7 +1,7 @@ --- id: cookbooks-partitioned title: Partitioned topics -sidebar_label: Partitioned Topics +sidebar_label: "Partitioned Topics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/cookbooks-retention-expiry.md b/site2/website-next/docs/cookbooks-retention-expiry.md index 09071660220af..a470eff40b986 100644 --- a/site2/website-next/docs/cookbooks-retention-expiry.md +++ b/site2/website-next/docs/cookbooks-retention-expiry.md @@ -1,7 +1,7 @@ --- id: cookbooks-retention-expiry title: Message retention and expiry -sidebar_label: Message retention and expiry +sidebar_label: "Message retention and expiry" --- import Tabs from '@theme/Tabs'; @@ -78,6 +78,7 @@ You can set a retention policy for a namespace by specifying the namespace, a si } ]}> + You can use the [`set-retention`](reference-pulsar-admin.md#namespaces-set-retention) subcommand and specify a namespace, a size limit using the `-s`/`--size` flag, and a time limit using the `-t`/`--time` flag. In the following example, the size limit is set to 10 GB and the time limit is set to 3 hours for each topic within the `my-tenant/my-ns` namespace. @@ -208,6 +209,7 @@ $ pulsar-admin namespaces get-retention my-tenant/my-ns admin.namespaces().getRetention(namespace); ``` + @@ -297,6 +299,7 @@ BacklogQuota quota = new BacklogQuota(sizeLimit, policy); admin.namespaces().setBacklogQuota(namespace, quota); ``` + @@ -351,6 +354,7 @@ Map quotas = admin.namespaces().getBacklogQuotas(namespace); ``` + @@ -396,6 +400,7 @@ $ pulsar-admin namespaces remove-backlog-quota my-tenant/my-ns admin.namespaces().removeBacklogQuota(namespace); ``` + @@ -464,6 +469,7 @@ $ pulsar-admin namespaces set-message-ttl my-tenant/my-ns \ admin.namespaces().setNamespaceMessageTTL(namespace, ttlInSeconds); ``` + @@ -512,6 +518,7 @@ $ pulsar-admin namespaces get-message-ttl my-tenant/my-ns admin.namespaces().getNamespaceMessageTTL(namespace) ``` + @@ -559,6 +566,7 @@ $ pulsar-admin namespaces remove-message-ttl my-tenant/my-ns admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + diff --git a/site2/website-next/docs/cookbooks-tiered-storage.md b/site2/website-next/docs/cookbooks-tiered-storage.md index 61f14878b2c48..a41e2bb126b98 100644 --- a/site2/website-next/docs/cookbooks-tiered-storage.md +++ b/site2/website-next/docs/cookbooks-tiered-storage.md @@ -1,7 +1,7 @@ --- id: cookbooks-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" --- import Tabs from '@theme/Tabs'; @@ -54,7 +54,7 @@ Currently we support driver of types: > though it requires that you specify an endpoint url using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if > using a S3 compatible data store, other than AWS. -```conf +``` managedLedgerOffloadDriver=aws-s3 @@ -69,7 +69,7 @@ Everything that you store in Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` s3ManagedLedgerOffloadBucket=pulsar-topic-offload @@ -80,7 +80,7 @@ but a recommended configuration. If it is not configured, It will use the defaul With AWS S3, the default region is `US East (N. Virginia)`. Page [AWS Regions and Endpoints](https://docs.aws.amazon.com/general/latest/gr/rande.html) contains more information. -```conf +``` s3ManagedLedgerOffloadRegion=eu-west-3 @@ -99,7 +99,7 @@ Once you have created a set of credentials in the AWS IAM console, they can be c If you are on AWS instance with an instance profile that provides credentials, Pulsar will use these credentials if no other mechanism is provided -2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ```conf/pulsar_env.sh```. +2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ``` ```bash @@ -121,7 +121,9 @@ PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessK 4. Set the access credentials in ```~/.aws/credentials```. -```conf [default] +``` + +[default] aws_access_key_id=ABC123456789 aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c @@ -131,7 +133,7 @@ aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c If you want to assume an IAM role, this can be done via specifying the following: -```conf +``` s3ManagedLedgerOffloadRole= s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload @@ -146,7 +148,8 @@ This will use the `DefaultAWSCredentialsProviderChain` for assuming this role. Pulsar also provides some knobs to configure the size of requests sent to AWS S3. -- ```s3ManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of +- ``` + a "part" sent during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```s3ManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from AWS S3. Default is 1MB. @@ -159,7 +162,7 @@ Buckets are the basic containers that hold your data. Everything that you store Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` gcsManagedLedgerOffloadBucket=pulsar-topic-offload @@ -171,7 +174,7 @@ a recommended configuration. If it is not configured, It will use the default re Regarding GCS, buckets are default created in the `us multi-regional location`, page [Bucket Locations](https://cloud.google.com/storage/docs/bucket-locations) contains more information. -```conf +``` gcsManagedLedgerOffloadRegion=europe-west3 @@ -196,7 +199,7 @@ To generate service account credentials or view the public credentials that you' > Notes: Make ensure that the service account you create has permission to operate GCS, you need to assign **Storage Admin** permission to your service account in [here](https://cloud.google.com/storage/docs/access-control/iam). -```conf +``` gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804d5e6a6f33.json" @@ -206,7 +209,8 @@ gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804 Pulsar also provides some knobs to configure the size of requests sent to GCS. -- ```gcsManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of a "part" sent +- ``` + during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```gcsManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from GCS. Default is 1MB. @@ -220,16 +224,17 @@ In both cases, these should not be touched unless you know what you are doing. You can configure the connection address in the `broker.conf` file. -```conf +``` fileSystemURI="hdfs://127.0.0.1:9000" ``` + #### Configure Hadoop profile path The configuration file is stored in the Hadoop profile path. It contains various settings, such as base path, authentication, and so on. -```conf +``` fileSystemProfilePath="../conf/filesystem_offload_core_site.xml" @@ -239,7 +244,7 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us **Example** -```conf +``` fs.defaultFS @@ -270,7 +275,6 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us io.map.index.interval 128 - ``` @@ -279,7 +283,7 @@ For more information about the configurations in `org.apache.hadoop.io.MapFile`, Namespace policies can be configured to offload data automatically once a threshold is reached. The threshold is based on the size of data that the topic has stored on the pulsar cluster. Once the topic reaches the threshold, an offload operation will be triggered. Setting a negative value to the threshold will disable automatic offloading. Setting the threshold to 0 will cause the broker to offload data as soon as it possiby can. -```bash +``` $ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace @@ -292,13 +296,13 @@ $ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-name By default, once messages were offloaded to long term storage, brokers will read them from long term storage, but messages still exists in bookkeeper for a period depends on the administrator's configuration. For messages exists in both bookkeeper and long term storage, if they are preferred to read from bookkeeper, you can use command to change this configuration. -```bash +``` # default value for -orp is tiered-storage-first $ bin/pulsar-admin namespaces set-offload-policies my-tenant/my-namespace -orp bookkeeper-first $ bin/pulsar-admin topics set-offload-policies my-tenant/my-namespace/topic1 -orp bookkeeper-first -``` +``` ## Triggering offload manually @@ -306,7 +310,7 @@ Offloading can manually triggered through a REST endpoint on the Pulsar broker. When triggering offload, you must specify the maximum size, in bytes, of backlog which will be retained locally on the bookkeeper. The offload mechanism will offload segments from the start of the topic backlog until this condition is met. -```bash +``` $ bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 @@ -315,7 +319,7 @@ Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages be The command to triggers an offload will not wait until the offload operation has completed. To check the status of the offload, use offload-status. -```bash +``` $ bin/pulsar-admin topics offload-status my-tenant/my-namespace/topic1 Offload is currently running @@ -324,7 +328,7 @@ Offload is currently running To wait for offload to complete, add the -w flag. -```bash +``` $ bin/pulsar-admin topics offload-status -w my-tenant/my-namespace/topic1 Offload was a success @@ -333,7 +337,7 @@ Offload was a success If there is an error offloading, the error will be propagated to the offload-status command. -```bash +``` $ bin/pulsar-admin topics offload-status persistent://public/default/topic1 Error in offload @@ -341,5 +345,5 @@ null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website-next/docs/deploy-aws.md b/site2/website-next/docs/deploy-aws.md index ff0e4376f8052..d627a7d5e054e 100644 --- a/site2/website-next/docs/deploy-aws.md +++ b/site2/website-next/docs/deploy-aws.md @@ -1,14 +1,14 @@ --- id: deploy-aws title: Deploying a Pulsar cluster on AWS using Terraform and Ansible -sidebar_label: Amazon Web Services +sidebar_label: "Amazon Web Services" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -> For instructions on deploying a single Pulsar cluster manually rather than using Terraform and Ansible, see [Deploying a Pulsar cluster on bare metal](deploy-bare-metal.md). For instructions on manually deploying a multi-cluster Pulsar instance, see [Deploying a Pulsar instance on bare metal](deploy-bare-metal-multi-cluster.md). +> For instructions on deploying a single Pulsar cluster manually rather than using Terraform and Ansible, see [Deploying a Pulsar cluster on bare metal](deploy-bare-metal.md). For instructions on manually deploying a multi-cluster Pulsar instance, see [Deploying a Pulsar instance on bare metal](deploy-bare-metal-multi-cluster). One of the easiest ways to get a Pulsar [cluster](reference-terminology.md#cluster) running on [Amazon Web Services](https://aws.amazon.com/) (AWS) is to use the [Terraform](https://terraform.io) infrastructure provisioning tool and the [Ansible](https://www.ansible.com) server automation tool. Terraform can create the resources necessary for running the Pulsar cluster---[EC2](https://aws.amazon.com/ec2/) instances, networking and security infrastructure, etc.---While Ansible can install and run Pulsar on the provisioned resources. @@ -23,7 +23,9 @@ In order to install a Pulsar cluster on AWS using Terraform and Ansible, you nee You also need to make sure that you are currently logged into your AWS account via the `aws` tool: ```bash + $ aws configure + ``` ## Installation @@ -31,7 +33,9 @@ $ aws configure You can install Ansible on Linux or macOS using pip. ```bash + $ pip install ansible + ``` You can install Terraform using the instructions [here](https://www.terraform.io/intro/getting-started/install.html). @@ -39,8 +43,10 @@ You can install Terraform using the instructions [here](https://www.terraform.io You also need to have the Terraform and Ansible configuration for Pulsar locally on your machine. You can find them in the [GitHub repository](https://github.com/apache/pulsar) of Pulsar, which you can fetch using Git commands: ```bash + $ git clone https://github.com/apache/pulsar $ cd pulsar/deployment/terraform-ansible/aws + ``` ## SSH setup @@ -53,27 +59,40 @@ $ cd pulsar/deployment/terraform-ansible/aws > > 1. update `ansible.cfg` with following values: > + > ```shell +> > private_key_file=~/.ssh/pulsar_aws +> +> > ``` + > > 2. update `terraform.tfvars` with following values: > + > ```shell +> > public_key_path=~/.ssh/pulsar_aws.pub +> +> > ``` In order to create the necessary AWS resources using Terraform, you need to create an SSH key. Enter the following commands to create a private SSH key in `~/.ssh/id_rsa` and a public key in `~/.ssh/id_rsa.pub`: ```bash + $ ssh-keygen -t rsa + ``` Do *not* enter a passphrase (hit **Enter** instead when the prompt comes out). Enter the following command to verify that a key has been created: ```bash + $ ls ~/.ssh id_rsa id_rsa.pub + ``` ## Create AWS resources using Terraform @@ -81,24 +100,30 @@ id_rsa id_rsa.pub To start building AWS resources with Terraform, you need to install all Terraform dependencies. Enter the following command: ```bash + $ terraform init # This will create a .terraform folder + ``` After that, you can apply the default Terraform configuration by entering this command: ```bash + $ terraform apply + ``` Then you see this prompt below: ```bash + Do you want to perform these actions? Terraform will perform the actions described above. Only 'yes' will be accepted to approve. Enter a value: + ``` Type `yes` and hit **Enter**. Applying the configuration could take several minutes. When the configuration applying finishes, you can see `Apply complete!` along with some other information, including the number of resources created. @@ -142,13 +167,17 @@ All EC2 instances for the cluster run in the [us-west-2](http://docs.aws.amazon. When you apply the Terraform configuration by entering the command `terraform apply`, Terraform outputs a value for the `pulsar_service_url`. The value should look something like this: ``` + pulsar://pulsar-elb-1800761694.us-west-2.elb.amazonaws.com:6650 + ``` You can fetch that value at any time by entering the command `terraform output pulsar_service_url` or parsing the `terraform.tstate` file (which is JSON, even though the filename does not reflect that): ```bash + $ cat terraform.tfstate | jq .modules[0].outputs.pulsar_service_url.value + ``` ### Destroy your cluster @@ -156,7 +185,9 @@ $ cat terraform.tfstate | jq .modules[0].outputs.pulsar_service_url.value At any point, you can destroy all AWS resources associated with your cluster using Terraform's `destroy` command: ```bash + $ terraform destroy + ``` ## Setup Disks @@ -166,10 +197,12 @@ Before you run the Pulsar playbook, you need to mount the disks to the correct d To setup disks on bookie nodes, enter this command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ setup-disk.yaml + ``` After that, the disks is mounted under `/mnt/journal` as journal disk, and `/mnt/storage` as ledger disk. @@ -179,25 +212,29 @@ Remember to enter this command just only once. If you attempt to enter this comm Once you have created the necessary AWS resources using Terraform, you can install and run Pulsar on the Terraform-created EC2 instances using Ansible. -(Optional) If you want to use any [built-in IO connectors](io-connectors.md) , edit the `Download Pulsar IO packages` task in the `deploy-pulsar.yaml` file and uncomment the connectors you want to use. +(Optional) If you want to use any [built-in IO connectors](io-connectors) , edit the `Download Pulsar IO packages` task in the `deploy-pulsar.yaml` file and uncomment the connectors you want to use. To run the playbook, enter this command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ ../deploy-pulsar.yaml + ``` If you have created a private SSH key at a location different from `~/.ssh/id_rsa`, you can specify the different location using the `--private-key` flag in the following command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ --private-key="~/.ssh/some-non-default-key" \ ../deploy-pulsar.yaml + ``` ## Access the cluster @@ -207,24 +244,30 @@ You can now access your running Pulsar using the unique Pulsar connection URL fo For a quick demonstration of accessing the cluster, we can use the Python client for Pulsar and the Python shell. First, install the Pulsar Python module using pip: ```bash + $ pip install pulsar-client + ``` Now, open up the Python shell using the `python` command: ```bash + $ python + ``` Once you are in the shell, enter the following command: ```python + >>> import pulsar >>> client = pulsar.Client('pulsar://pulsar-elb-1800761694.us-west-2.elb.amazonaws.com:6650') # Make sure to use your connection URL >>> producer = client.create_producer('persistent://public/default/test-topic') >>> producer.send('Hello world') >>> client.close() + ``` If all of these commands are successful, Pulsar clients can now use your cluster! diff --git a/site2/website-next/docs/deploy-bare-metal-multi-cluster.md b/site2/website-next/docs/deploy-bare-metal-multi-cluster.md index 8ab7625cfc548..4db16d84af556 100644 --- a/site2/website-next/docs/deploy-bare-metal-multi-cluster.md +++ b/site2/website-next/docs/deploy-bare-metal-multi-cluster.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal-multi-cluster title: Deploying a multi-cluster on bare metal -sidebar_label: Bare metal multi-cluster +sidebar_label: "Bare metal multi-cluster" --- import Tabs from '@theme/Tabs'; @@ -10,13 +10,13 @@ import TabItem from '@theme/TabItem'; > **Tips** > -> 1. You can use single-cluster Pulsar installation in most use cases, such as experimenting with Pulsar or using Pulsar in a startup or in a single team. If you need to run a multi-cluster Pulsar instance, see the [guide](deploy-bare-metal-multi-cluster.md). +> 1. You can use single-cluster Pulsar installation in most use cases, such as experimenting with Pulsar or using Pulsar in a startup or in a single team. If you need to run a multi-cluster Pulsar instance, see the [guide](deploy-bare-metal-multi-cluster). > -> 2. If you want to use all built-in [Pulsar IO](io-overview.md) connectors, you need to download `apache-pulsar-io-connectors`package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you have run a separate cluster of function workers for [Pulsar Functions](functions-overview.md). +> 2. If you want to use all built-in [Pulsar IO](io-overview.md) connectors, you need to download `apache-pulsar-io-connectors`package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you have run a separate cluster of function workers for [Pulsar Functions](functions-overview). > ->3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders`package and install `apache-pulsar-offloaders` under `offloaders` directory in the Pulsar directory on every broker node. For more details of how to configure this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md). +>3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders`package and install `apache-pulsar-offloaders` under `offloaders` directory in the Pulsar directory on every broker node. For more details of how to configure this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage). -A Pulsar instance consists of multiple Pulsar clusters working in unison. You can distribute clusters across data centers or geographical regions and replicate the clusters amongst themselves using [geo-replication](administration-geo.md).Deploying a multi-cluster Pulsar instance consists of the following steps: +A Pulsar instance consists of multiple Pulsar clusters working in unison. You can distribute clusters across data centers or geographical regions and replicate the clusters amongst themselves using [geo-replication](administration-geo).Deploying a multi-cluster Pulsar instance consists of the following steps: 1. Deploying two separate ZooKeeper quorums: a local quorum for each cluster in the instance and a configuration store quorum for instance-wide tasks 2. Initializing cluster metadata for each cluster @@ -25,7 +25,7 @@ A Pulsar instance consists of multiple Pulsar clusters working in unison. You ca > #### Run Pulsar locally or on Kubernetes? -> This guide shows you how to deploy Pulsar in production in a non-Kubernetes environment. If you want to run a standalone Pulsar cluster on a single machine for development purposes, see the [Setting up a local cluster](getting-started-standalone.md) guide. If you want to run Pulsar on [Kubernetes](https://kubernetes.io), see the [Pulsar on Kubernetes](deploy-kubernetes.md) guide, which includes sections on running Pulsar on Kubernetes, on Google Kubernetes Engine and on Amazon Web Services. +> This guide shows you how to deploy Pulsar in production in a non-Kubernetes environment. If you want to run a standalone Pulsar cluster on a single machine for development purposes, see the [Setting up a local cluster](getting-started-standalone.md) guide. If you want to run Pulsar on [Kubernetes](https://kubernetes.io), see the [Pulsar on Kubernetes](deploy-kubernetes) guide, which includes sections on running Pulsar on Kubernetes, on Google Kubernetes Engine and on Amazon Web Services. ## System requirement @@ -33,7 +33,6 @@ Currently, Pulsar is available for 64-bit **macOS**, **Linux**, and **Windows**. :::note - Broker is only supported on 64-bit JVM. ::: @@ -51,23 +50,27 @@ To get started running Pulsar, download a binary tarball release in one of the f * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget 'https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=pulsar/pulsar-@pulsar:version@/apache-pulsar-@pulsar:version@-bin.tar.gz' -O apache-pulsar-@pulsar:version@-bin.tar.gz + ``` Once you download the tarball, untar it and `cd` into the resulting directory: ```bash + $ tar xvfz apache-pulsar-@pulsar:version@-bin.tar.gz $ cd apache-pulsar-@pulsar:version@ + ``` The Pulsar binary package initially contains the following directories: Directory | Contains :---------|:-------- -`bin` | [Command-line tools](reference-cli-tools.md) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) +`bin` | [Command-line tools](reference-cli-tools) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) `conf` | Configuration files for Pulsar, including for [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more -`examples` | A Java JAR file containing example [Pulsar Functions](functions-overview.md) +`examples` | A Java JAR file containing example [Pulsar Functions](functions-overview) `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files that Pulsar uses `licenses` | License files, in `.txt` form, for various components of the Pulsar codebase @@ -76,7 +79,7 @@ The following directories are created once you begin running Pulsar: Directory | Contains :---------|:-------- `data` | The data storage directory that ZooKeeper and BookKeeper use -`instances` | Artifacts created for [Pulsar Functions](functions-overview.md) +`instances` | Artifacts created for [Pulsar Functions](functions-overview) `logs` | Logs that the installation creates @@ -99,16 +102,17 @@ You need to stand up one local ZooKeeper cluster per Pulsar cluster for deployin To begin, add all ZooKeeper servers to the quorum configuration specified in the [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) file. Add a `server.N` line for each node in the cluster to the configuration, where `N` is the number of the ZooKeeper node. The following is an example for a three-node cluster: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` On each host, you need to specify the ID of the node in the `myid` file of each node, which is in `data/zookeeper` folder of each server by default (you can change the file location via the [`dataDir`](reference-configuration.md#zookeeper-dataDir) parameter). :::tip - See the [Multi-server setup guide](https://zookeeper.apache.org/doc/r3.4.10/zookeeperAdmin.html#sc_zkMulitServerSetup) in the ZooKeeper documentation for detailed information on `myid` and more. ::: @@ -116,8 +120,10 @@ See the [Multi-server setup guide](https://zookeeper.apache.org/doc/r3.4.10/zook On a ZooKeeper server at `zk1.us-west.example.com`, for example, you could set the `myid` value like this: ```shell + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com` the command looks like `echo 2 > data/zookeeper/myid` and so on. @@ -125,7 +131,9 @@ On `zk2.us-west.example.com` the command looks like `echo 2 > data/zookeeper/myi Once you add each server to the `zookeeper.conf` configuration and each server has the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start zookeeper + ``` ### Deploy the configuration store @@ -138,13 +146,15 @@ If you deploy a single-cluster instance, you do not need a separate cluster for If your Pulsar instance consists of just one cluster, then you can deploy a configuration store on the same machines as the local ZooKeeper quorum but run on different TCP ports. -To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorom.You need to use the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: +To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorum. You need to use the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 server.3=zk3.us-west.example.com:2185:2186 + ``` As before, create the `myid` files for each server on `data/global-zookeeper/myid`. @@ -160,7 +170,9 @@ Again, given the very low expected load on the configuration store servers, you For example, assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also assume, each cluster has its own local ZK servers named such as the following: ``` + zk[1-3].${CLUSTER}.example.com + ``` In this scenario if you want to pick the quorum participants from few clusters and let all the others be ZK observers. For example, to form a 7 servers quorum, you can pick 3 servers from `us-west`, 2 from `us-central` and 2 from `us-east`. @@ -170,6 +182,7 @@ This method guarantees that writes to configuration store is possible even if on The ZK configuration in all the servers looks like: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 @@ -186,12 +199,15 @@ server.12=zk3.eu-central.example.com:2185:2186:observer server.13=zk1.ap-south.example.com:2185:2186:observer server.14=zk2.ap-south.example.com:2185:2186:observer server.15=zk3.ap-south.example.com:2185:2186:observer + ``` Additionally, ZK observers need to have the following parameters: ```properties + peerType=observer + ``` ##### Start the service @@ -199,7 +215,9 @@ peerType=observer Once your configuration store configuration is in place, you can start up the service using [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) ```shell + $ bin/pulsar-daemon start configuration-store + ``` ## Cluster metadata initialization @@ -209,6 +227,7 @@ Once you set up the cluster-specific ZooKeeper and configuration store quorums f You can initialize this metadata using the [`initialize-cluster-metadata`](reference-cli-tools.md#pulsar-initialize-cluster-metadata) command of the [`pulsar`](reference-cli-tools.md#pulsar) CLI tool. The following is an example: ```shell + $ bin/pulsar initialize-cluster-metadata \ --cluster us-west \ --zookeeper zk1.us-west.example.com:2181 \ @@ -217,6 +236,7 @@ $ bin/pulsar initialize-cluster-metadata \ --web-service-url-tls https://pulsar.us-west.example.com:8443/ \ --broker-service-url pulsar://pulsar.us-west.example.com:6650/ \ --broker-service-url-tls pulsar+ssl://pulsar.us-west.example.com:6651/ + ``` As you can see from the example above, you need to specify the following: @@ -227,7 +247,7 @@ As you can see from the example above, you need to specify the following: * The web service URL for the cluster * A broker service URL enabling interaction with the [brokers](reference-terminology.md#broker) in the cluster -If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. +If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. Make sure to run `initialize-cluster-metadata` for each cluster in your instance. @@ -248,12 +268,17 @@ You can start a bookie in two ways: in the foreground or as a background daemon. To start a bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` You can verify that the bookie works properly using the `bookiesanity` command for the [BookKeeper shell](reference-cli-tools.md#bookkeeper-shell): + ```bash + $ bin/bookkeeper shell bookiesanity + ``` This command creates a new ledger on the local bookie, writes a few entries, reads them back and finally deletes the ledger. @@ -261,7 +286,9 @@ This command creates a new ledger on the local bookie, writes a few entries, rea After you have started all bookies, you can use the `simpletest` command for [BookKeeper shell](reference-cli-tools.md#shell) on any bookie node, to verify that all bookies in the cluster are running. ```bash + $ bin/bookkeeper shell simpletest --ensemble --writeQuorum --ackQuorum --numEntries + ``` Bookie hosts are responsible for storing message data on disk. In order for bookies to provide optimal performance, having a suitable hardware configuration is essential for the bookies. The following are key dimensions for bookie hardware capacity. @@ -292,6 +319,7 @@ You also need to specify the name of the [cluster](reference-terminology.md#clus The following is an example configuration: ```properties + # Local ZooKeeper servers zookeeperServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 @@ -311,6 +339,7 @@ webServicePort=8080 # Port to use to server HTTPS request webServicePortTls=8443 + ``` ### Broker hardware @@ -322,32 +351,38 @@ Pulsar brokers do not require any special hardware since they do not use the loc You can start a broker in the background by using [nohup](https://en.wikipedia.org/wiki/Nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start broker + ``` You can also start brokers in the foreground by using [`pulsar broker`](reference-cli-tools.md#broker): ```shell + $ bin/pulsar broker + ``` ## Service discovery -[Clients](getting-started-clients.md) connecting to Pulsar brokers need to communicate with an entire Pulsar instance using a single URL. Pulsar provides a built-in service discovery mechanism that you can set up using the instructions immediately below. +[Clients](getting-started-clients) connecting to Pulsar brokers need to communicate with an entire Pulsar instance using a single URL. -You can also use your own service discovery system . If you use your own system, you only need to satisfy just one requirement: when a client performs an HTTP request to an [endpoint](reference-configuration.md) for a Pulsar cluster, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to some active brokers in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. +You can use your own service discovery system. If you use your own system, you only need to satisfy just one requirement: when a client performs an HTTP request to an [endpoint](reference-configuration) for a Pulsar cluster, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to some active brokers in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. > **Service discovery already provided by many scheduling systems** > Many large-scale deployment systems, such as [Kubernetes](deploy-kubernetes), have service discovery systems built in. If you run Pulsar on such a system, you may not need to provide your own service discovery mechanism. ## Admin client and verification -At this point your Pulsar instance should be ready to use. You can now configure client machines that can serve as [administrative clients](admin-api-overview.md) for each cluster. You can use the [`conf/client.conf`](reference-configuration.md#client) configuration file to configure admin clients. +At this point your Pulsar instance should be ready to use. You can now configure client machines that can serve as [administrative clients](admin-api-overview) for each cluster. You can use the [`conf/client.conf`](reference-configuration.md#client) configuration file to configure admin clients. The most important thing is that you point the [`serviceUrl`](reference-configuration.md#client-serviceUrl) parameter to the correct service URL for the cluster: ```properties + serviceUrl=http://pulsar.us-west.example.com:8080/ + ``` ## Provision new tenants @@ -358,9 +393,11 @@ Pulsar is built as a fundamentally multi-tenant system. If a new tenant wants to use the system, you need to create a new one. You can create a new tenant by using the [`pulsar-admin`](reference-pulsar-admin.md#tenants) CLI tool: ```shell + $ bin/pulsar-admin tenants create test-tenant \ --allowed-clusters us-west \ --admin-roles test-admin-role + ``` In this command, users who identify with `test-admin-role` role can administer the configuration for the `test-tenant` tenant. The `test-tenant` tenant can only use the `us-west` cluster. From now on, this tenant can manage its resources. @@ -371,7 +408,9 @@ Once you create a tenant, you need to create [namespaces](reference-terminology. The first step is to create a namespace. A namespace is an administrative unit that can contain many topics. A common practice is to create a namespace for each different use case from a single tenant. ```shell + $ bin/pulsar-admin namespaces create test-tenant/ns1 + ``` ##### Test producer and consumer @@ -385,23 +424,32 @@ You can use a topic in the namespace that you have just created. Topics are auto The topic name in this case could be: ```http + persistent://test-tenant/ns1/my-topic + ``` Start a consumer that creates a subscription on the topic and waits for messages: ```shell + $ bin/pulsar-perf consume persistent://test-tenant/ns1/my-topic + ``` Start a producer that publishes messages at a fixed rate and reports stats every 10 seconds: ```shell + $ bin/pulsar-perf produce persistent://test-tenant/ns1/my-topic + ``` To report the topic stats: ```shell + $ bin/pulsar-admin topics stats persistent://test-tenant/ns1/my-topic + ``` + diff --git a/site2/website-next/docs/deploy-bare-metal.md b/site2/website-next/docs/deploy-bare-metal.md index 0e0bad83c7be9..1efaab6a84e06 100644 --- a/site2/website-next/docs/deploy-bare-metal.md +++ b/site2/website-next/docs/deploy-bare-metal.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal title: Deploy a cluster on bare metal -sidebar_label: Bare metal +sidebar_label: "Bare metal" --- import Tabs from '@theme/Tabs'; @@ -11,11 +11,11 @@ import TabItem from '@theme/TabItem'; > **Tips** > -> 1. You can use single-cluster Pulsar installation in most use cases, such as experimenting with Pulsar or using Pulsar in a startup or in a single team. If you need to run a multi-cluster Pulsar instance, see the [guide](deploy-bare-metal-multi-cluster.md). +> 1. You can use single-cluster Pulsar installation in most use cases, such as experimenting with Pulsar or using Pulsar in a startup or in a single team. If you need to run a multi-cluster Pulsar instance, see the [guide](deploy-bare-metal-multi-cluster). > -> 2. If you want to use all built-in [Pulsar IO](io-overview.md) connectors, you need to download `apache-pulsar-io-connectors`package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you have run a separate cluster of function workers for [Pulsar Functions](functions-overview.md). +> 2. If you want to use all built-in [Pulsar IO](io-overview.md) connectors, you need to download `apache-pulsar-io-connectors`package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you have run a separate cluster of function workers for [Pulsar Functions](functions-overview). > -> 3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders`package and install `apache-pulsar-offloaders` under `offloaders` directory in the Pulsar directory on every broker node. For more details of how to configure this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md). +> 3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders`package and install `apache-pulsar-offloaders` under `offloaders` directory in the Pulsar directory on every broker node. For more details of how to configure this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage). Deploying a Pulsar cluster consists of the following steps: @@ -47,6 +47,7 @@ To run Pulsar on bare metal, the following configuration is recommended: * If you do not have a DNS server, you can use the multi-host format in the service URL instead. ::: + Each machine in your cluster needs to have [Java 8](https://adoptopenjdk.net/?variant=openjdk8) or [Java 11](https://adoptopenjdk.net/?variant=openjdk11) installed. The following is a diagram showing the basic setup: @@ -74,26 +75,26 @@ To start a Pulsar instance, below are the minimum and the recommended hardware s 1. The minimum hardware settings (250 Pulsar topics) - Broker - - CPU: 0.2 - - Memory: 256MB + - CPU: 0.2 + - Memory: 256MB - Bookie - - CPU: 0.2 - - Memory: 256MB - - Storage: - - Journal: 8GB, PD-SSD - - Ledger: 16GB, PD-STANDARD + - CPU: 0.2 + - Memory: 256MB + - Storage: + - Journal: 8GB, PD-SSD + - Ledger: 16GB, PD-STANDARD 2. The recommended hardware settings (1000 Pulsar topics) - Broker - - CPU: 8 - - Memory: 8GB + - CPU: 8 + - Memory: 8GB - Bookie - - CPU: 4 - - Memory: 8GB - - Storage: - - Journal: 256GB, PD-SSD - - Ledger: 2TB, PD-STANDARD + - CPU: 4 + - Memory: 8GB + - Storage: + - Journal: 256GB, PD-SSD + - Ledger: 2TB, PD-STANDARD ## Install the Pulsar binary package @@ -108,21 +109,25 @@ To get started deploying a Pulsar cluster on bare metal, you need to download a * Using [wget](https://www.gnu.org/software/wget): ```bash + $ wget pulsar:binary_release_url + ``` Once you download the tarball, untar it and `cd` into the resulting directory: ```bash + $ tar xvzf apache-pulsar-@pulsar:version@-bin.tar.gz $ cd apache-pulsar-@pulsar:version@ + ``` The extracted directory contains the following subdirectories: Directory | Contains :---------|:-------- -`bin` |[command-line tools](reference-cli-tools.md) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) +`bin` |[command-line tools](reference-cli-tools) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) `conf` | Configuration files for Pulsar, including for [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more `data` | The data storage directory that ZooKeeper and BookKeeper use `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files that Pulsar uses @@ -144,19 +149,23 @@ To use `builtin` connectors, you need to download the connectors tarball release * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar + ``` Once you download the .nar file, copy the file to directory `connectors` in the pulsar directory. For example, if you download the connector file `pulsar-io-aerospike-@pulsar:version@.nar`: ```bash + $ mkdir connectors $ mv pulsar-io-aerospike-@pulsar:version@.nar connectors $ ls connectors pulsar-io-aerospike-@pulsar:version@.nar ... + ``` ## [Install Tiered Storage Offloaders (optional)](https://pulsar.apache.org/docs/en/next/standalone/#install-tiered-storage-offloaders-optional) @@ -176,12 +185,15 @@ To use tiered storage offloaders, you need to download the offloaders tarball re * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:offloader_release_url + ``` Once you download the tarball, in the Pulsar directory, untar the offloaders package and copy the offloaders as `offloaders` in the Pulsar directory: ```bash + $ tar xvfz apache-pulsar-offloaders-@pulsar:version@-bin.tar.gz // you can find a directory named `apache-pulsar-offloaders-@pulsar:version@` in the pulsar directory @@ -191,9 +203,10 @@ $ mv apache-pulsar-offloaders-@pulsar:version@/offloaders offloaders $ ls offloaders tiered-storage-jcloud-@pulsar:version@.nar + ``` -For more details of how to configure tiered storage feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md) +For more details of how to configure tiered storage feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage) ## Deploy a ZooKeeper cluster @@ -205,9 +218,11 @@ For more details of how to configure tiered storage feature, you can refer to th To begin, add all ZooKeeper servers to the configuration specified in [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) (in the Pulsar directory that you create [above](#install-the-pulsar-binary-package)). The following is an example: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` > If you only have one machine on which to deploy Pulsar, you only need to add one server entry in the configuration file. @@ -219,8 +234,10 @@ On each host, you need to specify the ID of the node in the `myid` file, which i For example, on a ZooKeeper server like `zk1.us-west.example.com`, you can set the `myid` value as follows: ```bash + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com`, the command is `echo 2 > data/zookeeper/myid` and so on. @@ -228,7 +245,9 @@ On `zk2.us-west.example.com`, the command is `echo 2 > data/zookeeper/myid` and Once you add each server to the `zookeeper.conf` configuration and have the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start zookeeper + ``` > If you plan to deploy Zookeeper with the Bookie on the same node, you need to start zookeeper by using different stats @@ -241,6 +260,7 @@ Once you deploy ZooKeeper for your cluster, you need to write some metadata to Z You can initialize this metadata using the [`initialize-cluster-metadata`](reference-cli-tools.md#pulsar-initialize-cluster-metadata) command of the [`pulsar`](reference-cli-tools.md#pulsar) CLI tool. This command can be run on any machine in your ZooKeeper cluster. The following is an example: ```shell + $ bin/pulsar initialize-cluster-metadata \ --cluster pulsar-cluster-1 \ --zookeeper zk1.us-west.example.com:2181 \ @@ -249,6 +269,7 @@ $ bin/pulsar initialize-cluster-metadata \ --web-service-url-tls https://pulsar.us-west.example.com:8443 \ --broker-service-url pulsar://pulsar.us-west.example.com:6650 \ --broker-service-url-tls pulsar+ssl://pulsar.us-west.example.com:6651 + ``` As you can see from the example above, you will need to specify the following: @@ -259,29 +280,39 @@ Flag | Description `--zookeeper` | A "local" ZooKeeper connection string for the cluster. This connection string only needs to include *one* machine in the ZooKeeper cluster. `--configuration-store` | The configuration store connection string for the entire instance. As with the `--zookeeper` flag, this connection string only needs to include *one* machine in the ZooKeeper cluster. `--web-service-url` | The web service URL for the cluster, plus a port. This URL should be a standard DNS name. The default port is 8080 (you had better not use a different port). -`--web-service-url-tls` | If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster. The default port is 8443 (you had better not use a different port). +`--web-service-url-tls` | If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster. The default port is 8443 (you had better not use a different port). `--broker-service-url` | A broker service URL enabling interaction with the brokers in the cluster. This URL should not use the same DNS name as the web service URL but should use the `pulsar` scheme instead. The default port is 6650 (you had better not use a different port). -`--broker-service-url-tls` | If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. The default port is 6651 (you had better not use a different port). +`--broker-service-url-tls` | If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. The default port is 6651 (you had better not use a different port). > If you do not have a DNS server, you can use multi-host format in the service URL with the following settings: > + > ```shell +> > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` +> > If you want to use an existing BookKeeper cluster, you can add the `--existing-bk-metadata-service-uri` flag as follows: > + > ```shell +> > --existing-bk-metadata-service-uri "zk+null://zk1:2181;zk2:2181/ledgers" \ > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` + > You can obtain the metadata service URI of the existing BookKeeper cluster by using the `bin/bookkeeper shell whatisinstanceid` command. You must enclose the value in double quotes since the multiple metadata service URIs are separated with semicolons. ## Deploy a BookKeeper cluster @@ -291,7 +322,9 @@ Flag | Description You can configure BookKeeper bookies using the [`conf/bookkeeper.conf`](reference-configuration.md#bookkeeper) configuration file. The most important step in configuring bookies for our purposes here is ensuring that [`zkServers`](reference-configuration.md#bookkeeper-zkServers) is set to the connection string for the ZooKeeper cluster. The following is an example: ```properties + zkServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 + ``` Once you appropriately modify the `zkServers` parameter, you can make any other configuration changes that you require. You can find a full listing of the available BookKeeper configuration parameters [here](reference-configuration.md#bookkeeper). However, consulting the [BookKeeper documentation](http://bookkeeper.apache.org/docs/latest/reference/config/) for a more in-depth guide might be a better choice. @@ -301,19 +334,25 @@ Once you apply the desired configuration in `conf/bookkeeper.conf`, you can star To start the bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` To start the bookie in the foreground: ```bash + $ bin/pulsar bookie + ``` You can verify that a bookie works properly by running the `bookiesanity` command on the [BookKeeper shell](reference-cli-tools.md#shell): ```bash + $ bin/bookkeeper shell bookiesanity + ``` This command creates an ephemeral BookKeeper ledger on the local bookie, writes a few entries, reads them back, and finally deletes the ledger. @@ -321,7 +360,9 @@ This command creates an ephemeral BookKeeper ledger on the local bookie, writes After you start all the bookies, you can use `simpletest` command for [BookKeeper shell](reference-cli-tools.md#shell) on any bookie node, to verify all the bookies in the cluster are up running. ```bash + $ bin/bookkeeper shell simpletest --ensemble --writeQuorum --ackQuorum --numEntries + ``` This command creates a `num-bookies` sized ledger on the cluster, writes a few entries, and finally deletes the ledger. @@ -336,28 +377,36 @@ Pulsar brokers are the last thing you need to deploy in your Pulsar cluster. Bro The most important element of broker configuration is ensuring that each broker is aware of the ZooKeeper cluster that you have deployed. Ensure that the [`zookeeperServers`](reference-configuration.md#broker-zookeeperServers) and [`configurationStoreServers`](reference-configuration.md#broker-configurationStoreServers) parameters are correct. In this case, since you only have 1 cluster and no configuration store setup, the `configurationStoreServers` point to the same `zookeeperServers`. ```properties + zookeeperServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 configurationStoreServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 + ``` You also need to specify the cluster name (matching the name that you provided when you [initialize the metadata of the cluster](#initialize-cluster-metadata)): ```properties + clusterName=pulsar-cluster-1 + ``` In addition, you need to match the broker and web service ports provided when you initialize the metadata of the cluster (especially when you use a different port than the default): ```properties + brokerServicePort=6650 brokerServicePortTls=6651 webServicePort=8080 webServicePortTls=8443 + ``` > If you deploy Pulsar in a one-node cluster, you should update the replication settings in `conf/broker.conf` to `1`. > + > ```properties +> > # Number of bookies to use when creating a ledger > managedLedgerDefaultEnsembleSize=1 > @@ -366,25 +415,31 @@ webServicePortTls=8443 > > # Number of guaranteed copies (acks to wait before write is complete) > managedLedgerDefaultAckQuorum=1 +> +> > ``` ### Enable Pulsar Functions (optional) -If you want to enable [Pulsar Functions](functions-overview.md), you can follow the instructions as below: +If you want to enable [Pulsar Functions](functions-overview), you can follow the instructions as below: 1. Edit `conf/broker.conf` to enable functions worker, by setting `functionsWorkerEnabled` to `true`. - ```conf - functionsWorkerEnabled=true - ``` + ```conf + + functionsWorkerEnabled=true + + ``` 2. Edit `conf/functions_worker.yml` and set `pulsarFunctionsCluster` to the cluster name that you provide when you [initialize the metadata of the cluster](#initialize-cluster-metadata). - ```conf - pulsarFunctionsCluster: pulsar-cluster-1 - ``` + ```conf + + pulsarFunctionsCluster: pulsar-cluster-1 + + ``` -If you want to learn more options about deploying the functions worker, check out [Deploy and manage functions worker](functions-worker.md). +If you want to learn more options about deploying the functions worker, check out [Deploy and manage functions worker](functions-worker). ### Start Brokers @@ -393,13 +448,17 @@ You can then provide any other configuration changes that you want in the [`conf You can start a broker in the foreground using the [`pulsar broker`](reference-cli-tools.md#pulsar-broker) command: ```bash + $ bin/pulsar broker + ``` You can start a broker in the background using the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start broker + ``` Once you successfully start up all the brokers that you intend to use, your Pulsar cluster should be ready to go! @@ -411,24 +470,32 @@ Once your Pulsar cluster is up and running, you should be able to connect with i To use the `pulsar-client` tool, first modify the client configuration file in [`conf/client.conf`](reference-configuration.md#client) in your binary package. You need to change the values for `webServiceUrl` and `brokerServiceUrl`, substituting `localhost` (which is the default), with the DNS name that you assign to your broker/bookie hosts. The following is an example: ```properties + webServiceUrl=http://us-west.example.com:8080 brokerServiceurl=pulsar://us-west.example.com:6650 + ``` > If you do not have a DNS server, you can specify multi-host in service URL as follows: > + > ```properties +> > webServiceUrl=http://host1:8080,host2:8080,host3:8080 > brokerServiceurl=pulsar://host1:6650,host2:6650,host3:6650 +> +> > ``` Once that is complete, you can publish a message to the Pulsar topic: ```bash + $ bin/pulsar-client produce \ persistent://public/default/test \ -n 1 \ -m "Hello Pulsar" + ``` > You may need to use a different cluster name in the topic if you specify a cluster name other than `pulsar-cluster-1`. @@ -436,18 +503,22 @@ $ bin/pulsar-client produce \ This command publishes a single message to the Pulsar topic. In addition, you can subscribe to the Pulsar topic in a different terminal before publishing messages as below: ```bash + $ bin/pulsar-client consume \ persistent://public/default/test \ -n 100 \ -s "consumer-test" \ -t "Exclusive" + ``` Once you successfully publish the above message to the topic, you should see it in the standard output: ```bash + ----- got message ----- Hello Pulsar + ``` ## Run Functions @@ -457,6 +528,7 @@ Hello Pulsar Create an ExclamationFunction `exclamation`. ```bash + bin/pulsar-admin functions create \ --jar examples/api-examples.jar \ --classname org.apache.pulsar.functions.api.examples.ExclamationFunction \ @@ -465,16 +537,22 @@ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ --name exclamation + ``` Check whether the function runs as expected by [triggering](functions-deploying.md#triggering-pulsar-functions) the function. ```bash + bin/pulsar-admin functions trigger --name exclamation --trigger-value "hello world" + ``` You should see the following output: ```shell + hello world! + ``` + diff --git a/site2/website-next/docs/deploy-dcos.md b/site2/website-next/docs/deploy-dcos.md index ac067b0f7a838..f7dabdf5d470c 100644 --- a/site2/website-next/docs/deploy-dcos.md +++ b/site2/website-next/docs/deploy-dcos.md @@ -1,7 +1,7 @@ --- id: deploy-dcos title: Deploy Pulsar on DC/OS -sidebar_label: DC/OS +sidebar_label: "DC/OS" --- import Tabs from '@theme/Tabs'; @@ -9,7 +9,7 @@ import TabItem from '@theme/TabItem'; > **Tips** -> To enable all built-in [Pulsar IO](io-overview.md) connectors in your Pulsar deploymente, we recommend you use `apachepulsar/pulsar-all` image instead of `apachepulsar/pulsar` image; the former has already bundled [all built-in connectors](io-overview.md#working-with-connectors). +> To enable all built-in [Pulsar IO](io-overview) connectors in your Pulsar deploymente, we recommend you use `apachepulsar/pulsar-all` image instead of `apachepulsar/pulsar` image; the former has already bundled [all built-in connectors](io-overview.md#working-with-connectors). [DC/OS](https://dcos.io/) (the DataCenter Operating System) is a distributed operating system for deploying and managing applications and systems on [Apache Mesos](http://mesos.apache.org/). DC/OS is an open-source tool created and maintained by [Mesosphere](https://mesosphere.com/). @@ -25,7 +25,9 @@ You need to prepare your environment before running Pulsar on DC/OS. * The [`PulsarGroups.json`](https://github.com/apache/pulsar/blob/master/deployment/dcos/PulsarGroups.json) configuration file from the Pulsar GitHub repo. ```bash + $ curl -O https://raw.githubusercontent.com/apache/pulsar/master/deployment/dcos/PulsarGroups.json + ``` Each node in the DC/OS-managed Mesos cluster must have at least: @@ -41,7 +43,9 @@ Alternatively, you can change the configuration in `PulsarGroups.json` according You can deploy Pulsar on DC/OS using this command: ```bash + $ dcos marathon group add PulsarGroups.json + ``` This command deploys Docker container instances in three groups, which together comprise a Pulsar cluster: @@ -128,7 +132,9 @@ Now that you have a fully deployed Pulsar cluster, you can run a simple consumer You can clone a [Pulsar Java tutorial](https://github.com/streamlio/pulsar-java-tutorial) repo. This repo contains a simple Pulsar consumer and producer (you can find more information in the `README` file in this repo). ```bash + $ git clone https://github.com/streamlio/pulsar-java-tutorial + ``` Change the `SERVICE_URL` from `pulsar://localhost:6650` to `pulsar://a1.dcos:6650` in both [`ConsumerTutorial.java`](https://github.com/streamlio/pulsar-java-tutorial/blob/master/src/main/java/tutorial/ConsumerTutorial.java) file and [`ProducerTutorial.java`](https://github.com/streamlio/pulsar-java-tutorial/blob/master/src/main/java/tutorial/ProducerTutorial.java) file. @@ -140,7 +146,9 @@ Now, you can change the message number from 10 to 10000000 in the main method in Then, you can compile the project code using the command below: ```bash + $ mvn clean package + ``` ### Run the consumer and producer @@ -148,13 +156,17 @@ $ mvn clean package Execute this command to run the consumer: ```bash + $ mvn exec:java -Dexec.mainClass="tutorial.ConsumerTutorial" + ``` Execute this command to run the producer: ```bash + $ mvn exec:java -Dexec.mainClass="tutorial.ProducerTutorial" + ``` You see that the producer is producing messages and the consumer is consuming messages through the DC/OS GUI. @@ -176,10 +188,13 @@ You can shut down and uninstall the `pulsar` application from DC/OS at any time 1. Click the three dots at the right end of Pulsar group and choose **Delete** on the DC/OS GUI. - ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) + ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) 2. Use the command below. - ```bash - $ dcos marathon group remove /pulsar - ``` + ```bash + + $ dcos marathon group remove /pulsar + + ``` + diff --git a/site2/website-next/docs/deploy-docker.md b/site2/website-next/docs/deploy-docker.md index 2cb5288db5260..b7a6bfe472f60 100644 --- a/site2/website-next/docs/deploy-docker.md +++ b/site2/website-next/docs/deploy-docker.md @@ -1,7 +1,7 @@ --- id: deploy-docker title: Deploy a cluster on Docker -sidebar_label: Docker +sidebar_label: "Docker" --- import Tabs from '@theme/Tabs'; @@ -22,34 +22,42 @@ To run Pulsar on Docker, you need to create a container for each Pulsar componen You can pull a Pulsar image from [Docker Hub](https://hub.docker.com/r/apachepulsar/pulsar-all/tags) with the following command. ``` + docker pull apachepulsar/pulsar-all:latest + ``` ### Create three containers Create containers for ZooKeeper, BookKeeper and broker. In this example, they are named as `zookeeper`, `bookkeeper` and `broker` respectively. You can name them as you want with the `--name` flag. By default, the container names are created randomly. ``` + docker run -it --name bookkeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name zookeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name broker apachepulsar/pulsar-all:latest /bin/bash + ``` ### Create a network To deploy a Pulsar cluster on Docker, you need to create a `network` and connect the containers of ZooKeeper, BookKeeper and broker to this network. The following command creates the network `pulsar`: ``` + docker network create pulsar + ``` ### Connect containers to network Connect the containers of ZooKeeper, BookKeeper and broker to the `pulsar` network with the following commands. ``` + docker network connect pulsar zookeeper docker network connect pulsar bookkeeper docker network connect pulsar broker + ``` To check whether the containers are successfully connected to the network, enter the `docker network inspect pulsar` command. -For detailed information about how to deploy ZooKeeper cluster, BookKeeper cluster, brokers, see [deploy a cluster on bare metal](deploy-bare-metal.md). +For detailed information about how to deploy ZooKeeper cluster, BookKeeper cluster, brokers, see [deploy a cluster on bare metal](deploy-bare-metal). diff --git a/site2/website-next/docs/deploy-kubernetes.md b/site2/website-next/docs/deploy-kubernetes.md index 3cc27eb2575ed..e2e55ac97ff43 100644 --- a/site2/website-next/docs/deploy-kubernetes.md +++ b/site2/website-next/docs/deploy-kubernetes.md @@ -1,7 +1,7 @@ --- id: deploy-kubernetes title: Deploy Pulsar on Kubernetes -sidebar_label: Kubernetes +sidebar_label: "Kubernetes" --- import Tabs from '@theme/Tabs'; @@ -9,6 +9,6 @@ import TabItem from '@theme/TabItem'; To get up and running with these charts as fast as possible, in a **non-production** use case, we provide -a [quick start guide](getting-started-helm.md) for Proof of Concept (PoC) deployments. +a [quick start guide](getting-started-helm) for Proof of Concept (PoC) deployments. -To configure and install a Pulsar cluster on Kubernetes for production usage, follow the complete [Installation Guide](helm-install.md). \ No newline at end of file +To configure and install a Pulsar cluster on Kubernetes for production usage, follow the complete [Installation Guide](helm-install). \ No newline at end of file diff --git a/site2/website-next/docs/deploy-monitoring.md b/site2/website-next/docs/deploy-monitoring.md index 82fb03b35b0de..221e5cbe324fd 100644 --- a/site2/website-next/docs/deploy-monitoring.md +++ b/site2/website-next/docs/deploy-monitoring.md @@ -1,7 +1,7 @@ --- id: deploy-monitoring title: Monitor -sidebar_label: Monitor +sidebar_label: "Monitor" --- import Tabs from '@theme/Tabs'; @@ -21,13 +21,17 @@ You can collect Pulsar broker metrics from brokers and export the metrics in JSO * *Destination dumps*, which contain stats for each individual topic. You can fetch the destination dumps using the command below: ```shell + bin/pulsar-admin broker-stats destinations + ``` * Broker metrics, which contain the broker information and topics stats aggregated at namespace level. You can fetch the broker metrics by using the following command: ```shell + bin/pulsar-admin broker-stats monitoring-metrics + ``` All the message rates are updated every minute. @@ -35,7 +39,9 @@ All the message rates are updated every minute. The aggregated broker metrics are also exposed in the [Prometheus](https://prometheus.io) format at: ```shell + http://$BROKER_ADDRESS:8080/metrics + ``` ### ZooKeeper stats @@ -43,8 +49,10 @@ http://$BROKER_ADDRESS:8080/metrics The local ZooKeeper, configuration store server and clients that are shipped with Pulsar can expose detailed stats through Prometheus. ```shell + http://$LOCAL_ZK_SERVER:8000/metrics http://$GLOBAL_ZK_SERVER:8001/metrics + ``` The default port of local ZooKeeper is `8000` and the default port of configuration store is `8001`. You can change the default port of local ZooKeeper and configuration store by specifying system property `stats_server_port`. @@ -56,7 +64,9 @@ You can configure the stats frameworks for BookKeeper by modifying the `statsPro The default BookKeeper configuration enables the Prometheus exporter. The configuration is included with Pulsar distribution. ```shell + http://$BOOKIE_ADDRESS:8000/metrics + ``` The default port for bookie is `8000`. You can change the port by configuring `prometheusStatsHttpPort` in the `conf/bookkeeper.conf` file. @@ -65,11 +75,13 @@ The default port for bookie is `8000`. You can change the port by configuring `p The acknowledgment state is persistent to the ledger first. When the acknowledgment state fails to be persistent to the ledger, they are persistent to ZooKeeper. To track the stats of acknowledgement, you can configure the metrics for the managed cursor. ``` -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"") + +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:") brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cursor_name:"") + ``` Those metrics are added in the Prometheus interface, you can monitor and check the metrics stats in the Grafana. @@ -79,26 +91,32 @@ Those metrics are added in the Prometheus interface, you can monitor and check t You can collect functions worker stats from `functions-worker` and export the metrics in JSON formats, which contain functions worker JVM metrics. ``` + pulsar-admin functions-worker monitoring-metrics + ``` You can collect functions and connectors metrics from `functions-worker` and export the metrics in JSON formats. ``` + pulsar-admin functions-worker function-stats + ``` The aggregated functions and connectors metrics can be exposed in Prometheus formats as below. You can get [`FUNCTIONS_WORKER_ADDRESS`](http://pulsar.apache.org/docs/en/next/functions-worker/) and `WORKER_PORT` from the `functions_worker.yml` file. ``` + http://$FUNCTIONS_WORKER_ADDRESS:$WORKER_PORT/metrics: + ``` ## Configure Prometheus You can use Prometheus to collect all the metrics exposed for Pulsar components and set up [Grafana](https://grafana.com/) dashboards to display the metrics and monitor your Pulsar cluster. For details, refer to [Prometheus guide](https://prometheus.io/docs/introduction/getting_started/). -When you run Pulsar on bare metal, you can provide the list of nodes to be probed. When you deploy Pulsar in a Kubernetes cluster, the monitoring is setup automatically. For details, refer to [Kubernetes instructions](helm-deploy.md). +When you run Pulsar on bare metal, you can provide the list of nodes to be probed. When you deploy Pulsar in a Kubernetes cluster, the monitoring is setup automatically. For details, refer to [Kubernetes instructions](helm-deploy). ## Dashboards @@ -106,7 +124,7 @@ When you collect time series statistics, the major problem is to make sure the n ### Pulsar per-topic dashboard -The per-topic dashboard instructions are available at [Pulsar manager](administration-pulsar-manager.md). +The per-topic dashboard instructions are available at [Pulsar manager](administration-pulsar-manager). ### Grafana @@ -117,9 +135,11 @@ When you deploy Pulsar on Kubernetes, a `pulsar-grafana` Docker image is enabled Enter the command below to use the dashboard manually: ```shell + docker run -p3000:3000 \ -e PROMETHEUS_URL=http://$PROMETHEUS_HOST:9090/ \ apachepulsar/pulsar-grafana:latest + ``` The following are some Grafana dashboards examples: diff --git a/site2/website-next/docs/developing-binary-protocol.md b/site2/website-next/docs/developing-binary-protocol.md index 822a1aba59325..bda6a1a62c70d 100644 --- a/site2/website-next/docs/developing-binary-protocol.md +++ b/site2/website-next/docs/developing-binary-protocol.md @@ -1,7 +1,7 @@ --- id: develop-binary-protocol title: Pulsar binary protocol specification -sidebar_label: Binary protocol +sidebar_label: "Binary protocol" --- import Tabs from '@theme/Tabs'; @@ -32,42 +32,55 @@ The Pulsar protocol allows for two types of commands: Simple (payload-free) commands have this basic structure: -| Component | Description | Size (in bytes) | -|:------------|:----------------------------------------------------------------------------------------|:----------------| -| totalSize | The size of the frame, counting everything that comes after it (in bytes) | 4 | -| commandSize | The size of the protobuf-serialized command | 4 | -| message | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | +| Component | Description | Size (in bytes) | +|:--------------|:----------------------------------------------------------------------------------------|:----------------| +| `totalSize` | The size of the frame, counting everything that comes after it (in bytes) | 4 | +| `commandSize` | The size of the protobuf-serialized command | 4 | +| `message` | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | ### Payload commands Payload commands have this basic structure: -| Component | Description | Size (in bytes) | -|:-------------|:--------------------------------------------------------------------------------------------|:----------------| -| totalSize | The size of the frame, counting everything that comes after it (in bytes) | 4 | -| commandSize | The size of the protobuf-serialized command | 4 | -| message | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | -| magicNumber | A 2-byte byte array (`0x0e01`) identifying the current format | 2 | -| checksum | A [CRC32-C checksum](http://www.evanjones.ca/crc32c.html) of everything that comes after it | 4 | -| metadataSize | The size of the message [metadata](#message-metadata) | 4 | -| metadata | The message [metadata](#message-metadata) stored as a binary protobuf message | | -| payload | Anything left in the frame is considered the payload and can include any sequence of bytes | | +| Component | Required or optional| Description | Size (in bytes) | +|:-----------------------------------|:----------|:--------------------------------------------------------------------------------------------|:----------------| +| `totalSize` | Required | The size of the frame, counting everything that comes after it (in bytes) | 4 | +| `commandSize` | Required | The size of the protobuf-serialized command | 4 | +| `message` | Required | The protobuf message serialized in a raw binary format (rather than in protobuf format) | | +| `magicNumberOfBrokerEntryMetadata` | Optional | A 2-byte byte array (`0x0e02`) identifying the broker entry metadata
**Note**: `magicNumberOfBrokerEntryMetadata` , `brokerEntryMetadataSize`, and `brokerEntryMetadata` should be used **together**. | 2 | +| `brokerEntryMetadataSize` | Optional | The size of the broker entry metadata | 4 | +| `brokerEntryMetadata` | Optional | The broker entry metadata stored as a binary protobuf message | | +| `magicNumber` | Required | A 2-byte byte array (`0x0e01`) identifying the current format | 2 | +| `checksum` | Required | A [CRC32-C checksum](http://www.evanjones.ca/crc32c.html) of everything that comes after it | 4 | +| `metadataSize` | Required | The size of the message [metadata](#message-metadata) | 4 | +| `metadata` | Required | The message [metadata](#message-metadata) stored as a binary protobuf message | | +| `payload` | Required | Anything left in the frame is considered the payload and can include any sequence of bytes | | + +## Broker entry metadata + +Broker entry metadata is stored alongside the message metadata as a serialized protobuf message. +It is created by the broker when the message arrived at the broker and passed without changes to the consumer if configured. + +| Field | Required or optional | Description | +|:-------------------|:----------------|:------------------------------------------------------------------------------------------------------------------------------| +| `broker_timestamp` | Optional | The timestamp when a message arrived at the broker (`id est` as the number of milliseconds since January 1st, 1970 in UTC) | +| `index` | Optional | The index of the message. It is assigned by the broker. ## Message metadata -Message metadata is stored alongside the application-specified payload as a serialized protobuf message. Metadata is created by the producer and passed on unchanged to the consumer. +Message metadata is stored alongside the application-specified payload as a serialized protobuf message. Metadata is created by the producer and passed without changes to the consumer. -| Field | Description | -|:-------------------------------------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `producer_name` | The name of the producer that published the message | -| `sequence_id` | The sequence ID of the message, assigned by producer | -| `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 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 | +| Field | Required or optional | Description | +|:-------------------------|:----------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `producer_name` | Required | The name of the producer that published the message | +| `sequence_id` | Required | The sequence ID of the message, assigned by producer | +| `publish_time` | Required | The publish timestamp in Unix time (i.e. as the number of milliseconds since January 1st, 1970 in UTC) | +| `properties` | Required | 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 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 | ### Batch messages @@ -79,19 +92,19 @@ object. For a single batch, the payload format will look like this: -| Field | Description | -|:--------------|:------------------------------------------------------------| -| metadataSizeN | The size of the single message metadata serialized Protobuf | -| metadataN | Single message metadata | -| payloadN | Message payload passed by application | +| Field | Required or optional | Description | +|:----------------|:---------------------|:-----------------------------------------------------------| +| `metadataSizeN` | Required |The size of the single message metadata serialized Protobuf | +| `metadataN` | Required |Single message metadata | +| `payloadN` | Required |Message payload passed by application | Each metadata field looks like this; -| Field | Description | -|:---------------------------|:--------------------------------------------------------| -| properties | Application-defined properties | -| partition key *(optional)* | Key to indicate the hashing to a particular partition | -| payload_size | Size of the payload for the single message in the batch | +| Field | Required or optional | Description | +|:----------------|:----------------------|:--------------------------------------------------------| +| `properties` | Required | Application-defined properties | +| `partition key` | Optional | Key to indicate the hashing to a particular partition | +| `payload_size` | Required | Size of the payload for the single message in the batch | When compression is enabled, the whole batch will be compressed at once. @@ -349,7 +362,7 @@ message CommandFlow { Parameters: * `consumer_id` → Id of an already established consumer * `messagePermits` → Number of additional permits to grant to the broker for - pushing more messages + pushing more messages ##### Command Message diff --git a/site2/website-next/docs/developing-load-manager.md b/site2/website-next/docs/developing-load-manager.md index d2ed11b7fbb70..3a791e11681bf 100644 --- a/site2/website-next/docs/developing-load-manager.md +++ b/site2/website-next/docs/developing-load-manager.md @@ -1,7 +1,7 @@ --- id: develop-load-manager title: Modular load manager -sidebar_label: Modular load manager +sidebar_label: "Modular load manager" --- import Tabs from '@theme/Tabs'; @@ -18,11 +18,11 @@ There are two ways that you can enable the modular load manager: 2. Using the `pulsar-admin` tool. Here's an example: ```shell - + $ pulsar-admin update-dynamic-config \ - --config loadManagerClassName \ - --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl - + --config loadManagerClassName \ + --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl + ``` You can use the same method to change back to the original value. In either case, any mistake in specifying the load manager will cause Pulsar to default to `SimpleLoadManagerImpl`. @@ -33,111 +33,113 @@ There are a few different ways to determine which load manager is being used: 1. Use `pulsar-admin` to examine the `loadManagerClassName` element: - ```shell - + ```shell + $ bin/pulsar-admin brokers get-all-dynamic-config { - "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" + "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" } - + ``` If there is no `loadManagerClassName` element, then the default load manager is used. 2. Consult a ZooKeeper load report. With the module load manager, the load report in `/loadbalance/brokers/...` will have many differences. for example the `systemResourceUsage` sub-elements (`bandwidthIn`, `bandwidthOut`, etc.) are now all at the top level. Here is an example load report from the module load manager: - ```json - - { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 4.256510416666667 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 5.287239583333333 - }, - "bundles": [], - "cpu": { - "limit": 2400.0, - "usage": 5.7353247655435915 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - } - } - - ``` - - With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: - - ```json - - { - "systemResourceUsage": { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 0.0 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 0.0 - }, - "cpu": { - "limit": 2400.0, - "usage": 0.0 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - }, - "memory": { - "limit": 8192.0, - "usage": 3903.0 - } - } - } - - ``` + ```json + + { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 4.256510416666667 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 5.287239583333333 + }, + "bundles": [], + "cpu": { + "limit": 2400.0, + "usage": 5.7353247655435915 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + } + } + + ``` + + With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: + + ```json + + { + "systemResourceUsage": { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 0.0 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 0.0 + }, + "cpu": { + "limit": 2400.0, + "usage": 0.0 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + }, + "memory": { + "limit": 8192.0, + "usage": 3903.0 + } + } + } + + ``` 3. The command-line [broker monitor](reference-cli-tools.md#monitor-brokers) will have a different output format depending on which load manager implementation is being used. - Here is an example from the modular load manager: - - ``` - =================================================================================================================== - ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |4 |0 || - ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - =================================================================================================================== - - ``` - - Here is an example from the simple load manager: - - ``` - =================================================================================================================== - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |0 |0 || - ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || - ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.20 |1.89 | |1.27 |3.21 |3.21 || - ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || - ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || - =================================================================================================================== - - ``` + Here is an example from the modular load manager: + + ``` + + =================================================================================================================== + ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |4 |0 || + ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + =================================================================================================================== + + ``` + + Here is an example from the simple load manager: + + ``` + + =================================================================================================================== + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |0 |0 || + ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || + ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.20 |1.89 | |1.27 |3.21 |3.21 || + ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || + ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || + =================================================================================================================== + + ``` It is important to note that the module load manager is _centralized_, meaning that all requests to assign a bundle---whether it's been seen before or whether this is the first time---only get handled by the _lead_ broker (which can change over time). To determine the current lead broker, examine the `/loadbalance/leader` node in ZooKeeper. diff --git a/site2/website-next/docs/developing-tools.md b/site2/website-next/docs/developing-tools.md index 23348fb7f7c49..944a05ed99690 100644 --- a/site2/website-next/docs/developing-tools.md +++ b/site2/website-next/docs/developing-tools.md @@ -1,7 +1,7 @@ --- id: develop-tools title: Simulation tools -sidebar_label: Simulation tools +sidebar_label: "Simulation tools" --- import Tabs from '@theme/Tabs'; @@ -23,6 +23,7 @@ send signals to clients to start incurring load. The client implementation is in To Start a simulation client, use the `pulsar-perf` script with the command `simulation-client` as follows: ``` + pulsar-perf simulation-client --port --service-url ``` @@ -38,6 +39,7 @@ command with. To start a simulation controller, use the `pulsar-perf` script with the command `simulation-controller` as follows: ``` + pulsar-perf simulation-controller --cluster --client-port --clients @@ -50,32 +52,32 @@ names. In all cases, the BASE name of the tenants, namespaces, and topics are us `my_namespace`, and the topic name is `my_topic`. The controller can perform the following actions: * Create a topic with a producer and a consumer - * `trade [--rate ] - [--rand-rate ,] - [--size ]` + * `trade [--rate ] + [--rand-rate ,] + [--size ]` * Create a group of topics with a producer and a consumer - * `trade_group [--rate ] - [--rand-rate ,] - [--separation ] [--size ] - [--topics-per-namespace ]` + * `trade_group [--rate ] + [--rand-rate ,] + [--separation ] [--size ] + [--topics-per-namespace ]` * Change the configuration of an existing topic - * `change [--rate ] - [--rand-rate ,] - [--size ]` + * `change [--rate ] + [--rand-rate ,] + [--size ]` * Change the configuration of a group of topics - * `change_group [--rate ] [--rand-rate ,] - [--size ] [--topics-per-namespace ]` + * `change_group [--rate ] [--rand-rate ,] + [--size ] [--topics-per-namespace ]` * Shutdown a previously created topic - * `stop ` + * `stop ` * Shutdown a previously created group of topics - * `stop_group ` + * `stop_group ` * Copy the historical data from one ZooKeeper to another and simulate based on the message rates and sizes in that history * `copy [--rate-multiplier value]` * Simulate the load of the historical data on the current ZooKeeper (should be same ZooKeeper being simulated on) - * `simulate [--rate-multiplier value]` + * `simulate [--rate-multiplier value]` * Stream the latest data from the given active ZooKeeper to simulate the real-time load of that ZooKeeper. - * `stream [--rate-multiplier value]` + * `stream [--rate-multiplier value]` The "group" arguments in these commands allow the user to create or affect multiple topics at once. Groups are created when calling the `trade_group` command, and all topics from these groups may be subsequently modified or stopped @@ -104,6 +106,7 @@ console as it is updated using watchers. To start a broker monitor, use the `monitor-brokers` command in the `pulsar-perf` script: ``` + pulsar-perf monitor-brokers --connect-string ``` diff --git a/site2/website-next/docs/functions-api.md b/site2/website-next/docs/functions-api.md index 59dfee2b089c6..46d854ffbcca8 100644 --- a/site2/website-next/docs/functions-api.md +++ b/site2/website-next/docs/functions-api.md @@ -1,7 +1,7 @@ --- id: functions-api title: The Pulsar Functions API -sidebar_label: API +sidebar_label: "API" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-debug.md b/site2/website-next/docs/functions-debug.md index 106bc1ac276d2..bd68055ba7229 100644 --- a/site2/website-next/docs/functions-debug.md +++ b/site2/website-next/docs/functions-debug.md @@ -28,8 +28,8 @@ A Pulsar Function is a function with inputs and outputs, you can test a Pulsar F For example, if you have the following Pulsar Function: - ```java + import java.util.function.Function; public class JavaNativeExclamationFunction implements Function { @@ -38,27 +38,32 @@ public class JavaNativeExclamationFunction implements Function { return String.format("%s!", input); } } + ``` You can write a simple unit test to test Pulsar Function. -> #### Tip -> Pulsar uses testng for testing. +:::tip +Pulsar uses testng for testing. + +::: ```java + @Test public void testJavaNativeExclamationFunction() { JavaNativeExclamationFunction exclamation = new JavaNativeExclamationFunction(); String output = exclamation.apply("foo"); Assert.assertEquals(output, "foo!"); } + ``` The following Pulsar Function implements the `org.apache.pulsar.functions.api.Function` interface. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -68,21 +73,26 @@ public class ExclamationFunction implements Function { return String.format("%s!", input); } } + ``` In this situation, you can write a unit test for this function as well. Remember to mock the `Context` parameter. The following is an example. -> #### Tip -> Pulsar uses testng for testing. +:::tip +Pulsar uses testng for testing. + +::: ```java + @Test public void testExclamationFunction() { ExclamationFunction exclamation = new ExclamationFunction(); String output = exclamation.process("foo", mock(Context.class)); Assert.assertEquals(output, "foo!"); } + ``` ## Debug with localrun mode @@ -90,13 +100,16 @@ When you run a Pulsar Function in localrun mode, it launches an instance of the In this mode, a Pulsar Function consumes and produces actual data to a Pulsar cluster, and mirrors how the function actually runs in a Pulsar cluster. -> Note -> Currently, debugging with localrun mode is only supported by Pulsar Functions written in Java. You need Pulsar version 2.4.0 or later to do the following. Even though localrun is available in versions earlier than Pulsar 2.4.0, you cannot debug with localrun mode programmatically or run Functions as threads. +:::note -You can launch your function in the following manner. +Currently, debugging with localrun mode is only supported by Pulsar Functions written in Java. You need Pulsar version 2.4.0 or later to do the following. Even though localrun is available in versions earlier than Pulsar 2.4.0, you cannot debug with localrun mode programmatically or run Functions as threads. + +::: +You can launch your function in the following manner. ```java + FunctionConfig functionConfig = new FunctionConfig(); functionConfig.setName(functionName); functionConfig.setInputs(Collections.singleton(sourceTopic)); @@ -106,14 +119,15 @@ functionConfig.setOutput(sinkTopic); LocalRunner localRunner = LocalRunner.builder().functionConfig(functionConfig).build(); localRunner.start(true); + ``` So you can debug functions using an IDE easily. Set breakpoints and manually step through a function to debug with real data. The following example illustrates how to programmatically launch a function in localrun mode. - ```java + public class ExclamationFunction implements Function { @Override @@ -132,12 +146,13 @@ public static void main(String[] args) throws Exception { LocalRunner localRunner = LocalRunner.builder().functionConfig(functionConfig).build(); localRunner.start(false); } + ``` To use localrun mode programmatically, add the following dependency. - ```xml + org.apache.pulsar pulsar-functions-local-runner @@ -148,8 +163,11 @@ To use localrun mode programmatically, add the following dependency. For complete code samples, see [here](https://github.com/jerrypeng/pulsar-functions-demos/tree/master/debugging). -> Note -> Debugging with localrun mode for Pulsar Functions written in other languages will be supported soon. +:::note + +Debugging with localrun mode for Pulsar Functions written in other languages will be supported soon. + +::: ## Use log topic @@ -159,8 +177,8 @@ In Pulsar Functions, you can generate log information defined in functions to a **Example** - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -180,17 +198,19 @@ public class LoggingFunction implements Function { return null; } } + ``` As shown in the example above, you can get the logger via `context.getLogger()` and assign the logger to the `LOG` variable of `slf4j`, so you can define your desired log information in a function using the `LOG` variable. Meanwhile, you need to specify the topic to which the log information is produced. **Example** - ```bash + $ bin/pulsar-admin functions create \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` The message published to log topic contains several properties for better reasoning: @@ -208,9 +228,11 @@ With [Pulsar Functions CLI](reference-pulsar-admin.md#functions), you can debug * `list` * `trigger` -> **Tip** -> -> For complete commands of **Pulsar Functions CLI**, see [here](reference-pulsar-admin.md#functions)。 +:::tip + +For complete commands of **Pulsar Functions CLI**, see [here](reference-pulsar-admin.md#functions)。 + +::: ### `get` @@ -218,9 +240,10 @@ Get information about a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions get options + ``` **Options** @@ -232,32 +255,37 @@ $ pulsar-admin functions get options |`--namespace`|The namespace of a Pulsar Function. |`--tenant`|The tenant of a Pulsar Function. -> **Tip** -> -> `--fqfn` consists of `--name`, `--namespace` and `--tenant`, so you can specify either `--fqfn` or `--name`, `--namespace` and `--tenant`. +:::tip + +`--fqfn` consists of `--name`, `--namespace` and `--tenant`, so you can specify either `--fqfn` or `--name`, `--namespace` and `--tenant`. + +::: **Example** You can specify `--fqfn` to get information about a Pulsar Function. - ```bash + $ ./bin/pulsar-admin functions get public/default/ExclamationFunctio6 + ``` -Optionally, you can specify `--name`, `--namespace` and `--tenant` to get information about a Pulsar Function. +Optionally, you can specify `--name`, `--namespace` and `--tenant` to get information about a Pulsar Function. ```bash + $ ./bin/pulsar-admin functions get \ --tenant public \ --namespace default \ --name ExclamationFunctio6 + ``` As shown below, the `get` command shows input, output, runtime, and other information about the _ExclamationFunctio6_ function. - ```json + { "tenant": "public", "namespace": "default", @@ -276,6 +304,7 @@ As shown below, the `get` command shows input, output, runtime, and other inform "autoAck": true, "parallelism": 1 } + ``` ### `status` @@ -284,9 +313,10 @@ Check the current status of a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions status options + ``` **Options** @@ -301,18 +331,19 @@ $ pulsar-admin functions status options **Example** - ```bash + $ ./bin/pulsar-admin functions status \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ + ``` As shown below, the `status` command shows the number of instances, running instances, the instance running under the _ExclamationFunctio6_ function, received messages, successfully processed messages, system exceptions, the average latency and so on. - ```json + { "numInstances" : 1, "numRunning" : 1, @@ -334,6 +365,7 @@ As shown below, the `status` command shows the number of instances, running inst } } ] } + ``` ### `stats` @@ -342,9 +374,10 @@ Get the current stats of a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions stats options + ``` **Options** @@ -359,18 +392,19 @@ $ pulsar-admin functions stats options **Example** - ```bash + $ ./bin/pulsar-admin functions stats \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ + ``` The output is shown as follows: - ```json + { "receivedTotal" : 1, "processedSuccessfullyTotal" : 1, @@ -405,6 +439,7 @@ The output is shown as follows: } } ] } + ``` ### `list` @@ -413,9 +448,10 @@ List all Pulsar Functions running under a specific tenant and namespace. **Usage** - ```bash + $ pulsar-admin functions list options + ``` **Options** @@ -427,19 +463,22 @@ $ pulsar-admin functions list options **Example** - ```bash + $ ./bin/pulsar-admin functions list \ --tenant public \ --namespace default + ``` -As shown below, the `list` command returns three functions running under the _public_ tenant and the _default_ namespace. +As shown below, the `list` command returns three functions running under the _public_ tenant and the _default_ namespace. ```text + ExclamationFunctio1 ExclamationFunctio2 ExclamationFunctio3 + ``` ### `trigger` @@ -448,9 +487,10 @@ Trigger a specified Pulsar Function with a supplied value. This command simulate **Usage** - ```bash + $ pulsar-admin functions trigger options + ``` **Options** @@ -467,29 +507,35 @@ $ pulsar-admin functions trigger options **Example** - ```bash + $ ./bin/pulsar-admin functions trigger \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ --topic persistent://public/default/my-topic-1 \ --trigger-value "hello pulsar functions" + ``` As shown below, the `trigger` command returns the following result: - ```text + This is my function! + ``` -> #### **Note** -> You must specify the [entire topic name](getting-started-pulsar.md#topic-names) when using the `--topic` option. Otherwise, the following error occurs. -> -> +:::note + +You must specify the [entire topic name](getting-started-pulsar.md#topic-names) when using the `--topic` option. Otherwise, the following error occurs. + ```text ->Function in trigger function has unidentified topic -> ->Reason: Function in trigger function has unidentified topic ->``` + +Function in trigger function has unidentified topic +Reason: Function in trigger function has unidentified topic + +``` + +::: + diff --git a/site2/website-next/docs/functions-debugging.md b/site2/website-next/docs/functions-debugging.md index cb4d0cccb8d83..a104e84bc1579 100644 --- a/site2/website-next/docs/functions-debugging.md +++ b/site2/website-next/docs/functions-debugging.md @@ -1,7 +1,7 @@ --- id: functions-debugging title: How to debug Pulsar Functions -sidebar_label: Debugging +sidebar_label: "Debugging" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-deploy.md b/site2/website-next/docs/functions-deploy.md index 4ffcfd96be27a..2c3b5d01a32c2 100644 --- a/site2/website-next/docs/functions-deploy.md +++ b/site2/website-next/docs/functions-deploy.md @@ -12,12 +12,12 @@ import TabItem from '@theme/TabItem'; To deploy and manage Pulsar Functions, you need to have a Pulsar cluster running. There are several options for this: -* You can run a [standalone cluster](getting-started-standalone.md) locally on your own machine. -* You can deploy a Pulsar cluster on [Kubernetes](deploy-kubernetes.md), [Amazon Web Services](deploy-aws.md), [bare metal](deploy-bare-metal.md), [DC/OS](deploy-dcos.md), and more. +* You can run a [standalone cluster](getting-started-standalone) locally on your own machine. +* You can deploy a Pulsar cluster on [Kubernetes](deploy-kubernetes.md), [Amazon Web Services](deploy-aws.md), [bare metal](deploy-bare-metal.md), [DC/OS](deploy-dcos), and more. If you run a non-[standalone](reference-terminology.md#standalone) cluster, you need to obtain the service URL for the cluster. How you obtain the service URL depends on how you deploy your Pulsar cluster. -If you want to deploy and trigger Python user-defined functions, you need to install [the pulsar python client](http://pulsar.apache.org/docs/en/client-libraries-python/) on all the machines running [functions workers](functions-worker.md). +If you want to deploy and trigger Python user-defined functions, you need to install [the pulsar python client](http://pulsar.apache.org/docs/en/client-libraries-python/) on all the machines running [functions workers](functions-worker). ## Command-line interface @@ -43,12 +43,13 @@ Pulsar service URL | `pulsar://localhost:6650` Take the `create` command as an example. - ```bash + $ bin/pulsar-admin functions create \ --jar my-pulsar-functions.jar \ --classname org.example.MyFunction \ --inputs my-function-input-topic1,my-function-input-topic2 + ``` The function has default values for the function name (`MyFunction`), tenant (`public`), namespace (`default`), subscription type (`SHARED`), processing guarantees (`ATLEAST_ONCE`), and Pulsar service URL (`pulsar://localhost:6650`). @@ -57,48 +58,52 @@ The function has default values for the function name (`MyFunction`), tenant (`p If you run a Pulsar Function in **local run** mode, it runs on the machine from which you enter the commands (on your laptop, an [AWS EC2](https://aws.amazon.com/ec2/) instance, and so on). The following is a [`localrun`](reference-pulsar-admin.md#localrun) command example. - ```bash + $ bin/pulsar-admin functions localrun \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/input-1 \ --output persistent://public/default/output-1 + ``` By default, the function connects to a Pulsar cluster running on the same machine, via a local [broker](reference-terminology.md#broker) service URL of `pulsar://localhost:6650`. If you use local run mode to run a function but connect it to a non-local Pulsar cluster, you can specify a different broker URL using the `--brokerServiceUrl` flag. The following is an example. - ```bash + $ bin/pulsar-admin functions localrun \ --broker-service-url pulsar://my-cluster-host:6650 \ # Other function parameters + ``` ## Cluster mode When you run a Pulsar Function in **cluster** mode, the function code is uploaded to a Pulsar broker and runs *alongside the broker* rather than in your [local environment](#local-run-mode). You can run a function in cluster mode using the [`create`](reference-pulsar-admin.md#create-1) command. - ```bash + $ bin/pulsar-admin functions create \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/input-1 \ --output persistent://public/default/output-1 + ``` ### Update functions in cluster mode You can use the [`update`](reference-pulsar-admin.md#update-1) command to update a Pulsar Function running in cluster mode. The following command updates the function created in the [cluster mode](#cluster-mode) section. - ```bash + $ bin/pulsar-admin functions update \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/new-input-topic \ --output persistent://public/default/new-output-topic + ``` ### Parallelism @@ -107,40 +112,44 @@ Pulsar Functions run as processes or threads, which are called **instances**. Wh When you create a function, you can specify the *parallelism* of a function (the number of instances to run). You can set the parallelism factor using the `--parallelism` flag of the [`create`](reference-pulsar-admin.md#functions-create) command. - ```bash + $ bin/pulsar-admin functions create \ --parallelism 3 \ # Other function info + ``` You can adjust the parallelism of an already created function using the [`update`](reference-pulsar-admin.md#update-1) interface. - ```bash + $ bin/pulsar-admin functions update \ --parallelism 5 \ # Other function + ``` If you specify a function configuration via YAML, use the `parallelism` parameter. The following is a config file example. - ```yaml + # function-config.yaml parallelism: 3 inputs: - persistent://public/default/input-1 output: persistent://public/default/output-1 # other parameters + ``` The following is corresponding update command. - ```bash + $ bin/pulsar-admin functions update \ --function-config-file function-config.yaml + ``` ### Function instance resources @@ -155,14 +164,15 @@ Disk space | The number of bytes | Docker The following function creation command allocates 8 cores, 8 GB of RAM, and 10 GB of disk space to a function. - ```bash + $ bin/pulsar-admin functions create \ --jar target/my-functions.jar \ --classname org.example.functions.MyFunction \ --cpu 8 \ --ram 8589934592 \ --disk 10737418240 + ``` > #### Resources are *per instance* @@ -172,16 +182,17 @@ $ bin/pulsar-admin functions create \ Package management enables version management and simplifies the upgrade and rollback processes for Functions, Sinks, and Sources. When you use the same function, sink and source in different namespaces, you can upload them to a common package management system. -To use [Package management service](admin-api-packages.md), ensure that the package management service has been enabled in your cluster by setting the following properties in `broker.conf`. +To use [Package management service](admin-api-packages), ensure that the package management service has been enabled in your cluster by setting the following properties in `broker.conf`. > Note: Package management service is not enabled by default. - ```yaml + enablePackagesManagement=true packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider packagesReplicas=1 packagesManagementLedgerRootPath=/ledgers + ``` With Package management service enabled, you can upload your function packages by [upload a package](admin-api-packages.md#upload-a-package) to the service and get the [package URL](admin-api-packages.md#package-url). @@ -196,17 +207,18 @@ If a Pulsar Function is running in [cluster mode](#cluster-mode), you can **trig To learn how to trigger a function, you can start with Python function that returns a simple string based on the input. - ```python + # myfunc.py def process(input): return "This function has been triggered with a value of {0}".format(input) + ``` You can run the function in [local run mode](functions-deploy.md#local-run-mode). - ```bash + $ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ @@ -215,34 +227,38 @@ $ bin/pulsar-admin functions create \ --classname myfunc \ --inputs persistent://public/default/in \ --output persistent://public/default/out + ``` Then assign a consumer to listen on the output topic for messages from the `myfunc` function with the [`pulsar-client consume`](reference-cli-tools.md#consume) command. - ```bash + $ bin/pulsar-client consume persistent://public/default/out \ --subscription-name my-subscription --num-messages 0 # Listen indefinitely + ``` And then you can trigger the function. - ```bash + $ bin/pulsar-admin functions trigger \ --tenant public \ --namespace default \ --name myfunc \ --trigger-value "hello world" + ``` The consumer listening on the output topic produces something as follows in the log. - ``` + ----- got message ----- This function has been triggered with a value of hello world + ``` > #### Topic info is not required diff --git a/site2/website-next/docs/functions-deploying.md b/site2/website-next/docs/functions-deploying.md index eaa96872d3556..226a47f80f93f 100644 --- a/site2/website-next/docs/functions-deploying.md +++ b/site2/website-next/docs/functions-deploying.md @@ -1,7 +1,7 @@ --- id: functions-deploying title: Deploying and managing Pulsar Functions -sidebar_label: Deploying functions +sidebar_label: "Deploying functions" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-develop.md b/site2/website-next/docs/functions-develop.md index a1573671979a5..bd38683564254 100644 --- a/site2/website-next/docs/functions-develop.md +++ b/site2/website-next/docs/functions-develop.md @@ -35,6 +35,7 @@ The language-native function, which adds an exclamation point to all incoming st ```Java + import java.util.function.Function; public class JavaNativeExclamationFunction implements Function { @@ -43,29 +44,37 @@ public class JavaNativeExclamationFunction implements Function { return String.format("%s!", input); } } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeExclamationFunction.java). ```python + def process(input): return "{}!".format(input) + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/native_exclamation_function.py). -> Note -> You can write Pulsar Functions in python2 or python3. However, Pulsar only looks for `python` as the interpreter. -> -> If you're running Pulsar Functions on an Ubuntu system that only supports python3, you might fail to -> start the functions. In this case, you can create a symlink. Your system will fail if -> you subsequently install any other package that depends on Python 2.x. A solution is under development in [Issue 5518](https://github.com/apache/pulsar/issues/5518). -> -> +:::note + +You can write Pulsar Functions in python2 or python3. However, Pulsar only looks for `python` as the interpreter. +If you're running Pulsar Functions on an Ubuntu system that only supports python3, you might fail to +start the functions. In this case, you can create a symlink. Your system will fail if +you subsequently install any other package that depends on Python 2.x. A solution is under development in [Issue 5518](https://github.com/apache/pulsar/issues/5518). + ```bash -> sudo update-alternatives --install /usr/bin/python python /usr/bin/python3 10 ->``` + +sudo update-alternatives --install /usr/bin/python python /usr/bin/python3 10 + +``` + +::: @@ -91,6 +100,7 @@ The following example uses Pulsar Functions SDK. ```Java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -100,13 +110,16 @@ public class ExclamationFunction implements Function { return String.format("%s!", input); } } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/ExclamationFunction.java). ```python + from pulsar import Function class ExclamationFunction(Function): @@ -115,13 +128,16 @@ class ExclamationFunction(Function): def process(self, input, context): return input + '!' + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/exclamation_function.py). ```Go + package main import ( @@ -139,7 +155,9 @@ func HandleRequest(ctx context.Context, in []byte) error{ func main() { pf.Start(HandleRequest) } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f1626a53a1fe2e65dd25f377c1127/pulsar-function-go/examples/inputFunc/inputFunc.go#L20-L36). @@ -169,17 +187,20 @@ SerDe stands for **Ser**ialization and **De**serialization. Pulsar Functions use } ]}> + When you write Pulsar Functions in Java, the following basic Java types are built in and supported by default: `String`, `Double`, `Integer`, `Float`, `Long`, `Short`, and `Byte`. To customize Java types, you need to implement the following interface. - ```java + public interface SerDe { T deserialize(byte[] input); byte[] serialize(T input); } + ``` + SerDe works in the following ways in Java Functions. - If the input and output topics have schema, Pulsar Functions use schema for SerDe. - If the input or output topics do not exist, Pulsar Functions adopt the following rules to determine SerDe: @@ -189,12 +210,13 @@ SerDe works in the following ways in Java Functions. + In Python, the default SerDe is identity, meaning that the type is serialized as whatever type the producer function returns. You can specify the SerDe when [creating](functions-deploy.md#cluster-mode) or [running](functions-deploy.md#local-run-mode) functions. - ```bash + $ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ @@ -204,6 +226,7 @@ $ bin/pulsar-admin functions create \ --custom-serde-inputs '{"input-topic-1":"Serde1","input-topic-2":"Serde2"}' \ --output-serde-classname Serde3 \ --output output-topic-1 + ``` This case contains two input topics: `input-topic-1` and `input-topic-2`, each of which is mapped to a different SerDe class (the map must be specified as a JSON string). The output topic, `output-topic-1`, uses the `Serde3` class for SerDe. At the moment, all Pulsar Functions logic, include processing function and SerDe classes, must be contained within a single Python file. @@ -224,6 +247,7 @@ Custom SerDe | When you require explicit control over SerDe, potentially for per + Currently, the feature is not available in Go. @@ -247,8 +271,8 @@ Imagine that you're writing Pulsar Functions that are processing tweet objects, ]}> - ```java + public class Tweet { private String username; private String tweetContent; @@ -260,12 +284,13 @@ public class Tweet { // Standard setters and getters } + ``` To pass `Tweet` objects directly between Pulsar Functions, you need to provide a custom SerDe class. In the example below, `Tweet` objects are basically strings in which the username and tweet content are separated by a `|`. - ```java + package com.example.serde; import org.apache.pulsar.functions.api.SerDe; @@ -283,6 +308,7 @@ public class TweetSerde implements SerDe { return "%s|%s".format(input.getUsername(), input.getTweetContent()).getBytes(); } } + ``` To apply this customized SerDe to a particular Pulsar Function, you need to: @@ -292,12 +318,13 @@ To apply this customized SerDe to a particular Pulsar Function, you need to: The following is an example of [`create`](reference-pulsar-admin.md#create-1) operation. - ```bash + $ bin/pulsar-admin functions create \ --jar /path/to/your.jar \ --output-serde-classname com.example.serde.TweetSerde \ # Other function attributes + ``` > #### Custom SerDe classes must be packaged with your function JARs @@ -306,12 +333,13 @@ $ bin/pulsar-admin functions create \ - ```python + class Tweet(object): def __init__(self, username, tweet_content): self.username = username self.tweet_content = tweet_content + ``` In order to use this class in Pulsar Functions, you have two options: @@ -319,19 +347,21 @@ In order to use this class in Pulsar Functions, you have two options: 1. You can specify `PickleSerDe`, which applies the [`pickle`](https://docs.python.org/3/library/pickle.html) library SerDe. 2. You can create your own SerDe class. The following is an example. + ```python + + from pulsar import SerDe + + class TweetSerDe(SerDe): + + def serialize(self, input): + return bytes("{0}|{1}".format(input.username, input.tweet_content)) + + def deserialize(self, input_bytes): + tweet_components = str(input_bytes).split('|') + return Tweet(tweet_components[0], tweet_componentsp[1]) + + ``` -```python -from pulsar import SerDe - -class TweetSerDe(SerDe): - - def serialize(self, input): - return bytes("{0}|{1}".format(input.username, input.tweet_content)) - - def deserialize(self, input_bytes): - tweet_components = str(input_bytes).split('|') - return Tweet(tweet_components[0], tweet_componentsp[1]) -``` For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/custom_object_function.py). @@ -377,10 +407,11 @@ Java, Python and Go SDKs provide access to a **context object** that can be used } ]}> -The [Context](https://github.com/apache/pulsar/blob/master/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Context.java) interface provides a number of methods that you can use to access the function [context](#context). The various method signatures for the `Context` interface are listed as follows. +The [Context](https://github.com/apache/pulsar/blob/master/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Context.java) interface provides a number of methods that you can use to access the function [context](#context). The various method signatures for the `Context` interface are listed as follows. ```java + public interface Context { Record getCurrentRecord(); Collection getInputTopics(); @@ -413,12 +444,13 @@ public interface Context { PulsarAdmin getPulsarAdmin(); PulsarAdmin getPulsarAdmin(String clusterName); } + ``` The following example uses several methods available via the `Context` object. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -443,12 +475,14 @@ public class ContextFunction implements Function { return null; } } + ``` ``` + class ContextImpl(pulsar.Context): def get_message_id(self): ... @@ -509,12 +543,14 @@ class ContextImpl(pulsar.Context): ... def get_state(self, key): ... + ``` ``` + func (c *FunctionContext) GetInstanceID() int { return c.instanceConf.instanceID } @@ -566,12 +602,13 @@ func (c *FunctionContext) GetCurrentRecord() pulsar.Message { func (c *FunctionContext) NewOutputMessage(topic string) pulsar.Producer { return c.outputMessage(topic) } + ``` The following example uses several methods available via the `Context` object. - ``` + import ( "context" "fmt" @@ -585,6 +622,7 @@ func contextFunc(ctx context.Context) { fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) } } + ``` For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f1626a53a1fe2e65dd25f377c1127/pulsar-function-go/examples/contextFunc/contextFunc.go#L29-L34). @@ -596,12 +634,13 @@ For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f ### User config When you run or update Pulsar Functions created using SDK, you can pass arbitrary key/values to them with the command line with the `--user-config` flag. Key/values must be specified as JSON. The following function creation command passes a user configured key/value to a function. - ```bash + $ bin/pulsar-admin functions create \ --name word-filter \ # Other function configs --user-config '{"forbidden-word":"rosebud"}' + ``` -The Java SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. +The Java SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. ```bash + $ bin/pulsar-admin functions create \ # Other function configs --user-config '{"word-of-the-day":"verdure"}' + ``` To access that value in a Java function: - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -653,29 +694,32 @@ public class UserConfigFunction implements Function { return null; } } + ``` The `UserConfigFunction` function will log the string `"The word of the day is verdure"` every time the function is invoked (which means every time a message arrives). The `word-of-the-day` user config will be changed only when the function is updated with a new config value via the command line. You can also access the entire user config map or set a default value in case no value is present: - ```java + // Get the whole config map Map allConfigs = context.getUserConfigMap(); // Get value or resort to default String wotd = context.getUserConfigValueOrDefault("word-of-the-day", "perspicacious"); + ``` > For all key/value pairs passed to Java functions, both the key *and* the value are `String`. To set the value to be a different type, you need to deserialize from the `String` type. -In Python function, you can access the configuration value like this. +In Python function, you can access the configuration value like this. ```python + from pulsar import Function class WordFilter(Function): @@ -689,21 +733,23 @@ class WordFilter(Function): # Otherwise publish the message else: return input + ``` The Python SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. - ```bash + $ bin/pulsar-admin functions create \ # Other function configs \ --user-config '{"word-of-the-day":"verdure"}' + ``` To access that value in a Python function: - ```python + from pulsar import Function class UserConfigFunction(Function): @@ -714,23 +760,26 @@ class UserConfigFunction(Function): logger.warn('No word of the day provided') else: logger.info("The word of the day is {0}".format(wotd)) + ``` + The Go SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. - ```bash + $ bin/pulsar-admin functions create \ --go path/to/go/binary --user-config '{"word-of-the-day":"lackadaisical"}' + ``` To access that value in a Go function: - ```go + func contextFunc(ctx context.Context) { fc, ok := pf.FromContext(ctx) if !ok { @@ -745,6 +794,7 @@ func contextFunc(ctx context.Context) { logutil.Infof("The word of the day is %s", wotd.(string)) } } + ``` @@ -770,10 +820,11 @@ func contextFunc(ctx context.Context) { } ]}> -Pulsar Functions that use the Java SDK have access to an [SLF4j](https://www.slf4j.org/) [`Logger`](https://www.slf4j.org/api/org/apache/log4j/Logger.html) object that can be used to produce logs at the chosen log level. The following example logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. +Pulsar Functions that use the Java SDK have access to an [SLF4j](https://www.slf4j.org/) [`Logger`](https://www.slf4j.org/api/org/apache/log4j/Logger.html) object that can be used to produce logs at the chosen log level. The following example logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -793,17 +844,19 @@ public class LoggingFunction implements Function { return null; } } + ``` If you want your function to produce logs, you need to specify a log topic when creating or running the function. The following is an example. - ```bash + $ bin/pulsar-admin functions create \ --jar my-functions.jar \ --classname my.package.LoggingFunction \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` All logs produced by `LoggingFunction` above can be accessed via the `persistent://public/default/logging-function-logs` topic. @@ -812,8 +865,8 @@ All logs produced by `LoggingFunction` above can be accessed via the `persistent Additionally, you can use the XML file, `functions_log4j2.xml`, to customize the function log level. To customize the function log level, create or update `functions_log4j2.xml` in your Pulsar conf directory (for example, `/etc/pulsar/` on bare-metal, or `/pulsar/conf` on Kubernetes) to contain contents such as: - ```xml + pulsar-functions-instance 30 @@ -836,7 +889,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in Console SYSTEM_OUT - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -845,7 +898,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in ${sys:pulsar.function.log.dir}/${sys:pulsar.function.log.file}-%d{MM-dd-yyyy}-%i.log.gz true - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -878,7 +931,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in ${sys:pulsar.function.log.dir}/${sys:pulsar.function.log.file}.bk-%d{MM-dd-yyyy}-%i.log.gz true - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -924,19 +977,24 @@ To customize the function log level, create or update `functions_log4j2.xml` in + ``` The properties set like: ```xml + pulsar.log.level debug + ``` + propagate to places where they are referenced, such as: ```xml + ${sys:pulsar.log.level} @@ -944,12 +1002,14 @@ propagate to places where they are referenced, such as: ${sys:pulsar.log.level} + ``` + In the above example, debug level logging would be applied to ALL function logs. This may be more verbose than you desire. To be more selective, you can apply different log levels to different classes or modules. For example: - ```xml + com.example.module info @@ -958,10 +1018,13 @@ This may be more verbose than you desire. To be more selective, you can apply di ${sys:pulsar.log.appender} + ``` + You can be more specific as well, such as applying a more verbose log level to a class in the module, such as: ```xml + com.example.module.className debug @@ -970,6 +1033,7 @@ You can be more specific as well, such as applying a more verbose log level to a Console + ``` Each `` entry allows you to output the log to a target specified in the definition of the Appender. @@ -978,28 +1042,34 @@ Additivity pertains to whether log messages will be duplicated if multiple Logge To disable additivity, specify ```xml + false + ``` + as shown in examples above. Disabling additivity prevents duplication of log messages when one or more `` entries contain classes or modules that overlap. The `` is defined in the `` section, such as: ```xml + Console SYSTEM_OUT - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n + ``` -Pulsar Functions that use the Python SDK have access to a logging object that can be used to produce logs at the chosen log level. The following example function that logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. +Pulsar Functions that use the Python SDK have access to a logging object that can be used to produce logs at the chosen log level. The following example function that logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```python + from pulsar import Function class LoggingFunction(Function): @@ -1010,17 +1080,19 @@ class LoggingFunction(Function): logger.warn("A warning was received in message {0}".format(context.get_message_id())) else: logger.info("Message {0} received\nContent: {1}".format(msg_id, input)) + ``` If you want your function to produce logs on a Pulsar topic, you need to specify a **log topic** when creating or running the function. The following is an example. - ```bash + $ bin/pulsar-admin functions create \ --py logging_function.py \ --classname logging_function.LoggingFunction \ --log-topic logging-function-logs \ # Other function configs + ``` All logs produced by `LoggingFunction` above can be accessed via the `logging-function-logs` topic. @@ -1028,10 +1100,11 @@ Additionally, you can specify the function log level through the broker XML file -The following Go Function example shows different log levels based on the function input. +The following Go Function example shows different log levels based on the function input. ``` + import ( "context" @@ -1051,6 +1124,7 @@ func loggerFunc(ctx context.Context, input []byte) { func main() { pf.Start(loggerFunc) } + ``` When you use `logTopic` related functionalities in Go Function, import `github.com/apache/pulsar/pulsar-function-go/logutil`, and you do not have to use the `getLogger()` context object. @@ -1077,8 +1151,8 @@ Pulsar Functions using the Java SDK has access to the Pulsar admin client, which Below is an example of how to use the Pulsar admin client exposed from the Function `context`. - ``` + import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1106,17 +1180,19 @@ public class CursorManagementFunction implements Function { return null; } } + ``` If you want your function to get access to the Pulsar admin client, you need to enable this feature by setting `exposeAdminClientEnabled=true` in the `functions_worker.yml` file. You can test whether this feature is enabled or not using the command `pulsar-admin functions localrun` with the flag `--web-service-url`. - ``` + $ bin/pulsar-admin functions localrun \ --jar my-functions.jar \ --classname my.package.CursorManagementFunction \ --web-service-url http://pulsar-web-service:8080 \ # Other function configs + ``` @@ -1127,15 +1203,17 @@ $ bin/pulsar-admin functions localrun \ Pulsar Functions allows you to deploy and manage processing functions that consume messages from and publish messages to Pulsar topics easily. It is important to ensure that the running functions are healthy at any time. Pulsar Functions can publish arbitrary metrics to the metrics interface which can be queried. -> **Note** -> -> If a Pulsar Function uses the language-native interface for Java or Python, that function is not able to publish metrics and stats to Pulsar. +:::note + +If a Pulsar Function uses the language-native interface for Java or Python, that function is not able to publish metrics and stats to Pulsar. + +::: You can monitor Pulsar Functions that have been deployed with the following methods: - Check the metrics provided by Pulsar. - Pulsar Functions expose the metrics that can be collected and used for monitoring the health of **Java, Python, and Go** functions. You can check the metrics by following the [monitoring](deploy-monitoring.md) guide. + Pulsar Functions expose the metrics that can be collected and used for monitoring the health of **Java, Python, and Go** functions. You can check the metrics by following the [monitoring](deploy-monitoring) guide. For the complete list of the function metrics, see [here](reference-metrics.md#pulsar-functions). @@ -1162,10 +1240,11 @@ Here are examples of how to customize metrics for Java and Python functions. } ]}> -You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. +You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1183,14 +1262,16 @@ public class MetricRecorderFunction implements Function { return null; } } + ``` -You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. The following is an example. +You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. The following is an example. ```python + from pulsar import Function class MetricRecorderFunction(Function): @@ -1199,9 +1280,12 @@ class MetricRecorderFunction(Function): if input == 11: context.record_metric('elevens-count', 1) + ``` + + Currently, the feature is not available in Go. @@ -1210,7 +1294,7 @@ Currently, the feature is not available in Go. ## Security -If you want to enable security on Pulsar Functions, first you should enable security on [Functions Workers](functions-worker.md). For more details, refer to [Security settings](functions-worker.md#security-settings). +If you want to enable security on Pulsar Functions, first you should enable security on [Functions Workers](functions-worker). For more details, refer to [Security settings](functions-worker.md#security-settings). Pulsar Functions can support the following providers: @@ -1238,10 +1322,11 @@ At the same time, Pulsar Functions provides two interfaces, **SecretsProvider** } ]}> -You can get secret provider using the [`Context`](#context) object. The following is an example: +You can get secret provider using the [`Context`](#context) object. The following is an example: ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -1262,14 +1347,16 @@ public class GetSecretProviderFunction implements Function { return null; } } + ``` -You can get secret provider using the [`Context`](#context) object. The following is an example: +You can get secret provider using the [`Context`](#context) object. The following is an example: ```python + from pulsar import Function class GetSecretProviderFunction(Function): @@ -1280,11 +1367,12 @@ class GetSecretProviderFunction(Function): logger.warn('No secret provider') else: logger.info("The secret provider is {0}".format(secret_provider)) -``` +``` + Currently, the feature is not available in Go. @@ -1300,8 +1388,11 @@ States are key-value pairs, where the key is a string and the value is arbitrary You can access states within Pulsar Java Functions using the `putState`, `putStateAsync`, `getState`, `getStateAsync`, `incrCounter`, `incrCounterAsync`, `getCounter`, `getCounterAsync` and `deleteState` calls on the context object. You can access states within Pulsar Python Functions using the `putState`, `getState`, `incrCounter`, `getCounter` and `deleteState` calls on the context object. You can also manage states using the [querystate](#query-state) and [putstate](#putstate) options to `pulsar-admin functions`. -> Note -> State storage is not available in Go. +:::note + +State storage is not available in Go. + +::: ### API @@ -1318,26 +1409,28 @@ You can access states within Pulsar Java Functions using the `putState`, `putSta } ]}> + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](functions-develop.md#context) object when you are using Java SDK functions. #### incrCounter - ```java + /** * Increment the builtin distributed counter referred by key * @param key The name of the key * @param amount The amount to be incremented */ void incrCounter(String key, long amount); + ``` The application can use `incrCounter` to change the counter of a given `key` by the given `amount`. #### incrCounterAsync - ```java + /** * Increment the builtin distributed counter referred by key * but dont wait for the completion of the increment operation @@ -1346,14 +1439,15 @@ The application can use `incrCounter` to change the counter of a given `key` by * @param amount The amount to be incremented */ CompletableFuture incrCounterAsync(String key, long amount); + ``` The application can use `incrCounterAsync` to asynchronously change the counter of a given `key` by the given `amount`. #### getCounter - ```java + /** * Retrieve the counter value for the key. * @@ -1361,6 +1455,7 @@ The application can use `incrCounterAsync` to asynchronously change the counter * @return the amount of the counter value for this key */ long getCounter(String key); + ``` The application can use `getCounter` to retrieve the counter of a given `key` mutated by `incrCounter`. @@ -1370,8 +1465,8 @@ general key/value state. #### getCounterAsync - ```java + /** * Retrieve the counter value for the key, but don't wait * for the operation to be completed @@ -1380,14 +1475,15 @@ general key/value state. * @return the amount of the counter value for this key */ CompletableFuture getCounterAsync(String key); + ``` The application can use `getCounterAsync` to asynchronously retrieve the counter of a given `key` mutated by `incrCounterAsync`. #### putState - ```java + /** * Update the state value for the key. * @@ -1395,12 +1491,13 @@ The application can use `getCounterAsync` to asynchronously retrieve the counter * @param value state value of the key */ void putState(String key, ByteBuffer value); + ``` #### putStateAsync - ```java + /** * Update the state value for the key, but don't wait for the operation to be completed * @@ -1408,14 +1505,15 @@ The application can use `getCounterAsync` to asynchronously retrieve the counter * @param value state value of the key */ CompletableFuture putStateAsync(String key, ByteBuffer value); + ``` The application can use `putStateAsync` to asynchronously update the state of a given `key`. #### getState - ```java + /** * Retrieve the state value for the key. * @@ -1423,12 +1521,13 @@ The application can use `putStateAsync` to asynchronously update the state of a * @return the state value for the key. */ ByteBuffer getState(String key); + ``` #### getStateAsync - ```java + /** * Retrieve the state value for the key, but don't wait for the operation to be completed * @@ -1436,33 +1535,37 @@ The application can use `putStateAsync` to asynchronously update the state of a * @return the state value for the key. */ CompletableFuture getStateAsync(String key); + ``` The application can use `getStateAsync` to asynchronously retrieve the state of a given `key`. #### deleteState - ```java + /** * Delete the state value for the key. * * @param key name of the key */ + ``` Counters and binary values share the same keyspace, so this deletes either type. + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](#context) object when you are using Python SDK functions. #### incr_counter - ```python + def incr_counter(self, key, amount): """incr the counter of a given key in the managed state""" + ``` Application can use `incr_counter` to change the counter of a given `key` by the given `amount`. @@ -1470,10 +1573,11 @@ If the `key` does not exist, a new key is created. #### get_counter - ```python + def get_counter(self, key): """get the counter of a given key in the managed state""" + ``` Application can use `get_counter` to retrieve the counter of a given `key` mutated by `incrCounter`. @@ -1483,28 +1587,31 @@ general key/value state. #### put_state - ```python + def put_state(self, key, value): """update the value of a given key in the managed state""" + ``` The key is a string, and the value is arbitrary binary data. #### get_state - ```python + def get_state(self, key): """get the value of a given key in the managed state""" + ``` #### del_counter - ```python + def del_counter(self, key): """delete the counter of a given key in the managed state""" + ``` Counters and binary values share the same keyspace, so this deletes either type. @@ -1519,8 +1626,8 @@ A Pulsar Function can use the [State API](#api) for storing state into Pulsar's and retrieving state back from Pulsar's state storage. Additionally Pulsar also provides CLI commands for querying its state. - ```shell + $ bin/pulsar-admin functions querystate \ --tenant \ --namespace \ @@ -1528,6 +1635,7 @@ $ bin/pulsar-admin functions querystate \ --state-storage-url \ --key \ [---watch] + ``` If `--watch` is specified, the CLI will watch the value of the provided `state-key`. @@ -1548,11 +1656,11 @@ If `--watch` is specified, the CLI will watch the value of the provided `state-k ]}> -{@inject: github:`WordCountFunction`:/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/WordCountFunction.java} is a very good example +{@inject: github:WordCountFunction:/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/WordCountFunction.java} is a very good example demonstrating on how Application can easily store `state` in Pulsar Functions. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1565,6 +1673,7 @@ public class WordCountFunction implements Function { return null; } } + ``` The logic of this `WordCount` function is pretty simple and straightforward: @@ -1575,14 +1684,15 @@ The logic of this `WordCount` function is pretty simple and straightforward: - ```python + from pulsar import Function class WordCount(Function): def process(self, item, context): for word in item.split(): context.incr_counter(word, 1) + ``` The logic of this `WordCount` function is pretty simple and straightforward: diff --git a/site2/website-next/docs/functions-guarantees.md b/site2/website-next/docs/functions-guarantees.md index b16307e4b635f..7e00201076287 100644 --- a/site2/website-next/docs/functions-guarantees.md +++ b/site2/website-next/docs/functions-guarantees.md @@ -1,7 +1,7 @@ --- id: functions-guarantees title: Processing guarantees -sidebar_label: Processing guarantees +sidebar_label: "Processing guarantees" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-metrics.md b/site2/website-next/docs/functions-metrics.md index e64a068869009..4f73f59847700 100644 --- a/site2/website-next/docs/functions-metrics.md +++ b/site2/website-next/docs/functions-metrics.md @@ -1,7 +1,7 @@ --- id: functions-metrics title: Metrics for Pulsar Functions -sidebar_label: Metrics +sidebar_label: "Metrics" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-overview.md b/site2/website-next/docs/functions-overview.md index 90773e7575bc1..1194c88ec6737 100644 --- a/site2/website-next/docs/functions-overview.md +++ b/site2/website-next/docs/functions-overview.md @@ -1,7 +1,7 @@ --- id: functions-overview title: Pulsar Functions overview -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -59,8 +59,8 @@ If you implement the classic word count example using Pulsar Functions, it looks To write the function in Java with [Pulsar Functions SDK for Java](functions-develop.md#available-apis), you can write the function as follows. - ```java + package org.example.functions; import org.apache.pulsar.functions.api.Context; @@ -79,12 +79,13 @@ public class WordCountFunction implements Function { return null; } } + ``` Bundle and build the JAR file to be deployed, and then deploy it in your Pulsar cluster using the [command line](functions-deploy.md#command-line-interface) as follows. - ```bash + $ bin/pulsar-admin functions create \ --jar target/my-jar-with-dependencies.jar \ --classname org.example.functions.WordCountFunction \ @@ -93,6 +94,7 @@ $ bin/pulsar-admin functions create \ --name word-count \ --inputs persistent://public/default/sentences \ --output persistent://public/default/count + ``` ### Content-based routing example @@ -105,8 +107,8 @@ For example, a function takes items (strings) as input and publishes them to eit If you implement this routing functionality in Python, it looks something like this: - ```python + from pulsar import Function class RoutingFunction(Function): @@ -130,12 +132,13 @@ class RoutingFunction(Function): else: warning = "The item {0} is neither a fruit nor a vegetable".format(item) context.get_logger().warn(warning) + ``` If this code is stored in `~/router.py`, then you can deploy it in your Pulsar cluster using the [command line](functions-deploy.md#command-line-interface) as follows. - ```bash + $ bin/pulsar-admin functions create \ --py ~/router.py \ --classname router.RoutingFunction \ @@ -143,6 +146,7 @@ $ bin/pulsar-admin functions create \ --namespace default \ --name route-fruit-veg \ --inputs persistent://public/default/basket-items + ``` ### Functions, messages and message types @@ -154,15 +158,16 @@ Pulsar Functions take byte arrays as inputs and spit out byte arrays as output. ## Fully Qualified Function Name (FQFN) Each Pulsar Function has a **Fully Qualified Function Name** (FQFN) that consists of three elements: the function tenant, namespace, and function name. FQFN looks like this: - ```http + tenant/namespace/name + ``` FQFNs enable you to create multiple functions with the same name provided that they are in different namespaces. ## Supported languages -Currently, you can write Pulsar Functions in Java, Python, and Go. For details, refer to [Develop Pulsar Functions](functions-develop.md). +Currently, you can write Pulsar Functions in Java, Python, and Go. For details, refer to [Develop Pulsar Functions](functions-develop). ## Processing guarantees Pulsar Functions provide three different messaging semantics that you can apply to any function. @@ -177,12 +182,13 @@ Delivery semantics | Description ### Apply processing guarantees to a function You can set the processing guarantees for a Pulsar Function when you create the Function. The following [`pulsar-function create`](reference-pulsar-admin.md#create-1) command creates a function with effectively-once guarantees applied. - ```bash + $ bin/pulsar-admin functions create \ --name my-effectively-once-function \ --processing-guarantees EFFECTIVELY_ONCE \ # Other function configs + ``` The available options for `--processing-guarantees` are: @@ -196,9 +202,11 @@ The available options for `--processing-guarantees` are: ### Update the processing guarantees of a function You can change the processing guarantees applied to a function using the [`update`](reference-pulsar-admin.md#update-1) command. The following is an example. - ```bash + $ bin/pulsar-admin functions update \ --processing-guarantees ATMOST_ONCE \ # Other function configs + ``` + diff --git a/site2/website-next/docs/functions-package.md b/site2/website-next/docs/functions-package.md index bfe9e099c30b7..65ca40495491b 100644 --- a/site2/website-next/docs/functions-package.md +++ b/site2/website-next/docs/functions-package.md @@ -10,12 +10,15 @@ import TabItem from '@theme/TabItem'; You can package Pulsar functions in Java, Python, and Go. Packaging the window function in Java is the same as [packaging a function in Java](#java). -> **Note** -> Currently, the window function is not available in Python and Go. +:::note + +Currently, the window function is not available in Python and Go. + +::: ## Prerequisite -Before running a Pulsar function, you need to start Pulsar. You can [run a standalone Pulsar in Docker](getting-started-docker.md), or [run Pulsar in Kubernetes](getting-started-helm.md). +Before running a Pulsar function, you need to start Pulsar. You can [run a standalone Pulsar in Docker](getting-started-docker.md), or [run Pulsar in Kubernetes](getting-started-helm). To check whether the Docker image starts, you can use the `docker ps` command. @@ -25,151 +28,158 @@ To package a function in Java, complete the following steps. 1. Create a new maven project with a pom file. In the following code sample, the value of `mainClass` is your package name. - -```Java - - - 4.0.0 - - java-function - java-function - 1.0-SNAPSHOT - - - - org.apache.pulsar - pulsar-functions-api - 2.6.0 - - - - - - - maven-assembly-plugin - - false - - jar-with-dependencies - - - - org.example.test.ExclamationFunction - - - - - - make-assembly - package - - assembly - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - 8 - 8 - - - - - - -``` + ```Java + + + + 4.0.0 + + java-function + java-function + 1.0-SNAPSHOT + + + + org.apache.pulsar + pulsar-functions-api + 2.6.0 + + + + + + + maven-assembly-plugin + + false + + jar-with-dependencies + + + + org.example.test.ExclamationFunction + + + + + + make-assembly + package + + assembly + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + + + ``` 2. Write a Java function. + ``` + + package org.example.test; -``` - package org.example.test; - - import java.util.function.Function; + import java.util.function.Function; - public class ExclamationFunction implements Function { - @Override - public String apply(String s) { - return "This is my function!"; - } - } -``` - - For the imported package, you can use one of the following interfaces: - - Function interface provided by Java 8: `java.util.function.Function` - - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` - - The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. - - The following example uses `org.apache.pulsar.functions.api.Function` interface with context. - - -``` - package org.example.functions; - import org.apache.pulsar.functions.api.Context; - import org.apache.pulsar.functions.api.Function; - - import java.util.Arrays; - public class WordCountFunction implements Function { - // This function is invoked every time a message is published to the input topic - @Override - public Void process(String input, Context context) throws Exception { - Arrays.asList(input.split(" ")).forEach(word -> { - String counterKey = word.toLowerCase(); - context.incrCounter(counterKey, 1); - }); - return null; + public class ExclamationFunction implements Function { + @Override + public String apply(String s) { + return "This is my function!"; } - } -``` + } + + ``` + + For the imported package, you can use one of the following interfaces: + - Function interface provided by Java 8: `java.util.function.Function` + - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` + + The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. + + The following example uses `org.apache.pulsar.functions.api.Function` interface with context. + + ``` + + package org.example.functions; + import org.apache.pulsar.functions.api.Context; + import org.apache.pulsar.functions.api.Function; + + import java.util.Arrays; + public class WordCountFunction implements Function { + // This function is invoked every time a message is published to the input topic + @Override + public Void process(String input, Context context) throws Exception { + Arrays.asList(input.split(" ")).forEach(word -> { + String counterKey = word.toLowerCase(); + context.incrCounter(counterKey, 1); + }); + return null; + } + } + + ``` 3. Package the Java function. + ```bash + + mvn package + + ``` -```bash - mvn package -``` - - After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. + After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. 4. Run the Java function. - (1) Copy the packaged jar file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Java function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --jar java-function-1.0-SNAPSHOT.jar \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name JavaFunction -``` - - The following log indicates that the Java function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the packaged jar file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Java function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname org.example.test.ExclamationFunction \ + --jar java-function-1.0-SNAPSHOT.jar \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name JavaFunction + + ``` + + The following log indicates that the Java function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Python @@ -185,69 +195,74 @@ To package a function with **one python file** in Python, complete the following 1. Write a Python function. + ``` + + from pulsar import Function // import the Function module from Pulsar -``` - from pulsar import Function // import the Function module from Pulsar - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass - def process(self, input, context): - return input + '!' -``` + def process(self, input, context): + return input + '!' + + ``` - In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. + In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. - `process()` mainly has two parameters: + `process()` mainly has two parameters: - - `input` represents your input. + - `input` represents your input. - - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. + - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. 2. Install a Python client. - The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. + The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. - -```bash - pip install python-client==2.6.0 -``` + ```bash + + pip install pulsar-client==2.6.0 + + ``` 3. Run the Python Function. - (1) Copy the Python function file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Python function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --py \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name PythonFunction -``` - - The following log indicates that the Python function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the Python function file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname . \ + --py \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` + + The following log indicates that the Python function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### ZIP file @@ -255,58 +270,64 @@ To package a function with the **ZIP file** in Python, complete the following st 1. Prepare the ZIP file. - The following is required when packaging the ZIP file of the Python Function. - - -```text - Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: - "func/src" - "func/requirements.txt" - "func/deps" -``` - Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. - - -```text - . - ├── deps - │   └── sh-1.12.14-py2.py3-none-any.whl - └── src - └── exclamation.py -``` + The following is required when packaging the ZIP file of the Python Function. + + ```text + + Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: + "func/src" + "func/requirements.txt" + "func/deps" + + ``` + + Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. + + ```text + + . + ├── deps + │   └── sh-1.12.14-py2.py3-none-any.whl + └── src + └── exclamation.py + + ``` 2. Run the Python Function. - (1) Copy the ZIP file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Python function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname exclamation \ - --py \ - --inputs persistent://public/default/in-topic \ - --output persistent://public/default/out-topic \ - --tenant public \ - --namespace default \ - --name PythonFunction -``` - - The following log indicates that the Python function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the ZIP file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname exclamation \ + --py \ + --inputs persistent://public/default/in-topic \ + --output persistent://public/default/out-topic \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` + + The following log indicates that the Python function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### PIP @@ -314,50 +335,54 @@ The PIP method is only supported in Kubernetes runtime. To package a function wi 1. Configure the `functions_worker.yml` file. - -```text - #### Kubernetes Runtime #### - installUserCodeDependencies: true -``` + ```text + + #### Kubernetes Runtime #### + installUserCodeDependencies: true + + ``` 2. Write your Python Function. + ``` + + from pulsar import Function + import js2xml -``` - from pulsar import Function - import js2xml - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass - def process(self, input, context): - // add your logic - return input + '!' -``` + def process(self, input, context): + // add your logic + return input + '!' + + ``` - You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. + You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. 3. Generate the `whl` file. - -```shell script - $ cd $PULSAR_HOME/pulsar-functions/scripts/python - $ chmod +x generate.sh - $ ./generate.sh - # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 -``` - - The output is written in `/path/to/python/output`: - - -```text - -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl - -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz - -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl -``` + ```shell script + + $ cd $PULSAR_HOME/pulsar-functions/scripts/python + $ chmod +x generate.sh + $ ./generate.sh + # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 + + ``` + + The output is written in `/path/to/python/output`: + + ```text + + -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl + -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz + -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl + + ``` ## Go @@ -365,99 +390,107 @@ To package a function in Go, complete the following steps. 1. Write a Go function. - Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". + Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". + ``` + + import ( + "context" + "fmt" -``` - import ( - "context" - "fmt" + "github.com/apache/pulsar/pulsar-function-go/pf" + ) - "github.com/apache/pulsar/pulsar-function-go/pf" - ) + func HandleRequest(ctx context.Context, input []byte) error { + fmt.Println(string(input) + "!") + return nil + } - func HandleRequest(ctx context.Context, input []byte) error { - fmt.Println(string(input) + "!") - return nil - } + func main() { + pf.Start(HandleRequest) + } + + ``` + + You can use context to connect to the Go function. - func main() { - pf.Start(HandleRequest) + ``` + + if fc, ok := pf.FromContext(ctx); ok { + fmt.Printf("function ID is:%s, ", fc.GetFuncID()) + fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) } -``` - - You can use context to connect to the Go function. - - -``` - if fc, ok := pf.FromContext(ctx); ok { - fmt.Printf("function ID is:%s, ", fc.GetFuncID()) - fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) - } -``` - - When writing a Go function, remember that - - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. - - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: - - -``` - func () - func () error - func (input) error - func () (output, error) - func (input) (output, error) - func (context.Context) error - func (context.Context, input) error - func (context.Context) (output, error) - func (context.Context, input) (output, error) -``` + + ``` + + When writing a Go function, remember that + - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. + - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: + + ``` + + func () + func () error + func (input) error + func () (output, error) + func (input) (output, error) + func (context.Context) error + func (context.Context, input) error + func (context.Context) (output, error) + func (context.Context, input) (output, error) + + ``` 2. Build the Go function. - -``` - go build .go -``` + ``` + + go build .go + + ``` 3. Run the Go Function. - (1) Copy the Go function file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Go function with the following command. - - -``` - ./bin/pulsar-admin functions localrun \ - --go [your go function path] - --inputs [input topics] \ - --output [output topic] \ - --tenant [default:public] \ - --namespace [default:default] \ - --name [custom unique go function name] -``` - - The following log indicates that the Go function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the Go function file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Go function with the following command. + + ``` + + ./bin/pulsar-admin functions localrun \ + --go [your go function path] + --inputs [input topics] \ + --output [output topic] \ + --tenant [default:public] \ + --namespace [default:default] \ + --name [custom unique go function name] + + ``` + + The following log indicates that the Go function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Start Functions in cluster mode If you want to start a function in cluster mode, replace `localrun` with `create` in the commands above. The following log indicates that your function starts successfully. - -```text + ```text + "Created successfully" -``` + + ``` For information about parameters on `--classname`, `--jar`, `--py`, `--go`, `--inputs`, run the command `./bin/pulsar-admin functions` or see [here](reference-pulsar-admin.md#functions). \ No newline at end of file diff --git a/site2/website-next/docs/functions-quickstart.md b/site2/website-next/docs/functions-quickstart.md index 9178e6a768770..d822c7993bc10 100644 --- a/site2/website-next/docs/functions-quickstart.md +++ b/site2/website-next/docs/functions-quickstart.md @@ -1,7 +1,7 @@ --- id: functions-quickstart title: Getting started with Pulsar Functions -sidebar_label: Getting started +sidebar_label: "Getting started" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-runtime.md b/site2/website-next/docs/functions-runtime.md index c573761f74337..4358f63cda32e 100644 --- a/site2/website-next/docs/functions-runtime.md +++ b/site2/website-next/docs/functions-runtime.md @@ -14,8 +14,11 @@ You can use the following methods to run functions. - *Process*: Invoke functions in processes forked by functions worker. - *Kubernetes*: Submit functions as Kubernetes StatefulSets by functions worker. -> **Note** -> Pulsar supports adding labels to the Kubernetes StatefulSets and services while launching functions, which facilitates selecting the target Kubernetes objects. +:::note + +Pulsar supports adding labels to the Kubernetes StatefulSets and services while launching functions, which facilitates selecting the target Kubernetes objects. + +::: The differences of the thread and process modes are: - Thread mode: when a function runs in thread mode, it runs on the same Java virtual machine (JVM) with functions worker. @@ -24,11 +27,12 @@ The differences of the thread and process modes are: ## Configure thread runtime It is easy to configure *Thread* runtime. In most cases, you do not need to configure anything. You can customize the thread group name with the following settings: - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory functionRuntimeFactoryConfigs: threadGroupName: "Your Function Container Group" + ``` *Thread* runtime is only supported in Java function. @@ -36,8 +40,8 @@ functionRuntimeFactoryConfigs: ## Configure process runtime When you enable *Process* runtime, you do not need to configure anything. - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.process.ProcessRuntimeFactory functionRuntimeFactoryConfigs: # the directory for storing the function logs @@ -48,6 +52,7 @@ functionRuntimeFactoryConfigs: pythonInstanceLocation: # change the extra dependencies location: extraFunctionDependenciesDir: + ``` *Process* runtime is supported in Java, Python, and Go functions. @@ -60,16 +65,18 @@ The manifests, generated by the functions worker, include a `StatefulSet`, a `Se The Kubernetes runtime supports secrets, so you can create a Kubernetes secret and expose it as an environment variable in the pod. The Kubernetes runtime is extensible, you can implement classes and customize the way how to generate Kubernetes manifests, how to pass auth data to pods, and how to integrate secrets. -> **Tip** -> -> For the rules of translating Pulsar object names into Kubernetes resource labels, see [here](admin-api-overview.md#how-to-define-pulsar-resource-names-when-running-pulsar-in-kubernetes). +:::tip + +For the rules of translating Pulsar object names into Kubernetes resource labels, see [here](admin-api-overview.md#how-to-define-pulsar-resource-names-when-running-pulsar-in-kubernetes). + +::: ### Basic configuration It is easy to configure Kubernetes runtime. You can just uncomment the settings of `kubernetesContainerFactory` in the `functions_worker.yaml` file. The following is an example. - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntimeFactory functionRuntimeFactoryConfigs: # uri to kubernetes cluster, leave it to empty and it will use the kubernetes settings in function worker @@ -143,6 +150,7 @@ functionRuntimeFactoryConfigs: extraFunctionDependenciesDir: # Additional memory padding added on top of the memory requested by the function per on a per instance basis percentMemoryPadding: 10 + ``` If you run functions worker embedded in a broker on Kubernetes, you can use the default settings. @@ -153,10 +161,11 @@ If you run functions worker standalone (that is, not embedded) on Kubernetes, yo For example, both Pulsar brokers and Function Workers run in the `pulsar` K8S namespace. The brokers have a service called `brokers` and the functions worker has a service called `func-worker`. The settings are as follows: - ```yaml + pulsarServiceUrl: pulsar://broker.pulsar:6650 // or pulsar+ssl://broker.pulsar:6651 if using TLS pulsarAdminUrl: http://func-worker.pulsar:8080 // or https://func-worker:8443 if using TLS + ``` ### Run RBAC in Kubernetes clusters @@ -170,8 +179,8 @@ If you run RBAC in your Kubernetes cluster, make sure that the service account y The following is sufficient: - ```yaml + apiVersion: rbac.authorization.k8s.io/v1beta1 kind: ClusterRole metadata: @@ -207,11 +216,13 @@ roleRef: subjectsKubernetesSec: - kind: ServiceAccount name: functions-worker + ``` If the service-account is not properly configured, an error message similar to this is displayed: ```bash + 22:04:27.696 [Timer-0] ERROR org.apache.pulsar.functions.runtime.KubernetesRuntimeFactory - Error while trying to fetch configmap example-pulsar-4qvmb5gur3c6fc9dih0x1xn8b-function-worker-config at namespace pulsar io.kubernetes.client.ApiException: Forbidden at io.kubernetes.client.ApiClient.handleResponse(ApiClient.java:882) ~[io.kubernetes-client-java-2.0.0.jar:?] @@ -222,6 +233,7 @@ io.kubernetes.client.ApiException: Forbidden at org.apache.pulsar.functions.runtime.KubernetesRuntimeFactory$1.run(KubernetesRuntimeFactory.java:275) [org.apache.pulsar-pulsar-functions-runtime-2.4.0-42c3bf949.jar:2.4.0-42c3bf949] at java.util.TimerThread.mainLoop(Timer.java:555) [?:1.8.0_212] at java.util.TimerThread.run(Timer.java:505) [?:1.8.0_212] + ``` ### Integrate Kubernetes secrets @@ -230,8 +242,8 @@ In order to safely distribute secrets, Pulasr Functions can reference Kubernetes You can create a secret in the namespace where your functions are deployed. For example, you deploy functions to the `pulsar-func` Kubernetes namespace, and you have a secret named `database-creds` with a field name `password`, which you want to mount in the pod as an environment variable called `DATABASE_PASSWORD`. The following functions configuration enables you to reference that secret and mount the value as an environment variable in the pod. - ```Yaml + tenant: "mytenant" namespace: "mynamespace" name: "myfunction" @@ -254,17 +266,21 @@ The `org.apache.pulsar.functions.auth.KubernetesFunctionAuthProvider` interface Pulsar includes an implementation of this interface for token authentication, and distributes the certificate authority via the same implementation. The configuration is similar as follows: - ```Yaml + functionAuthProviderClassName: org.apache.pulsar.functions.auth.KubernetesSecretsTokenAuthProvider + ``` For token authentication, the functions worker captures the token that is used to deploy (or update) the function. The token is saved as a secret and mounted into the pod. For custom authentication or TLS, you need to implement this interface or use an alternative mechanism to provide authentication. If you use token authentication and TLS encryption to secure the communication with the cluster, Pulsar passes your certificate authority (CA) to the client, so the client obtains what it needs to authenticate the cluster, and trusts the cluster with your signed certificate. -> **Note** -> If you use tokens that expire when deploying functions, these tokens will expire. +:::note + +If you use tokens that expire when deploying functions, these tokens will expire. + +::: ### Run clusters with authentication @@ -272,8 +288,8 @@ When you run a functions worker in a standalone process (that is, not embedded i For example, if you use token authentication, you need to configure the following properties in the `function-worker.yml` file. - ```Yaml + clientAuthenticationPlugin: org.apache.pulsar.client.impl.auth.AuthenticationToken clientAuthenticationParameters: file:///etc/pulsar/token/admin-token.txt configurationStoreServers: zookeeper-cluster:2181 # auth requires a connection to zookeeper @@ -287,10 +303,14 @@ superUserRoles: properties: tokenSecretKey: file:///etc/pulsar/jwt/secret # if using a secret token, key file must be DER-encoded tokenPublicKey: file:///etc/pulsar/jwt/public.key # if using public/private key tokens, key file must be DER-encoded + ``` -> **Note** -> You must configure both the Function Worker authorization or authentication for the server to authenticate requests and configure the client to be authenticated to communicate with the broker. +:::note + +You must configure both the Function Worker authorization or authentication for the server to authenticate requests and configure the client to be authenticated to communicate with the broker. + +::: ### Customize Kubernetes runtime @@ -304,8 +324,8 @@ Pulsar includes a built-in implementation. To use the basic implementation, set Below is an example of `customRuntimeOptions`. - ```json + { "jobName": "jobname", // the k8s pod name to run this function instance "jobNamespace": "namespace", // the k8s namespace to run this function in @@ -336,6 +356,7 @@ Below is an example of `customRuntimeOptions`. } } } + ``` ## Run clusters with geo-replication @@ -344,16 +365,18 @@ If you run multiple clusters tied together with geo-replication, it is important For example, if you have two clusters: `east-1` and `west-1`, you can configure the functions workers for `east-1` and `west-1` perspectively as follows. - ```Yaml + pulsarFunctionsCluster: east-1 pulsarFunctionsNamespace: public/functions-east-1 -``` +``` ```Yaml + pulsarFunctionsCluster: west-1 pulsarFunctionsNamespace: public/functions-west-1 + ``` This ensures the two different Functions Workers use distinct sets of topics for their internal coordination. @@ -364,8 +387,8 @@ When configuring a standalone functions worker, you need to configure properties You need to configure the following required properties. - ```Yaml + workerPort: 8080 workerPortTls: 8443 # when using TLS tlsCertificateFilePath: /etc/pulsar/tls/tls.crt # when using TLS @@ -375,4 +398,5 @@ pulsarServiceUrl: pulsar://broker.pulsar:6650/ # or pulsar+ssl://pulsar-prod-bro pulsarWebServiceUrl: http://broker.pulsar:8080/ # or https://pulsar-prod-broker.pulsar:8443/ when using TLS useTls: true # when using TLS, critical! -``` \ No newline at end of file +``` + diff --git a/site2/website-next/docs/functions-state.md b/site2/website-next/docs/functions-state.md index e91eb194690a5..55dc6e5ab375f 100644 --- a/site2/website-next/docs/functions-state.md +++ b/site2/website-next/docs/functions-state.md @@ -1,7 +1,7 @@ --- id: functions-state title: Pulsar Functions State Storage (Developer Preview) -sidebar_label: State Storage +sidebar_label: "State Storage" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/functions-worker.md b/site2/website-next/docs/functions-worker.md index a49314067473a..dcb7c20ef6519 100644 --- a/site2/website-next/docs/functions-worker.md +++ b/site2/website-next/docs/functions-worker.md @@ -7,14 +7,17 @@ sidebar_label: "Setup: Pulsar Functions Worker" import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -Before using Pulsar Functions, you need to learn how to set up Pulsar Functions worker and how to [configure Functions runtime](functions-runtime.md). +Before using Pulsar Functions, you need to learn how to set up Pulsar Functions worker and how to [configure Functions runtime](functions-runtime). Pulsar `functions-worker` is a logic component to run Pulsar Functions in cluster mode. Two options are available, and you can select either based on your requirements. - [run with brokers](#run-functions-worker-with-brokers) - [run it separately](#run-functions-worker-separately) in a different broker -> Note -> The `--- Service Urls---` lines in the following diagrams represent Pulsar service URLs that Pulsar client and admin use to connect to a Pulsar cluster. +:::note + +The `--- Service Urls---` lines in the following diagrams represent Pulsar service URLs that Pulsar client and admin use to connect to a Pulsar cluster. + +::: ## Run Functions-worker with brokers @@ -24,9 +27,10 @@ The following diagram illustrates the deployment of functions-workers running al To enable functions-worker running as part of a broker, you need to set `functionsWorkerEnabled` to `true` in the `broker.conf` file. - ```conf + functionsWorkerEnabled=true + ``` If the `functionsWorkerEnabled` is set to `true`, the functions-worker is started as part of a broker. You need to configure the `conf/functions_worker.yml` file to customize your functions_worker. @@ -55,35 +59,41 @@ If you want to use Stateful-Functions related functions (for example, `putState Currently, the service uses the NAR package, so you need to set the configuration in `bookkeeper.conf`. - -```text - extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent -``` + ```text + + extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent + + ``` After starting bookie, use the following methods to check whether the streamStorage service is started correctly. Input: + + ```shell + + telnet localhost 4181 + ``` -```shell - telnet localhost 4181 -``` Output: -```text - Trying 127.0.0.1... - Connected to localhost. - Escape character is '^]'. -``` + ```text + + Trying 127.0.0.1... + Connected to localhost. + Escape character is '^]'. + + ``` 2. Turn on this function in `functions_worker.yml`. + ```text + + stateStorageServiceUrl: bk://:4181 + + ``` -```text - stateStorageServiceUrl: bk://:4181 -``` - - `bk-service-url` is the service URL pointing to the BookKeeper table service. + `bk-service-url` is the service URL pointing to the BookKeeper table service. ### Start Functions-worker with broker @@ -91,16 +101,18 @@ Once you have configured the `functions_worker.yml` file, you can start or resta And then you can use the following command to verify if `functions-worker` is running well. - ```bash + curl :8080/admin/v2/worker/cluster + ``` After entering the command above, a list of active function workers in the cluster is returned. The output is similar to the following. - ```json + [{"workerId":"","workerHostname":"","port":8080}] + ``` ## Run Functions-worker separately @@ -109,8 +121,11 @@ This section illustrates how to run `functions-worker` as a separate process in ![assets/functions-worker-separated.png](/assets/functions-worker-separated.png) -> Note -> In this mode, make sure `functionsWorkerEnabled` is set to `false`, so you won't start `functions-worker` with brokers by mistake. Also, while accessing the `functions-worker` to manage any of the functions, the `pulsar-admin` CLI tool or any of the clients should use the `workerHostname` and `workerPort` that you set in [Worker parameters](#worker-parameters) to generate an `--admin-url`. +:::note + +In this mode, make sure `functionsWorkerEnabled` is set to `false`, so you won't start `functions-worker` with brokers by mistake. Also, while accessing the `functions-worker` to manage any of the functions, the `pulsar-admin` CLI tool or any of the clients should use the `workerHostname` and `workerPort` that you set in [Worker parameters](#worker-parameters) to generate an `--admin-url`. + +::: ### Configure Functions-worker to run separately @@ -150,8 +165,8 @@ If you want to enable security on functions workers, you *should*: To enable TLS transport encryption, configure the following settings. - ``` + useTLS: true pulsarServiceUrl: pulsar+ssl://localhost:6651/ pulsarWebServiceUrl: https://localhost:8443 @@ -163,87 +178,98 @@ tlsTrustCertsFilePath: /path/to/ca.cert.pem // The path to trusted certificates used by the Pulsar client to authenticate with Pulsar brokers brokerClientTrustCertsFilePath: /path/to/ca.cert.pem + ``` -For details on TLS encryption, refer to [Transport Encryption using TLS](security-tls-transport.md). +For details on TLS encryption, refer to [Transport Encryption using TLS](security-tls-transport). ##### Enable Authentication Provider To enable authentication on Functions Worker, you need to configure the following settings. -> Note -> Substitute the *providers list* with the providers you want to enable. +:::note + +Substitute the *providers list* with the providers you want to enable. +::: ``` + authenticationEnabled: true authenticationProviders: [ provider1, provider2 ] + ``` For *TLS Authentication* provider, follow the example below to add the necessary settings. -See [TLS Authentication](security-tls-authentication.md) for more details. - +See [TLS Authentication](security-tls-authentication) for more details. ``` + brokerClientAuthenticationPlugin: org.apache.pulsar.client.impl.auth.AuthenticationTls brokerClientAuthenticationParameters: tlsCertFile:/path/to/admin.cert.pem,tlsKeyFile:/path/to/admin.key-pk8.pem authenticationEnabled: true authenticationProviders: ['org.apache.pulsar.broker.authentication.AuthenticationProviderTls'] + ``` For *SASL Authentication* provider, add `saslJaasClientAllowedIds` and `saslJaasBrokerSectionName` under `properties` if needed. - ``` + properties: saslJaasClientAllowedIds: .*pulsar.* saslJaasBrokerSectionName: Broker + ``` For *Token Authentication* provider, add necessary settings for `properties` if needed. -See [Token Authentication](security-jwt.md) for more details. +See [Token Authentication](security-jwt) for more details. Note: key files must be DER-encoded ``` + properties: tokenSecretKey: file://my/secret.key # If using public/private - # tokenPublicKey: file:///path/to/public.key + # tokenPublicKey: file:///path/to/public.key + ``` ##### Enable Authorization Provider To enable authorization on Functions Worker, you need to configure `authorizationEnabled`, `authorizationProvider` and `configurationStoreServers`. The authentication provider connects to `configurationStoreServers` to receive namespace policies. - ```yaml + authorizationEnabled: true authorizationProvider: org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider configurationStoreServers: + ``` You should also configure a list of superuser roles. The superuser roles are able to access any admin API. The following is a configuration example. - ```yaml + superUserRoles: - role1 - role2 - role3 + ``` ##### Enable End-to-End Encryption You can use the public and private key pair that the application configures to perform encryption. Only the consumers with a valid key can decrypt the encrypted messages. -To enable End-to-End encryption on Functions Worker, you can set it by specifying `--producer-config` in the command line terminal, for more information, please refer to [here](security-encryption.md). +To enable End-to-End encryption on Functions Worker, you can set it by specifying `--producer-config` in the command line terminal, for more information, please refer to [here](security-encryption). We include the relevant configuration information of `CryptoConfig` into `ProducerConfig`. The specific configurable field information about `CryptoConfig` is as follows: - ```text + public class CryptoConfig { private String cryptoKeyReaderClassName; private Map cryptoKeyReaderConfig; @@ -253,6 +279,7 @@ public class CryptoConfig { private ConsumerCryptoFailureAction consumerCryptoFailureAction; } + ``` - `producerCryptoFailureAction`: define the action if producer fail to encrypt data one of `FAIL`, `SEND`. @@ -270,16 +297,18 @@ If authentication is enabled on the BookKeeper cluster, you need configure the B Once you have finished configuring the `functions_worker.yml` configuration file, you can start a `functions-worker` in the background by using [nohup](https://en.wikipedia.org/wiki/Nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: - ```bash + bin/pulsar-daemon start functions-worker + ``` You can also start `functions-worker` in the foreground by using `pulsar` CLI tool: - ```bash + bin/pulsar functions-worker + ``` ### Configure Proxies for Functions-workers @@ -297,10 +326,11 @@ start proxies. To enable routing functions related admin requests to `functions-worker` in a proxy, you can edit the `proxy.conf` file to modify the following settings: - ```conf + functionWorkerWebServiceURL= functionWorkerWebServiceURLTLS= + ``` ## Compare the Run-with-Broker and Run-separately modes @@ -321,9 +351,10 @@ Use the `Run-separately` mode in the following cases: **Error message: Namespace missing local cluster name in clusters list** - ``` + Failed to get partitioned topic metadata: org.apache.pulsar.client.api.PulsarClientException$BrokerMetadataException: Namespace missing local cluster name in clusters list: local_cluster=xyz ns=public/functions clusters=[standalone] + ``` The error message prompts when either of the cases occurs: @@ -338,16 +369,18 @@ If any of these cases happens, follow the instructions below to fix the problem: 2. Get the current clusters list of `public/functions` namespace. - ```bash + bin/pulsar-admin namespaces get-clusters public/functions + ``` 3. Check if the cluster is in the clusters list. If the cluster is not in the list, add it to the list and update the clusters list. - ```bash + bin/pulsar-admin namespaces set-clusters --clusters , public/functions + ``` 4. After setting the cluster successfully, enable functions worker by setting `functionsWorkerEnabled=true`. diff --git a/site2/website-next/docs/getting-started-clients.md b/site2/website-next/docs/getting-started-clients.md index 1060aad167500..a6b1d28951fda 100644 --- a/site2/website-next/docs/getting-started-clients.md +++ b/site2/website-next/docs/getting-started-clients.md @@ -1,18 +1,22 @@ --- id: client-libraries title: Pulsar client libraries -sidebar_label: Overview +sidebar_label: "Overview" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar supports the following client libraries: -- [Java client](client-libraries-java.md) -- [Go client](client-libraries-go.md) -- [Python client](client-libraries-python.md) -- [C++ client](client-libraries-cpp.md) -- [Node.js client](client-libraries-node.md) -- [WebSocket client](client-libraries-websocket.md) -- [C# client](client-libraries-dotnet.md) +- [Java client](client-libraries-java) +- [Go client](client-libraries-go) +- [Python client](client-libraries-python) +- [C++ client](client-libraries-cpp) +- [Node.js client](client-libraries-node) +- [WebSocket client](client-libraries-websocket) +- [C# client](client-libraries-dotnet) ## Feature matrix Pulsar client feature matrix for different languages is listed on [Client Features Matrix](https://github.com/apache/pulsar/wiki/Client-Features-Matrix) page. diff --git a/site2/website-next/docs/getting-started-concepts-and-architecture.md b/site2/website-next/docs/getting-started-concepts-and-architecture.md index dc4961960a280..c21c41839bd30 100644 --- a/site2/website-next/docs/getting-started-concepts-and-architecture.md +++ b/site2/website-next/docs/getting-started-concepts-and-architecture.md @@ -1,9 +1,13 @@ --- id: concepts-architecture title: Pulsar concepts and architecture -sidebar_label: Concepts and architecture +sidebar_label: "Concepts and architecture" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + diff --git a/site2/website-next/docs/getting-started-docker.md b/site2/website-next/docs/getting-started-docker.md index 16dfa8b0e8fa7..e802d6d606680 100644 --- a/site2/website-next/docs/getting-started-docker.md +++ b/site2/website-next/docs/getting-started-docker.md @@ -1,9 +1,13 @@ --- id: standalone-docker title: Set up a standalone Pulsar in Docker -sidebar_label: Run Pulsar in Docker +sidebar_label: "Run Pulsar in Docker" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + For local development and testing, you can run Pulsar in standalone mode on your own machine within a Docker container. If you have not installed Docker, download the [Community edition](https://www.docker.com/community-edition) and follow the instructions for your OS. @@ -13,7 +17,9 @@ If you have not installed Docker, download the [Community edition](https://www.d * For MacOS, Linux, and Windows: ```shell - $ docker run -it -p 6650:6650 -p 8080:8080 --mount source=pulsardata,target=/pulsar/data --mount source=pulsarconf,target=/pulsar/conf apachepulsar/pulsar:{{pulsar:version}} bin/pulsar standalone + + $ docker run -it -p 6650:6650 -p 8080:8080 --mount source=pulsardata,target=/pulsar/data --mount source=pulsarconf,target=/pulsar/conf apachepulsar/pulsar:@pulsar:version@ bin/pulsar standalone + ``` A few things to note about this command: @@ -24,33 +30,41 @@ time the container is restarted. For details on the volumes you can use `docker If you start Pulsar successfully, you will see `INFO`-level log messages like this: ``` + 08:18:30.970 [main] INFO org.apache.pulsar.broker.web.WebService - HTTP Service started at http://0.0.0.0:8080 ... 07:53:37.322 [main] INFO org.apache.pulsar.broker.PulsarService - messaging service is ready, bootstrap service port = 8080, broker url= pulsar://localhost:6650, cluster=standalone, configs=org.apache.pulsar.broker.ServiceConfiguration@98b63c1 ... + ``` -> **Tip** -> When you start a local standalone cluster, a `public/default` +:::tip + +When you start a local standalone cluster, a `public/default` + +::: + namespace is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). ## Use Pulsar in Docker -Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) -and [C++](client-libraries-cpp.md). If you're running a local standalone cluster, you can +Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python) +and [C++](client-libraries-cpp). If you're running a local standalone cluster, you can use one of these root URLs to interact with your cluster: * `pulsar://localhost:6650` * `http://localhost:8080` -The following example will guide you get started with Pulsar quickly by using the [Python client API](client-libraries-python.md) +The following example will guide you get started with Pulsar quickly by using the [Python client API](client-libraries-python) client API. Install the Pulsar Python client library directly from [PyPI](https://pypi.org/project/pulsar-client/): ```shell + $ pip install pulsar-client + ``` ### Consume a message @@ -58,6 +72,7 @@ $ pip install pulsar-client Create a consumer and subscribe to the topic: ```python + import pulsar client = pulsar.Client('pulsar://localhost:6650') @@ -70,6 +85,7 @@ while True: consumer.acknowledge(msg) client.close() + ``` ### Produce a message @@ -77,6 +93,7 @@ client.close() Now start a producer to send some test messages: ```python + import pulsar client = pulsar.Client('pulsar://localhost:6650') @@ -86,22 +103,26 @@ for i in range(10): producer.send(('hello-pulsar-%d' % i).encode('utf-8')) client.close() + ``` ## Get the topic statistics In Pulsar, you can use REST, Java, or command-line tools to control every aspect of the system. -For details on APIs, refer to [Admin API Overview](admin-api-overview.md). +For details on APIs, refer to [Admin API Overview](admin-api-overview). In the simplest example, you can use curl to probe the stats for a particular topic: ```shell + $ curl http://localhost:8080/admin/v2/persistent/public/default/my-topic/stats | python -m json.tool + ``` The output is something like this: ```json + { "msgRateIn": 0.0, "msgThroughputIn": 0.0, @@ -178,6 +199,7 @@ The output is something like this: ], "isDurable": true, "isReplicated": false, + "allowOutOfOrderDelivery": false, "consumersAfterMarkDeletePosition": {}, "nonContiguousDeletedMessagesRanges": 0, "nonContiguousDeletedMessagesRangesSerializedSize": 0, @@ -190,4 +212,6 @@ The output is something like this: "nonContiguousDeletedMessagesRanges": 0, "nonContiguousDeletedMessagesRangesSerializedSize": 0 } + ``` + diff --git a/site2/website-next/docs/getting-started-helm.md b/site2/website-next/docs/getting-started-helm.md index b147f55c26362..4f0b253c3e351 100644 --- a/site2/website-next/docs/getting-started-helm.md +++ b/site2/website-next/docs/getting-started-helm.md @@ -1,9 +1,13 @@ --- id: kubernetes-helm title: Get started in Kubernetes -sidebar_label: Run Pulsar in Kubernetes +sidebar_label: "Run Pulsar in Kubernetes" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + This section guides you through every step of installing and running Apache Pulsar with Helm on Kubernetes quickly, including the following sections: - Install the Apache Pulsar on Kubernetes using Helm @@ -12,7 +16,7 @@ This section guides you through every step of installing and running Apache Puls - Produce and consume messages using Pulsar clients - Monitor Apache Pulsar status with Prometheus and Grafana -For deploying a Pulsar cluster for production usage, read the documentation on [how to configure and install a Pulsar Helm chart](helm-deploy.md). +For deploying a Pulsar cluster for production usage, read the documentation on [how to configure and install a Pulsar Helm chart](helm-deploy). ## Prerequisite @@ -20,120 +24,151 @@ For deploying a Pulsar cluster for production usage, read the documentation on [ - kubectl 1.14.0+ - Helm 3.0+ -> **Tip** -> For the following steps, step 2 and step 3 are for **developers** and step 4 and step 5 are for **administrators**. +:::tip + +For the following steps, step 2 and step 3 are for **developers** and step 4 and step 5 are for **administrators**. + +::: ## Step 0: Prepare a Kubernetes cluster -Before installing a Pulsar Helm chart, you have to create a Kubernetes cluster. You can follow [the instructions](helm-prepare.md) to prepare a Kubernetes cluster. +Before installing a Pulsar Helm chart, you have to create a Kubernetes cluster. You can follow [the instructions](helm-prepare) to prepare a Kubernetes cluster. We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start guide. To prepare a Kubernetes cluster, follow these steps: 1. Create a Kubernetes cluster on Minikube. - ```bash - minikube start --memory=8192 --cpus=4 --kubernetes-version= - ``` + ```bash + + minikube start --memory=8192 --cpus=4 --kubernetes-version= + + ``` - The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. + The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. 2. Set `kubectl` to use Minikube. - ```bash - kubectl config use-context minikube - ``` + ```bash + + kubectl config use-context minikube + + ``` 3. To use the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) with the local Kubernetes cluster on Minikube, enter the command below: - ```bash - minikube dashboard - ``` - The command automatically triggers opening a webpage in your browser. + ```bash + + minikube dashboard + + ``` + + The command automatically triggers opening a webpage in your browser. ## Step 1: Install Pulsar Helm chart 1. Add Pulsar charts repo. - ```bash - helm repo add apache https://pulsar.apache.org/charts - ``` + ```bash + + helm repo add apache https://pulsar.apache.org/charts + + ``` - ```bash - helm repo update - ``` + ```bash + + helm repo update + + ``` 2. Clone the Pulsar Helm chart repository. - ```bash - git clone https://github.com/apache/pulsar-helm-chart - cd pulsar-helm-chart - ``` + ```bash + + git clone https://github.com/apache/pulsar-helm-chart + cd pulsar-helm-chart + + ``` 3. Run the script `prepare_helm_release.sh` to create secrets required for installing the Apache Pulsar Helm chart. The username `pulsar` and password `pulsar` are used for logging into the Grafana dashboard and Pulsar Manager. - ```bash - ./scripts/pulsar/prepare_helm_release.sh \ - -n pulsar \ - -k pulsar-mini \ - -c - ``` + ```bash + + ./scripts/pulsar/prepare_helm_release.sh \ + -n pulsar \ + -k pulsar-mini \ + -c + + ``` 4. Use the Pulsar Helm chart to install a Pulsar cluster to Kubernetes. - > **Note** - > You need to specify `--set initialize=true` when installing Pulsar the first time. This command installs and starts Apache Pulsar. + :::note - ```bash - helm install \ - --values examples/values-minikube.yaml \ - --set initialize=true \ - --namespace pulsar \ - pulsar-mini apache/pulsar - ``` + You need to specify `--set initialize=true` when installing Pulsar the first time. This command installs and starts Apache Pulsar. -5. Check the status of all pods. - - ```bash - kubectl get pods -n pulsar - ``` + ::: - If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. + ```bash + + helm install \ + --values examples/values-minikube.yaml \ + --set initialize=true \ + --namespace pulsar \ + pulsar-mini apache/pulsar + + ``` - **Output** +5. Check the status of all pods. - ```bash - NAME READY STATUS RESTARTS AGE - pulsar-mini-bookie-0 1/1 Running 0 9m27s - pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s - pulsar-mini-broker-0 1/1 Running 0 9m27s - pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s - pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s - pulsar-mini-proxy-0 1/1 Running 0 9m27s - pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s - pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s - pulsar-mini-toolset-0 1/1 Running 0 9m27s - pulsar-mini-zookeeper-0 1/1 Running 0 9m27s - ``` + ```bash + + kubectl get pods -n pulsar + + ``` + + If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. + + **Output** + + ```bash + + NAME READY STATUS RESTARTS AGE + pulsar-mini-bookie-0 1/1 Running 0 9m27s + pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s + pulsar-mini-broker-0 1/1 Running 0 9m27s + pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s + pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s + pulsar-mini-proxy-0 1/1 Running 0 9m27s + pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s + pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s + pulsar-mini-toolset-0 1/1 Running 0 9m27s + pulsar-mini-zookeeper-0 1/1 Running 0 9m27s + + ``` 6. Check the status of all services in the namespace `pulsar`. - ```bash - kubectl get services -n pulsar - ``` - - **Output** - - ```bash - NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE - pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m - pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m - pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m - pulsar-mini-prometheus ClusterIP None 9090/TCP 11m - pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m - pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m - pulsar-mini-toolset ClusterIP None 11m - pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m - ``` + ```bash + + kubectl get services -n pulsar + + ``` + + **Output** + + ```bash + + NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE + pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m + pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m + pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m + pulsar-mini-prometheus ClusterIP None 9090/TCP 11m + pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m + pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m + pulsar-mini-toolset ClusterIP None 11m + pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m + + ``` ## Step 2: Use pulsar-admin to create Pulsar tenants/namespaces/topics @@ -141,65 +176,85 @@ We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start 1. Enter the `toolset` container. - ```bash - kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash - ``` + ```bash + + kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash + + ``` 2. In the `toolset` container, create a tenant named `apache`. - ```bash - bin/pulsar-admin tenants create apache - ``` + ```bash + + bin/pulsar-admin tenants create apache + + ``` - Then you can list the tenants to see if the tenant is created successfully. + Then you can list the tenants to see if the tenant is created successfully. - ```bash - bin/pulsar-admin tenants list - ``` + ```bash + + bin/pulsar-admin tenants list + + ``` - You should see a similar output as below. The tenant `apache` has been successfully created. + You should see a similar output as below. The tenant `apache` has been successfully created. - ```bash - "apache" - "public" - "pulsar" - ``` + ```bash + + "apache" + "public" + "pulsar" + + ``` 3. In the `toolset` container, create a namespace named `pulsar` in the tenant `apache`. - ```bash - bin/pulsar-admin namespaces create apache/pulsar - ``` + ```bash + + bin/pulsar-admin namespaces create apache/pulsar + + ``` - Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. + Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. - ```bash - bin/pulsar-admin namespaces list apache - ``` + ```bash + + bin/pulsar-admin namespaces list apache + + ``` - You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. + You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. - ```bash - "apache/pulsar" - ``` + ```bash + + "apache/pulsar" + + ``` 4. In the `toolset` container, create a topic `test-topic` with `4` partitions in the namespace `apache/pulsar`. - ```bash - bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 - ``` + ```bash + + bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 + + ``` 5. In the `toolset` container, list all the partitioned topics in the namespace `apache/pulsar`. - ```bash - bin/pulsar-admin topics list-partitioned-topics apache/pulsar - ``` + ```bash + + bin/pulsar-admin topics list-partitioned-topics apache/pulsar + + ``` - Then you can see all the partitioned topics in the namespace `apache/pulsar`. + Then you can see all the partitioned topics in the namespace `apache/pulsar`. - ```bash - "persistent://apache/pulsar/test-topic" - ``` + ```bash + + "persistent://apache/pulsar/test-topic" + + ``` ## Step 3: Use Pulsar client to produce and consume messages @@ -208,13 +263,17 @@ You can use the Pulsar client to create producers and consumers to produce and c By default, the Pulsar Helm chart exposes the Pulsar cluster through a Kubernetes `LoadBalancer`. In Minikube, you can use the following command to check the proxy service. ```bash + kubectl get services -n pulsar | grep pulsar-mini-proxy + ``` You will see a similar output as below. ```bash + pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 28m + ``` This output tells what are the node ports that Pulsar cluster's binary port and HTTP port are mapped to. The port after `80:` is the HTTP port while the port after `6650:` is the binary port. @@ -222,12 +281,15 @@ This output tells what are the node ports that Pulsar cluster's binary port and Then you can find the IP address and exposed ports of your Minikube server by running the following command. ```bash + minikube service pulsar-mini-proxy -n pulsar + ``` **Output** ```bash + |-----------|-------------------|-------------|-------------------------| | NAMESPACE | NAME | TARGET PORT | URL | |-----------|-------------------|-------------|-------------------------| @@ -241,12 +303,16 @@ minikube service pulsar-mini-proxy -n pulsar | pulsar | pulsar-mini-proxy | | http://127.0.0.1:61853 | | | | | http://127.0.0.1:61854 | |-----------|-------------------|-------------|------------------------| + ``` At this point, you can get the service URLs to connect to your Pulsar client. Here are URL examples: + ``` + webServiceUrl=http://127.0.0.1:61853/ brokerServiceUrl=pulsar://127.0.0.1:61854/ + ``` Then you can proceed with the following steps: @@ -255,95 +321,109 @@ Then you can proceed with the following steps: 2. Decompress the tarball based on your download file. - ```bash - tar -xf .tar.gz - ``` + ```bash + + tar -xf .tar.gz + + ``` 3. Expose `PULSAR_HOME`. - (1) Enter the directory of the decompressed download file. + (1) Enter the directory of the decompressed download file. - (2) Expose `PULSAR_HOME` as the environment variable. + (2) Expose `PULSAR_HOME` as the environment variable. - ```bash - export PULSAR_HOME=$(pwd) - ``` + ```bash + + export PULSAR_HOME=$(pwd) + + ``` 4. Configure the Pulsar client. - In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. + In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. 5. Create a subscription to consume messages from `apache/pulsar/test-topic`. - ```bash - bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 - ``` + ```bash + + bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 + + ``` 6. Open a new terminal. In the new terminal, create a producer and send 10 messages to the `test-topic` topic. - ```bash - bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 - ``` + ```bash + + bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 + + ``` 7. Verify the results. - - From the producer side - - **Output** - - The messages have been produced successfully. - - ```bash - 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced - ``` - - - From the consumer side - - **Output** - - At the same time, you can receive the messages as below. - - ```bash - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ``` + - From the producer side + + **Output** + + The messages have been produced successfully. + + ```bash + + 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced + + ``` + + - From the consumer side + + **Output** + + At the same time, you can receive the messages as below. + + ```bash + + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + + ``` ## Step 4: Use Pulsar Manager to manage the cluster -[Pulsar Manager](administration-pulsar-manager.md) is a web-based GUI management tool for managing and monitoring Pulsar. +[Pulsar Manager](administration-pulsar-manager) is a web-based GUI management tool for managing and monitoring Pulsar. 1. By default, the `Pulsar Manager` is exposed as a separate `LoadBalancer`. You can open the Pulsar Manager UI using the following command: - ```bash - minikube service -n pulsar pulsar-mini-pulsar-manager - ``` + ```bash + + minikube service -n pulsar pulsar-mini-pulsar-manager + + ``` 2. The Pulsar Manager UI will be open in your browser. You can use the username `pulsar` and password `pulsar` to log into Pulsar Manager. 3. In Pulsar Manager UI, you can create an environment. - - Click `New Environment` button in the top-left corner. - - Type `pulsar-mini` for the field `Environment Name` in the popup window. - - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. - - Click `Confirm` button in the popup window. + - Click `New Environment` button in the top-left corner. + - Type `pulsar-mini` for the field `Environment Name` in the popup window. + - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. + - Click `Confirm` button in the popup window. 4. After successfully created an environment, you are redirected to the `tenants` page of that environment. Then you can create `tenants`, `namespaces` and `topics` using the Pulsar Manager. @@ -353,9 +433,11 @@ Grafana is an open-source visualization tool, which can be used for visualizing 1. By default, the Grafana is exposed as a separate `LoadBalancer`. You can open the Grafana UI using the following command: - ```bash - minikube service pulsar-mini-grafana -n pulsar - ``` + ```bash + + minikube service pulsar-mini-grafana -n pulsar + + ``` 2. The Grafana UI is open in your browser. You can use the username `pulsar` and password `pulsar` to log into the Grafana Dashboard. diff --git a/site2/website-next/docs/getting-started-pulsar.md b/site2/website-next/docs/getting-started-pulsar.md index 6ba815e67971f..a17268b5b0c72 100644 --- a/site2/website-next/docs/getting-started-pulsar.md +++ b/site2/website-next/docs/getting-started-pulsar.md @@ -1,16 +1,20 @@ --- id: pulsar-2.0 title: Pulsar 2.0 -sidebar_label: Pulsar 2.0 +sidebar_label: "Pulsar 2.0" --- -Pulsar 2.0 is a major new release for Pulsar that brings some bold changes to the platform, including [simplified topic names](#topic-names), the addition of the [Pulsar Functions](functions-overview.md) feature, some terminology changes, and more. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar 2.0 is a major new release for Pulsar that brings some bold changes to the platform, including [simplified topic names](#topic-names), the addition of the [Pulsar Functions](functions-overview) feature, some terminology changes, and more. ## New features in Pulsar 2.0 Feature | Description :-------|:----------- -[Pulsar Functions](functions-overview.md) | A lightweight compute option for Pulsar +[Pulsar Functions](functions-overview) | A lightweight compute option for Pulsar ## Major changes @@ -25,8 +29,11 @@ Previously, Pulsar had a concept of properties. A property is essentially the ex Prior to version 2.0, *all* Pulsar topics had the following form: ```http + {persistent|non-persistent}://property/cluster/namespace/topic + ``` + Two important changes have been made in Pulsar 2.0: * There is no longer a [cluster component](#no-cluster) @@ -39,7 +46,9 @@ Two important changes have been made in Pulsar 2.0: The cluster component has been removed from topic names. Thus, all topic names now have the following form: ```http + {persistent|non-persistent}://tenant/namespace/topic + ``` > Existing topics that use the legacy name format will continue to work without any change, and there are no plans to change that. diff --git a/site2/website-next/docs/getting-started-standalone.md b/site2/website-next/docs/getting-started-standalone.md index dcab4f4741a4a..f07e9a12415e7 100644 --- a/site2/website-next/docs/getting-started-standalone.md +++ b/site2/website-next/docs/getting-started-standalone.md @@ -1,14 +1,18 @@ --- +slug: / id: standalone title: Set up a standalone Pulsar locally -sidebar_label: Run Pulsar locally -slug: / +sidebar_label: "Run Pulsar locally" --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + For local development and testing, you can run Pulsar in standalone mode on your machine. The standalone mode includes a Pulsar broker, the necessary ZooKeeper and BookKeeper components running inside of a single Java Virtual Machine (JVM) process. > **Pulsar in production?** -> If you're looking to run a full production Pulsar installation, see the [Deploying a Pulsar instance](deploy-bare-metal.md) guide. +> If you're looking to run a full production Pulsar installation, see the [Deploying a Pulsar instance](deploy-bare-metal) guide. ## Install Pulsar standalone @@ -18,17 +22,23 @@ This tutorial guides you through every step of installing Pulsar locally. Currently, Pulsar is available for 64-bit **macOS**, **Linux**, and **Windows**. To use Pulsar, you need to install 64-bit JRE/JDK 8 or later versions. -> **Tip** -> By default, Pulsar allocates 2G JVM heap memory to start. It can be changed in `conf/pulsar_env.sh` file under `PULSAR_MEM`. This is extra options passed into JVM. +:::tip + +By default, Pulsar allocates 2G JVM heap memory to start. It can be changed in `conf/pulsar_env.sh` file under `PULSAR_MEM`. This is extra options passed into JVM. + +::: + +:::note -> **Note** -> Broker is only supported on 64-bit JVM. +Broker is only supported on 64-bit JVM. + +::: ### Install Pulsar using binary release To get started with Pulsar, download a binary tarball release in one of the following ways: -* download from the Apache mirror (Pulsar {{pulsar:version}} binary release) +* download from the Apache mirror (Pulsar @pulsar:version@ binary release) * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -37,14 +47,18 @@ To get started with Pulsar, download a binary tarball release in one of the foll * use [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:binary_release_url + ``` After you download the tarball, untar it and use the `cd` command to navigate to the resulting directory: ```bash -$ tar xvfz apache-pulsar-{{pulsar:version}}-bin.tar.gz -$ cd apache-pulsar-{{pulsar:version}} + +$ tar xvfz apache-pulsar-@pulsar:version@-bin.tar.gz +$ cd apache-pulsar-@pulsar:version@ + ``` #### What your package contains @@ -55,7 +69,7 @@ Directory | Contains :---------|:-------- `bin` | Pulsar's command-line tools, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/). `conf` | Configuration files for Pulsar, including [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more. -`examples` | A Java JAR file containing [Pulsar Functions](functions-overview.md) example. +`examples` | A Java JAR file containing [Pulsar Functions](functions-overview) example. `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files used by Pulsar. `licenses` | License files, in the`.txt` form, for various components of the Pulsar [codebase](https://github.com/apache/pulsar). @@ -64,23 +78,24 @@ These directories are created once you begin running Pulsar. Directory | Contains :---------|:-------- `data` | The data storage directory used by ZooKeeper and BookKeeper. -`instances` | Artifacts created for [Pulsar Functions](functions-overview.md). +`instances` | Artifacts created for [Pulsar Functions](functions-overview). `logs` | Logs created by the installation. -> **Tip** -> If you want to use builtin connectors and tiered storage offloaders, you can install them according to the following instructions: -> -> * [Install builtin connectors (optional)](#install-builtin-connectors-optional) -> * [Install tiered storage offloaders (optional)](#install-tiered-storage-offloaders-optional) -> -> Otherwise, skip this step and perform the next step [Start Pulsar standalone](#start-pulsar-standalone). Pulsar can be successfully installed without installing bulitin connectors and tiered storage offloaders. +:::tip + +If you want to use builtin connectors and tiered storage offloaders, you can install them according to the following instructions: +* [Install builtin connectors (optional)](#install-builtin-connectors-optional) +* [Install tiered storage offloaders (optional)](#install-tiered-storage-offloaders-optional) +Otherwise, skip this step and perform the next step [Start Pulsar standalone](#start-pulsar-standalone). Pulsar can be successfully installed without installing bulitin connectors and tiered storage offloaders. + +::: ### Install builtin connectors (optional) Since `2.1.0-incubating` release, Pulsar releases a separate binary distribution, containing all the `builtin` connectors. To enable those `builtin` connectors, you can download the connectors tarball release in one of the following ways: -* download from the Apache mirror Pulsar IO Connectors {{pulsar:version}} release +* download from the Apache mirror Pulsar IO Connectors @pulsar:version@ release * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -89,35 +104,44 @@ To enable those `builtin` connectors, you can download the connectors tarball re * use [wget](https://www.gnu.org/software/wget): ```shell - $ wget pulsar:connector_release_url/{connector}-{{pulsar:version}}.nar + + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar + ``` After you download the nar file, copy the file to the `connectors` directory in the pulsar directory. -For example, if you download the `pulsar-io-aerospike-{{pulsar:version}}.nar` connector file, enter the following commands: +For example, if you download the `pulsar-io-aerospike-@pulsar:version@.nar` connector file, enter the following commands: ```bash + $ mkdir connectors -$ mv pulsar-io-aerospike-{{pulsar:version}}.nar connectors +$ mv pulsar-io-aerospike-@pulsar:version@.nar connectors $ ls connectors -pulsar-io-aerospike-{{pulsar:version}}.nar +pulsar-io-aerospike-@pulsar:version@.nar ... + ``` -> **Note** -> * If you are running Pulsar in a bare metal cluster, make sure `connectors` tarball is unzipped in every pulsar directory of the broker (or in every pulsar directory of function-worker if you are running a separate worker cluster for Pulsar Functions). -> -> * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos.md)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). +:::note + +* If you are running Pulsar in a bare metal cluster, make sure `connectors` tarball is unzipped in every pulsar directory of the broker (or in every pulsar directory of function-worker if you are running a separate worker cluster for Pulsar Functions). +* If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). + +::: ### Install tiered storage offloaders (optional) -> **Tip** -> - Since `2.2.0` release, Pulsar releases a separate binary distribution, containing the tiered storage offloaders. -> - To enable tiered storage feature, follow the instructions below; otherwise skip this section. +:::tip + +- Since `2.2.0` release, Pulsar releases a separate binary distribution, containing the tiered storage offloaders. +- To enable tiered storage feature, follow the instructions below; otherwise skip this section. -To get started with [tiered storage offloaders](concepts-tiered-storage.md), you need to download the offloaders tarball release on every broker node in one of the following ways: +::: -* download from the Apache mirror Pulsar Tiered Storage Offloaders {{pulsar:version}} release +To get started with [tiered storage offloaders](concepts-tiered-storage), you need to download the offloaders tarball release on every broker node in one of the following ways: + +* download from the Apache mirror Pulsar Tiered Storage Offloaders @pulsar:version@ release * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -126,52 +150,66 @@ To get started with [tiered storage offloaders](concepts-tiered-storage.md), you * use [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:offloader_release_url + ``` After you download the tarball, untar the offloaders package and copy the offloaders as `offloaders` in the pulsar directory: ```bash -$ tar xvfz apache-pulsar-offloaders-{{pulsar:version}}-bin.tar.gz -// you will find a directory named `apache-pulsar-offloaders-{{pulsar:version}}` in the pulsar directory +$ tar xvfz apache-pulsar-offloaders-@pulsar:version@-bin.tar.gz + +// you will find a directory named `apache-pulsar-offloaders-@pulsar:version@` in the pulsar directory // then copy the offloaders -$ mv apache-pulsar-offloaders-{{pulsar:version}}/offloaders offloaders +$ mv apache-pulsar-offloaders-@pulsar:version@/offloaders offloaders $ ls offloaders -tiered-storage-jcloud-{{pulsar:version}}.nar +tiered-storage-jcloud-@pulsar:version@.nar + ``` -For more information on how to configure tiered storage, see [Tiered storage cookbook](cookbooks-tiered-storage.md). +For more information on how to configure tiered storage, see [Tiered storage cookbook](cookbooks-tiered-storage). -> **Note** -> * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's pulsar directory. -> -> * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos.md)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. +:::note + +* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's pulsar directory. +* If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + +::: ## Start Pulsar standalone Once you have an up-to-date local copy of the release, you can start a local cluster using the [`pulsar`](reference-cli-tools.md#pulsar) command, which is stored in the `bin` directory, and specifying that you want to start Pulsar in standalone mode. ```bash + $ bin/pulsar standalone + ``` If you have started Pulsar successfully, you will see `INFO`-level log messages like this: ```bash + 21:59:29.327 [DLM-/stream/storage-OrderedScheduler-3-0] INFO org.apache.bookkeeper.stream.storage.impl.sc.StorageContainerImpl - Successfully started storage container (0). 21:59:34.576 [main] INFO org.apache.pulsar.broker.authentication.AuthenticationService - Authentication is disabled 21:59:34.576 [main] INFO org.apache.pulsar.websocket.WebSocketService - Pulsar WebSocket Service started + ``` -> **Tip** -> * The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. +:::tip + +* The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. + +::: + You can also run the service as a background process using the `pulsar-daemon start standalone` command. For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). > -> * By default, there is no encryption, authentication, or authorization configured. Apache Pulsar can be accessed from remote server without any authorization. Please do check [Security Overview](security-overview.md) document to secure your deployment. +> * By default, there is no encryption, authentication, or authorization configured. Apache Pulsar can be accessed from remote server without any authorization. Please do check [Security Overview](security-overview) document to secure your deployment. > > * When you start a local standalone cluster, a `public/default` [namespace](concepts-messaging.md#namespaces) is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). @@ -184,37 +222,51 @@ Pulsar provides a CLI tool called [`pulsar-client`](reference-cli-tools.md#pulsa The following command consumes a message with the subscription name `first-subscription` to the `my-topic` topic: ```bash + $ bin/pulsar-client consume my-topic -s "first-subscription" + ``` If the message has been successfully consumed, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 22:17:16.781 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 1 messages successfully consumed + ``` -> **Tip** -> As you have noticed that we do not explicitly create the `my-topic` topic, to which we consume the message. When you consume a message to a topic that does not yet exist, Pulsar creates that topic for you automatically. Producing a message to a topic that does not exist will automatically create that topic for you as well. +:::tip + +As you have noticed that we do not explicitly create the `my-topic` topic, to which we consume the message. When you consume a message to a topic that does not yet exist, Pulsar creates that topic for you automatically. Producing a message to a topic that does not exist will automatically create that topic for you as well. + +::: ### Produce a message The following command produces a message saying `hello-pulsar` to the `my-topic` topic: ```bash + $ bin/pulsar-client produce my-topic --messages "hello-pulsar" + ``` If the message has been successfully published to the topic, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 22:21:08.693 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 1 messages successfully produced + ``` ## Stop Pulsar standalone Press `Ctrl+C` to stop a local standalone Pulsar. -> **Tip** -> If the service runs as a background process using the `pulsar-daemon start standalone` command, then use the `pulsar-daemon stop standalone` command to stop the service. -> -> For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). +:::tip + +If the service runs as a background process using the `pulsar-daemon start standalone` command, then use the `pulsar-daemon stop standalone` command to stop the service. +For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). + +::: + diff --git a/site2/website-next/docs/helm-deploy.md b/site2/website-next/docs/helm-deploy.md index 1b876915b6dc9..50bb0775f5ecb 100644 --- a/site2/website-next/docs/helm-deploy.md +++ b/site2/website-next/docs/helm-deploy.md @@ -1,7 +1,7 @@ --- id: helm-deploy title: Deploy Pulsar cluster using Helm -sidebar_label: Deployment +sidebar_label: "Deployment" --- import Tabs from '@theme/Tabs'; @@ -20,25 +20,33 @@ In each section, collect the options that are combined to use with the `helm ins By default, the Pulsar Helm chart is installed to a namespace called `pulsar`. ```yaml + namespace: pulsar + ``` To install the Pulsar Helm chart into a different Kubernetes namespace, you can include this option in the `helm install` command. ```bash + --set namespace= + ``` By default, the Pulsar Helm chart doesn't create the namespace. ```yaml + namespaceCreate: false + ``` To use the Pulsar Helm chart to create the Kubernetes namespace automatically, you can include this option in the `helm install` command. ```bash + --set namespaceCreate=true + ``` ### Persistence @@ -46,22 +54,25 @@ To use the Pulsar Helm chart to create the Kubernetes namespace automatically, y By default, the Pulsar Helm chart creates Volume Claims with the expectation that a dynamic provisioner creates the underlying Persistent Volumes. ```yaml + volumes: persistence: true # configure the components to use local persistent volume # the local provisioner should be installed prior to enable local persistent volume local_storage: false + ``` To use local persistent volumes as the persistent storage for Helm release, you can install the [local storage provisioner](#install-local-storage-provisioner) and include the following option in the `helm install` command. ```bash + --set volumes.local_storage=true + ``` :::note - Before installing the production instance of Pulsar, ensure to plan the storage settings to avoid extra storage migration work. Because after initial installation, you must edit Kubernetes objects manually if you want to change storage settings. ::: @@ -69,7 +80,9 @@ Before installing the production instance of Pulsar, ensure to plan the storage The Pulsar Helm chart is designed for production use. To use the Pulsar Helm chart in a development environment (such as Minikube), you can disable persistence by including this option in your `helm install` command. ```bash + --set volumes.persistence=false + ``` ### Affinity @@ -77,14 +90,18 @@ The Pulsar Helm chart is designed for production use. To use the Pulsar Helm cha By default, `anti-affinity` is enabled to ensure pods of the same component can run on different nodes. ```yaml + affinity: anti_affinity: true + ``` To use the Pulsar Helm chart in a development environment (such as Minikue), you can disable `anti-affinity` by including this option in your `helm install` command. ```bash + --set affinity.anti_affinity=false + ``` ### Components @@ -94,6 +111,7 @@ The Pulsar Helm chart is designed for production usage. It deploys a production- You can customize the components to be deployed by turning on/off individual components. ```yaml + ## Components ## ## Control what components of Apache Pulsar to deploy for the cluster @@ -123,6 +141,7 @@ monitoring: prometheus: true # monitoring - grafana grafana: true + ``` ### Docker images @@ -130,6 +149,7 @@ monitoring: The Pulsar Helm chart is designed to enable controlled upgrades. So it can configure independent image versions for components. You can customize the images by setting individual component. ```yaml + ## Images ## ## Control what images to use for each component @@ -170,6 +190,7 @@ images: tag: v0.1.0 pullPolicy: IfNotPresent hasCommand: false + ``` ### TLS @@ -181,15 +202,19 @@ The Pulsar Helm chart can be configured to enable TLS (Transport Layer Security) To use the `cert-manager` to provision the TLS certificates, you have to install the [cert-manager](#install-cert-manager) before installing the Pulsar Helm chart. After successfully installing the cert-manager, you can set `certs.internal_issuer.enabled` to `true`. Therefore, the Pulsar Helm chart can use the `cert-manager` to generate `selfsigning` TLS certificates for the configured components. ```yaml + certs: internal_issuer: enabled: false component: internal-cert-issuer type: selfsigning + ``` + You can also customize the generated TLS certificates by configuring the fields as the following. ```yaml + tls: # common settings for generating certs common: @@ -202,6 +227,7 @@ tls: keySize: 4096 keyAlgorithm: rsa keyEncoding: pkcs8 + ``` #### Enable TLS @@ -209,13 +235,16 @@ tls: After installing the `cert-manager`, you can set `tls.enabled` to `true` to enable TLS encryption for the entire cluster. ```yaml + tls: enabled: false + ``` You can also configure whether to enable TLS encryption for individual component. ```yaml + tls: # settings for generating certs for proxy proxy: @@ -239,6 +268,7 @@ tls: # settings for generating certs for toolset toolset: cert_name: tls-toolset + ``` ### Authentication @@ -247,6 +277,7 @@ By default, authentication is disabled. You can set `auth.authentication.enabled Currently, the Pulsar Helm chart only supports JWT authentication provider. You can set `auth.authentication.provider` to `jwt` to use the JWT authentication provider. ```yaml + # Enable or disable broker authentication and authorization. auth: authentication: @@ -264,12 +295,15 @@ auth: proxy: "proxy-admin" # pulsar-admin client to broker/proxy communication client: "admin" + ``` To enable authentication, you can run [prepare helm release](#prepare-the-helm-release) to generate token secret keys and tokens for three super users specified in the `auth.superUsers` field. The generated token keys and super user tokens are uploaded and stored as Kubernetes secrets prefixed with `-token-`. You can use the following command to find those secrets. ```bash + kubectl get secrets -n + ``` ### Authorization @@ -277,15 +311,19 @@ kubectl get secrets -n By default, authorization is disabled. Authorization can be enabled only when authentication is enabled. ```yaml + auth: authorization: enabled: false + ``` To enable authorization, you can include this option in the `helm install` command. ```bash + --set auth.authorization.enabled=true + ``` ### CPU and RAM resource requirements @@ -303,9 +341,11 @@ To use local persistent volumes as the persistent storage, you need to install a One of the easiest way to get started is to use the local storage provisioner provided along with the Pulsar Helm chart. ``` + helm repo add streamnative https://charts.streamnative.io helm repo update helm install pulsar-storage-provisioner streamnative/local-storage-provisioner + ``` ### Install cert-manager @@ -317,9 +357,11 @@ For details about how to install the cert-manager, follow the [official instruct Alternatively, we provide a bash script [install-cert-manager.sh](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/cert-manager/install-cert-manager.sh) to install a cert-manager release to the namespace `cert-manager`. ```bash + git clone https://github.com/apache/pulsar-helm-chart cd pulsar-helm-chart ./scripts/cert-manager/install-cert-manager.sh + ``` ## Prepare Helm release @@ -327,18 +369,20 @@ cd pulsar-helm-chart Once you have install all the dependent charts and collected all of your configuration options, you can run [prepare_helm_release.sh](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/prepare_helm_release.sh) to prepare the Helm release. ```bash + git clone https://github.com/apache/pulsar-helm-chart cd pulsar-helm-chart ./scripts/pulsar/prepare_helm_release.sh -n -k + ``` The `prepare_helm_release` creates the following resources: - A Kubernetes namespace for installing the Pulsar release - JWT secret keys and tokens for three super users: `broker-admin`, `proxy-admin`, and `admin`. By default, it generates an asymmetric pubic/private key pair. You can choose to generate a symmetric secret key by specifying `--symmetric`. - - `proxy-admin` role is used for proxies to communicate to brokers. - - `broker-admin` role is used for inter-broker communications. - - `admin` role is used by the admin tools. + - `proxy-admin` role is used for proxies to communicate to brokers. + - `broker-admin` role is used for inter-broker communications. + - `admin` role is used by the admin tools. ## Deploy Pulsar cluster using Helm @@ -351,17 +395,18 @@ Once you have finished the following three things, you can install a Helm releas In this example, the Helm release is named `pulsar`. ```bash + helm repo add apache https://pulsar.apache.org/charts helm repo update helm install pulsar apache/pulsar \ --timeout 10m \ --set initialize=true \ --set [your configuration options] + ``` :::note - For the first deployment, add `--set initialize=true` option to initialize bookie and Pulsar cluster metadata. ::: @@ -385,5 +430,8 @@ The default values will create a `ClusterIP` for the following resources, which To find the IP addresses of those components, run the following command: ```bash + kubectl get service -n + ``` + diff --git a/site2/website-next/docs/helm-install.md b/site2/website-next/docs/helm-install.md index 5e391084c1dad..8c1328117a455 100644 --- a/site2/website-next/docs/helm-install.md +++ b/site2/website-next/docs/helm-install.md @@ -1,7 +1,7 @@ --- id: helm-install title: Install Apache Pulsar using Helm -sidebar_label: Install +sidebar_label: "Install " --- import Tabs from '@theme/Tabs'; @@ -24,7 +24,7 @@ Before deploying Pulsar, you need to prepare your environment. ### Tools -Install [`helm`](helm-tools.md) and [`kubectl`](helm-tools.md) on your computer. +Install [`helm`](helm-tools.md) and [`kubectl`](helm-tools) on your computer. ## Cloud cluster preparation @@ -34,8 +34,8 @@ To create and connect to the Kubernetes cluster, follow the instructions: ## Pulsar deployment -Once the environment is set up and configuration is generated, you can now proceed to the [deployment of Pulsar](helm-deploy.md). +Once the environment is set up and configuration is generated, you can now proceed to the [deployment of Pulsar](helm-deploy). ## Pulsar upgrade -To upgrade an existing Kubernetes installation, follow the [upgrade documentation](helm-upgrade.md). +To upgrade an existing Kubernetes installation, follow the [upgrade documentation](helm-upgrade). diff --git a/site2/website-next/docs/helm-overview.md b/site2/website-next/docs/helm-overview.md index 8bd16d3857031..0b52e06b6b7d5 100644 --- a/site2/website-next/docs/helm-overview.md +++ b/site2/website-next/docs/helm-overview.md @@ -1,7 +1,7 @@ --- id: helm-overview title: Apache Pulsar Helm Chart -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -17,47 +17,47 @@ The Apache Pulsar Helm chart provides one of the most convenient ways to operate The Apache Pulsar Helm chart contains all components to support the features and functions that Pulsar delivers. You can install and configure these components separately. - Pulsar core components: - - ZooKeeper - - Bookies - - Brokers - - Function workers - - Proxies + - ZooKeeper + - Bookies + - Brokers + - Function workers + - Proxies - Control center: - - Pulsar Manager - - Prometheus - - Grafana + - Pulsar Manager + - Prometheus + - Grafana Moreover, Helm chart supports: - Security - - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) - - self-signed - - [Let's Encrypt](https://letsencrypt.org/) - - TLS Encryption - - Proxy - - Broker - - Toolset - - Bookie - - ZooKeeper - - Authentication - - JWT - - Authorization + - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) + - self-signed + - [Let's Encrypt](https://letsencrypt.org/) + - TLS Encryption + - Proxy + - Broker + - Toolset + - Bookie + - ZooKeeper + - Authentication + - JWT + - Authorization - Storage - - Non-persistence storage - - Persistent volume - - Local persistent volumes + - Non-persistence storage + - Persistent volume + - Local persistent volumes - Functions - - Kubernetes Runtime - - Process Runtime - - Thread Runtime + - Kubernetes Runtime + - Process Runtime + - Thread Runtime - Operations - - Independent image versions for all components, enabling controlled upgrades + - Independent image versions for all components, enabling controlled upgrades ## Quick start -To run with Apache Pulsar Helm chart as fast as possible in a **non-production** use case, we provide a [quick start guide](getting-started-helm.md) for Proof of Concept (PoC) deployments. +To run with Apache Pulsar Helm chart as fast as possible in a **non-production** use case, we provide a [quick start guide](getting-started-helm) for Proof of Concept (PoC) deployments. -This guide walks you through deploying Apache Pulsar Helm chart with default values and features, but it is *not* suitable for deployments in production-ready environments. To deploy the charts in production under sustained load, you can follow the complete [Installation Guide](helm-install.md). +This guide walks you through deploying Apache Pulsar Helm chart with default values and features, but it is *not* suitable for deployments in production-ready environments. To deploy the charts in production under sustained load, you can follow the complete [Installation Guide](helm-install). ## Troubleshooting @@ -67,36 +67,40 @@ Although we have done our best to make these charts as seamless as possible, tro The Apache Pulsar Helm chart contains all required dependencies. -If you deploy a PoC for testing, we strongly suggest you follow this [Quick Start Guide](getting-started-helm.md) for your first iteration. +If you deploy a PoC for testing, we strongly suggest you follow this [Quick Start Guide](getting-started-helm) for your first iteration. -1. [Preparation](helm-prepare.md) -2. [Deployment](helm-deploy.md) +1. [Preparation](helm-prepare) +2. [Deployment](helm-deploy) ## Upgrading Once the Apache Pulsar Helm chart is installed, you can use `helm upgrade` command to configure and update it. ```bash + helm repo add apache https://pulsar.apache.org/charts helm repo update helm get values > pulsar.yaml helm upgrade apache/pulsar -f pulsar.yaml + ``` -For more detailed information, see [Upgrading](helm-upgrade.md). +For more detailed information, see [Upgrading](helm-upgrade). ## Uninstallation To uninstall the Apache Pulsar Helm chart, run the following command: ```bash + helm delete + ``` For the purposes of continuity, some Kubernetes objects in these charts cannot be removed by `helm delete` command. It is recommended to *consciously* remove these items, as they affect re-deployment. * PVCs for stateful data: remove these items. - - ZooKeeper: This is your metadata. - - BookKeeper: This is your data. - - Prometheus: This is your metrics data, which can be safely removed. + - ZooKeeper: This is your metadata. + - BookKeeper: This is your data. + - Prometheus: This is your metrics data, which can be safely removed. * Secrets: if the secrets are generated by the [prepare release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/prepare_helm_release.sh), they contain secret keys and tokens. You can use the [cleanup release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/cleanup_helm_release.sh) to remove these secrets and tokens as needed. diff --git a/site2/website-next/docs/helm-prepare.md b/site2/website-next/docs/helm-prepare.md index 14137a660729d..7ede02c20187c 100644 --- a/site2/website-next/docs/helm-prepare.md +++ b/site2/website-next/docs/helm-prepare.md @@ -1,7 +1,7 @@ --- id: helm-prepare title: Prepare Kubernetes resources -sidebar_label: Prepare +sidebar_label: "Prepare" --- import Tabs from '@theme/Tabs'; @@ -13,7 +13,7 @@ For a fully functional Pulsar cluster, you need a few resources before deploying - [Google Kubernetes Engine](#google-kubernetes-engine) - [Manual cluster creation](#manual-cluster-creation) - [Scripted cluster creation](#scripted-cluster-creation) - - [Create cluster with local SSDs](#create-cluster-with-local-ssds) + - [Create cluster with local SSDs](#create-cluster-with-local-ssds) ## Google Kubernetes Engine @@ -58,13 +58,17 @@ The following table describes all variables. Run the script, by passing in your desired parameters. It can work with the default parameters except for `PROJECT` which is required: ```bash + PROJECT= scripts/pulsar/gke_bootstrap_script.sh up + ``` The script can also be used to clean up the created GKE resources. ```bash + PROJECT= scripts/pulsar/gke_bootstrap_script.sh down + ``` #### Create cluster with local SSDs @@ -72,5 +76,8 @@ PROJECT= scripts/pulsar/gke_bootstrap_script.sh down To install a Pulsar Helm chart using local persistent volumes, you need to create a GKE cluster with local SSDs. You can do so Specifying the `USE_LOCAL_SSD` to be `true` in the following command to create a Pulsar cluster with local SSDs. ``` + PROJECT= USE_LOCAL_SSD=true LOCAL_SSD_COUNT= scripts/pulsar/gke_bootstrap_script.sh up + ``` + diff --git a/site2/website-next/docs/helm-tools.md b/site2/website-next/docs/helm-tools.md index a36c2fc5517da..e5173c5b035ca 100644 --- a/site2/website-next/docs/helm-tools.md +++ b/site2/website-next/docs/helm-tools.md @@ -1,7 +1,7 @@ --- id: helm-tools title: Required tools for deploying Pulsar Helm Chart -sidebar_label: Required Tools +sidebar_label: "Required Tools" --- import Tabs from '@theme/Tabs'; @@ -28,7 +28,7 @@ You can get Helm from the project's [releases page](https://github.com/helm/helm ### Next steps -Once kubectl and Helm are configured, you can configure your [Kubernetes cluster](helm-prepare.md). +Once kubectl and Helm are configured, you can configure your [Kubernetes cluster](helm-prepare). ## Additional information diff --git a/site2/website-next/docs/helm-upgrade.md b/site2/website-next/docs/helm-upgrade.md index 402efb07264f0..8c92cbc2db936 100644 --- a/site2/website-next/docs/helm-upgrade.md +++ b/site2/website-next/docs/helm-upgrade.md @@ -1,7 +1,7 @@ --- id: helm-upgrade title: Upgrade Pulsar Helm release -sidebar_label: Upgrade +sidebar_label: "Upgrade" --- import Tabs from '@theme/Tabs'; @@ -14,10 +14,10 @@ We also recommend that you need to provide all values using the `helm upgrade -- :::note - You can retrieve your previous `--set` arguments cleanly, with `helm get values `. If you direct this into a file (`helm get values > pulsar.yml`), you can safely ::: + pass this file through `-f`. Thus `helm upgrade apache/pulsar -f pulsar.yaml`. This safely replaces the behavior of `--reuse-values`. ## Steps @@ -25,16 +25,24 @@ pass this file through `-f`. Thus `helm upgrade apache/pulsar -f To upgrade Apache Pulsar to a newer version, follow these steps: 1. Check the change log for the specific version you would like to upgrade to. -2. Go through [deployment documentation](helm-deploy.md) step by step. +2. Go through [deployment documentation](helm-deploy) step by step. 3. Extract your previous `--set` arguments with the following command. - ```bash - helm get values > pulsar.yaml - ``` + + ```bash + + helm get values > pulsar.yaml + + ``` + 4. Decide all the values you need to set. 5. Perform the upgrade, with all `--set` arguments extracted in step 4. - ```bash - helm upgrade apache/pulsar \ - --version \ - -f pulsar.yaml \ - --set ... - ``` \ No newline at end of file + + ```bash + + helm upgrade apache/pulsar \ + --version \ + -f pulsar.yaml \ + --set ... + + ``` + diff --git a/site2/website-next/docs/io-aerospike-sink.md b/site2/website-next/docs/io-aerospike-sink.md index 174b20dcd143c..f3d48d912302b 100644 --- a/site2/website-next/docs/io-aerospike-sink.md +++ b/site2/website-next/docs/io-aerospike-sink.md @@ -1,7 +1,7 @@ --- id: io-aerospike-sink title: Aerospike sink connector -sidebar_label: Aerospike sink connector +sidebar_label: "Aerospike sink connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-aerospike.md b/site2/website-next/docs/io-aerospike.md index 3d7d84c481119..d0b460f39a07a 100644 --- a/site2/website-next/docs/io-aerospike.md +++ b/site2/website-next/docs/io-aerospike.md @@ -1,7 +1,7 @@ --- id: io-aerospike title: Aerospike Sink Connector -sidebar_label: Aerospike Sink Connector +sidebar_label: "Aerospike Sink Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-canal-source.md b/site2/website-next/docs/io-canal-source.md index 1c846f09fae04..b00d35f1ef501 100644 --- a/site2/website-next/docs/io-canal-source.md +++ b/site2/website-next/docs/io-canal-source.md @@ -1,7 +1,7 @@ --- id: io-canal-source title: Canal source connector -sidebar_label: Canal source connector +sidebar_label: "Canal source connector" --- import Tabs from '@theme/Tabs'; @@ -23,7 +23,7 @@ The configuration of Canal source connector has the following properties. |`destination`|true|None|Source destination that Canal source connector connects to. | `singleHostname` | false | None | Canal server address.| | `singlePort` | false | None | Canal server port.| -| `cluster` | true | false | Whether to enable cluster mode based on Canal server configuration or not.

  • true: **cluster** mode.
    If set to true, it talks to `zkServers` to figure out the actual database host.

  • false: **standalone** mode.
    If set to false, it connects to the database specified by `singleHostname` and `singlePort`.
  • | +| `cluster` | true | false | Whether to enable cluster mode based on Canal server configuration or not.

  • true: **cluster** mode.
    If set to true, it talks to `zkServers` to figure out the actual database host.

  • false: **standalone** mode.
    If set to false, it connects to the database specified by `singleHostname` and `singlePort`.
  • | | `zkServers` | true | None | Address and port of the Zookeeper that Canal source connector talks to figure out the actual database host.| | `batchSize` | false | 1000 | Batch size to fetch from Canal. | @@ -33,34 +33,38 @@ Before using the Canal connector, you can create a configuration file through on * JSON - ```json - { - "zkServers": "127.0.0.1:2181", - "batchSize": "5120", - "destination": "example", - "username": "", - "password": "", - "cluster": false, - "singleHostname": "127.0.0.1", - "singlePort": "11111", - } - ``` + ```json + + { + "zkServers": "127.0.0.1:2181", + "batchSize": "5120", + "destination": "example", + "username": "", + "password": "", + "cluster": false, + "singleHostname": "127.0.0.1", + "singlePort": "11111", + } + + ``` * YAML - You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. - - ```yaml - configs: - zkServers: "127.0.0.1:2181" - batchSize: 5120 - destination: "example" - username: "" - password: "" - cluster: false - singleHostname: "127.0.0.1" - singlePort: 11111 - ``` + You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. + + ```yaml + + configs: + zkServers: "127.0.0.1:2181" + batchSize: 5120 + destination: "example" + username: "" + password: "" + cluster: false + singleHostname: "127.0.0.1" + singlePort: 11111 + + ``` ## Usage @@ -68,139 +72,167 @@ Here is an example of storing MySQL data using the configuration file as above. 1. Start a MySQL server. - ```bash - $ docker pull mysql:5.7 - $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 - ``` + ```bash + + $ docker pull mysql:5.7 + $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 + + ``` 2. Create a configuration file `mysqld.cnf`. - ```bash - [mysqld] - pid-file = /var/run/mysqld/mysqld.pid - socket = /var/run/mysqld/mysqld.sock - datadir = /var/lib/mysql - #log-error = /var/log/mysql/error.log - # By default we only accept connections from localhost - #bind-address = 127.0.0.1 - # Disabling symbolic-links is recommended to prevent assorted security risks - symbolic-links=0 - log-bin=mysql-bin - binlog-format=ROW - server_id=1 - ``` + ```bash + + [mysqld] + pid-file = /var/run/mysqld/mysqld.pid + socket = /var/run/mysqld/mysqld.sock + datadir = /var/lib/mysql + #log-error = /var/log/mysql/error.log + # By default we only accept connections from localhost + #bind-address = 127.0.0.1 + # Disabling symbolic-links is recommended to prevent assorted security risks + symbolic-links=0 + log-bin=mysql-bin + binlog-format=ROW + server_id=1 + + ``` 3. Copy the configuration file `mysqld.cnf` to MySQL server. + + ```bash - ```bash - $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ - ``` + $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ + + ``` 4. Restart the MySQL server. + + ```bash - ```bash - $ docker restart pulsar-mysql - ``` + $ docker restart pulsar-mysql + + ``` 5. Create a test database in MySQL server. + + ```bash - ```bash - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' - ``` + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' + + ``` 6. Start a Canal server and connect to MySQL server. - ``` - $ docker pull canal/canal-server:v1.1.2 - $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 - ``` + ``` + + $ docker pull canal/canal-server:v1.1.2 + $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 + + ``` 7. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:2.3.0 - $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone - ``` + ```bash + + $ docker pull apachepulsar/pulsar:2.3.0 + $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone + + ``` 8. Modify the configuration file `canal-mysql-source-config.yaml`. - ```yaml - configs: - zkServers: "" - batchSize: "5120" - destination: "test" - username: "" - password: "" - cluster: false - singleHostname: "pulsar-canal-server" - singlePort: "11111" - ``` + ```yaml + + configs: + zkServers: "" + batchSize: "5120" + destination: "test" + username: "" + password: "" + cluster: false + singleHostname: "pulsar-canal-server" + singlePort: "11111" + + ``` 9. Create a consumer file `pulsar-client.py`. - ```python - import pulsar + ```python + + import pulsar - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', - subscription_name='my-sub') + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', + subscription_name='my-sub') - while True: - msg = consumer.receive() - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + while True: + msg = consumer.receive() + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) - client.close() - ``` + client.close() + + ``` 10. Copy the configuration file `canal-mysql-source-config.yaml` and the consumer file `pulsar-client.py` to Pulsar server. - ```bash - $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ - $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ - ``` + ```bash + + $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ + $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ + + ``` 11. Download a Canal connector and start it. - - ```bash - $ docker exec -it pulsar-standalone /bin/bash - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors - $ ./bin/pulsar-admin source localrun \ - --archive ./connectors/pulsar-io-canal-2.3.0.nar \ - --classname org.apache.pulsar.io.canal.CanalStringSource \ - --tenant public \ - --namespace default \ - --name canal \ - --destination-topic-name my-topic \ - --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ - --parallelism 1 - ``` + + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors + $ ./bin/pulsar-admin source localrun \ + --archive ./connectors/pulsar-io-canal-2.3.0.nar \ + --classname org.apache.pulsar.io.canal.CanalStringSource \ + --tenant public \ + --namespace default \ + --name canal \ + --destination-topic-name my-topic \ + --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ + --parallelism 1 + + ``` 12. Consume data from MySQL. - ```bash - $ docker exec -it pulsar-standalone /bin/bash - $ python pulsar-client.py - ``` + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ python pulsar-client.py + + ``` 13. Open another window to log in MySQL server. - ```bash - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal - ``` + ```bash + + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal + + ``` 14. Create a table, and insert, delete, and update data in MySQL server. - - ```bash - mysql> use test; - mysql> show tables; - mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, - `test_author` VARCHAR(40) NOT NULL, - `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; - mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); - mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; - mysql> DELETE FROM test_table WHERE test_title='c'; - ``` + + ```bash + + mysql> use test; + mysql> show tables; + mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, + `test_author` VARCHAR(40) NOT NULL, + `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; + mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); + mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; + mysql> DELETE FROM test_table WHERE test_title='c'; + + ``` diff --git a/site2/website-next/docs/io-cassandra-sink.md b/site2/website-next/docs/io-cassandra-sink.md index 5f0403fa9ed31..39cd538a3d9a2 100644 --- a/site2/website-next/docs/io-cassandra-sink.md +++ b/site2/website-next/docs/io-cassandra-sink.md @@ -1,7 +1,7 @@ --- id: io-cassandra-sink title: Cassandra sink connector -sidebar_label: Cassandra sink connector +sidebar_label: "Cassandra sink connector" --- import Tabs from '@theme/Tabs'; @@ -30,27 +30,30 @@ Before using the Cassandra sink connector, you need to create a configuration fi * JSON - ```json - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - ``` + ```json + + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` * YAML - - ``` - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" - ``` + ``` + + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" + + ``` ## Usage diff --git a/site2/website-next/docs/io-cassandra.md b/site2/website-next/docs/io-cassandra.md index 2a58c5a5ce82a..c6fb34a2a4987 100644 --- a/site2/website-next/docs/io-cassandra.md +++ b/site2/website-next/docs/io-cassandra.md @@ -1,7 +1,7 @@ --- id: io-cassandra title: Cassandra Sink Connector -sidebar_label: Cassandra Sink Connector +sidebar_label: "Cassandra Sink Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-cdc-canal.md b/site2/website-next/docs/io-cdc-canal.md index c16a3cd141a8d..f8556b5b47349 100644 --- a/site2/website-next/docs/io-cdc-canal.md +++ b/site2/website-next/docs/io-cdc-canal.md @@ -1,7 +1,7 @@ --- id: io-cdc-canal title: CDC Canal Connector -sidebar_label: CDC Canal Connector +sidebar_label: "CDC Canal Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-cdc-debezium.md b/site2/website-next/docs/io-cdc-debezium.md index 3d6fcb3e98dfc..cd9806f2f854b 100644 --- a/site2/website-next/docs/io-cdc-debezium.md +++ b/site2/website-next/docs/io-cdc-debezium.md @@ -1,7 +1,7 @@ --- id: io-cdc-debezium title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" --- import Tabs from '@theme/Tabs'; @@ -24,11 +24,11 @@ The configuration of Debezium source connector has the following properties. | `database.password` | true | null | The password for a database user that has the required privileges. | | `database.server.id` | true | null | The connector’s identifier that must be unique within a database cluster and similar to the database’s server-id configuration property. | | `database.server.name` | true | null | The logical name of a database server/cluster, which forms a namespace and it is used in all the names of Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro Connector is used. | -| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | +| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | | `key.converter` | true | null | The converter provided by Kafka Connect to convert record key. | | `value.converter` | true | null | The converter provided by Kafka Connect to convert record value. | | `database.history` | true | null | The name of the database history class. | -| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | +| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | | `database.history.pulsar.service.url` | true | null | Pulsar cluster service URL for history topic. | | `pulsar.service.url` | true | null | Pulsar cluster service URL for the offset topic used in Debezium. You can use the `bin/pulsar-admin --admin-url http://pulsar:8080 sources localrun --source-config-file configs/pg-pulsar-config.yaml` command to point to the target Pulsar cluster.| | `offset.storage.topic` | true | null | Record the last committed offsets that the connector successfully completes. | @@ -50,61 +50,65 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "pulsar.service.url": "pulsar://127.0.0.1:6650", - "offset.storage.topic": "offset-topic" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - ``` + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -112,72 +116,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar`. + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - ``` + * Use the **YAML** configuration file as shown previously. - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -189,45 +207,49 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "postgres", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "schema.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "postgres", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "schema.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for pg, docker image: debezium/example-postgress:0.8 - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "postgres" - database.dbname: "postgres" - database.server.name: "dbserver1" - schema.whitelist: "inventory" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.8 + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "postgres" + database.dbname: "postgres" + database.server.name: "dbserver1" + schema.whitelist: "inventory" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -236,129 +258,150 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-postgres:0.8 - $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 - ``` + ```bash + + $ docker pull debezium/example-postgres:0.8 + $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - ``` + + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a PostgreSQL client in docker. + + ```bash - ```bash - $ docker exec -it pulsar-postgresql /bin/bash - ``` + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to change the data of the table _products_. - ``` - psql -U postgres postgres - postgres=# \c postgres; - You are now connected to database "postgres" as user "postgres". - postgres=# SET search_path TO inventory; - SET - postgres=# select * from products; - id | name | description | weight - -----+--------------------+---------------------------------------------------------+-------- - 102 | car battery | 12V car battery | 8.1 - 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 - 104 | hammer | 12oz carpenter's hammer | 0.75 - 105 | hammer | 14oz carpenter's hammer | 0.875 - 106 | hammer | 16oz carpenter's hammer | 1 - 107 | rocks | box of assorted rocks | 5.3 - 108 | jacket | water resistent black wind breaker | 0.1 - 109 | spare tire | 24 inch spare tire | 22.2 - 101 | 1111111111 | Small 2-wheel scooter | 3.14 - (9 rows) - - postgres=# UPDATE products SET name='1111111111' WHERE id=107; - UPDATE 1 - ``` + ``` + + psql -U postgres postgres + postgres=# \c postgres; + You are now connected to database "postgres" as user "postgres". + postgres=# SET search_path TO inventory; + SET + postgres=# select * from products; + id | name | description | weight + -----+--------------------+---------------------------------------------------------+-------- + 102 | car battery | 12V car battery | 8.1 + 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 + 104 | hammer | 12oz carpenter's hammer | 0.75 + 105 | hammer | 14oz carpenter's hammer | 0.875 + 106 | hammer | 16oz carpenter's hammer | 1 + 107 | rocks | box of assorted rocks | 5.3 + 108 | jacket | water resistent black wind breaker | 0.1 + 109 | spare tire | 24 inch spare tire | 22.2 + 101 | 1111111111 | Small 2-wheel scooter | 3.14 + (9 rows) + + postgres=# UPDATE products SET name='1111111111' WHERE id=107; + UPDATE 1 + + ``` + + In the terminal window of subscribing topic, you can receive the following messages. + + ```bash + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for pg, docker image: debezium/example-postgress:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -367,78 +410,106 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 - ``` - Use the following commands to initialize the data. + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` + + Use the following commands to initialize the data. + + ``` bash - ``` bash - ./usr/local/bin/init-inventory.sh - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` + ./usr/local/bin/init-inventory.sh + + ``` + + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ``` + + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + ``` + + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml - ``` + + ``` + + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MongoDB client in docker. + + ``` + - ```bash - $ docker exec -it pulsar-mongodb /bin/bash - ``` + $ docker exec -it pulsar-mongodb /bin/bash + + ``` 6. A MongoDB client pops out. + + ``` + - ```bash - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) - ``` + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. - ``` + In the terminal window of subscribing topic, you can receive the following messages. + + ``` + + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + ``` + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -448,15 +519,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -469,10 +540,14 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupJavaInstance(JavaInstanceRunnable.java:200) at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` + +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= -``` \ No newline at end of file + +``` + diff --git a/site2/website-next/docs/io-cdc.md b/site2/website-next/docs/io-cdc.md index a49fe5df4af0a..d4f4f0f5f3778 100644 --- a/site2/website-next/docs/io-cdc.md +++ b/site2/website-next/docs/io-cdc.md @@ -1,7 +1,7 @@ --- id: io-cdc title: CDC connector -sidebar_label: CDC connector +sidebar_label: "CDC connector" --- import Tabs from '@theme/Tabs'; @@ -10,19 +10,14 @@ import TabItem from '@theme/TabItem'; CDC source connectors capture log changes of databases (such as MySQL, MongoDB, and PostgreSQL) into Pulsar. -:::note - -CDC source connectors are built on top of [Canal](https://github.com/alibaba/canal) and [Debezium](https://debezium.io/) and store all data into Pulsar cluster in a persistent, replicated, and partitioned way. - -::: - +> CDC source connectors are built on top of [Canal](https://github.com/alibaba/canal) and [Debezium](https://debezium.io/) and store all data into Pulsar cluster in a persistent, replicated, and partitioned way. Currently, Pulsar has the following CDC connectors. Name|Java Class |---|--- -[Canal source connector](io-canal-source.md)|[org.apache.pulsar.io.canal.CanalStringSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalStringSource.java) -[Debezium source connector](io-cdc-debezium.md)|
  • [org.apache.pulsar.io.debezium.DebeziumSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java)
  • [org.apache.pulsar.io.debezium.mysql.DebeziumMysqlSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/java/org/apache/pulsar/io/debezium/mysql/DebeziumMysqlSource.java)
  • [org.apache.pulsar.io.debezium.postgres.DebeziumPostgresSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/java/org/apache/pulsar/io/debezium/postgres/DebeziumPostgresSource.java)
  • +[Canal source connector](io-canal-source)|[org.apache.pulsar.io.canal.CanalStringSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalStringSource.java) +[Debezium source connector](io-cdc-debezium)|
  • [org.apache.pulsar.io.debezium.DebeziumSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java)
  • [org.apache.pulsar.io.debezium.mysql.DebeziumMysqlSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/java/org/apache/pulsar/io/debezium/mysql/DebeziumMysqlSource.java)
  • [org.apache.pulsar.io.debezium.postgres.DebeziumPostgresSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/java/org/apache/pulsar/io/debezium/postgres/DebeziumPostgresSource.java)
  • For more information about Canal and Debezium, see the information below. diff --git a/site2/website-next/docs/io-cli.md b/site2/website-next/docs/io-cli.md index 268f60f19e7b6..5476eec9af2c6 100644 --- a/site2/website-next/docs/io-cli.md +++ b/site2/website-next/docs/io-cli.md @@ -1,7 +1,7 @@ --- id: io-cli title: Connector Admin CLI -sidebar_label: CLI +sidebar_label: "CLI" --- import Tabs from '@theme/Tabs'; @@ -15,7 +15,9 @@ The `pulsar-admin` tool helps you manage Pulsar connectors. An interface for managing Pulsar IO sources (ingress data into Pulsar). ```bash + $ pulsar-admin sources subcommands + ``` Subcommands are: @@ -52,7 +54,9 @@ Submit a Pulsar IO source connector to run in a Pulsar cluster. #### Usage ```bash + $ pulsar-admin sources create options + ``` #### Options @@ -84,7 +88,9 @@ Update a already submitted Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources update options + ``` #### Options @@ -117,7 +123,9 @@ Delete a Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources delete options + ``` #### Option @@ -135,7 +143,9 @@ Get the information about a Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources get options + ``` #### Options @@ -153,7 +163,9 @@ Check the current status of a Pulsar Source. #### Usage ```bash + $ pulsar-admin sources status options + ``` #### Options @@ -172,7 +184,9 @@ List all running Pulsar IO source connectors. #### Usage ```bash + $ pulsar-admin sources list options + ``` #### Options @@ -190,7 +204,9 @@ Stop a source instance. #### Usage ```bash + $ pulsar-admin sources stop options + ``` #### Options @@ -209,7 +225,9 @@ Start a source instance. #### Usage ```bash + $ pulsar-admin sources start options + ``` #### Options @@ -229,7 +247,9 @@ Restart a source instance. #### Usage ```bash + $ pulsar-admin sources restart options + ``` #### Options @@ -248,7 +268,9 @@ Run a Pulsar IO source connector locally rather than deploying it to the Pulsar #### Usage ```bash + $ pulsar-admin sources localrun options + ``` #### Options @@ -287,7 +309,9 @@ Get the list of Pulsar IO connector sources supported by Pulsar cluster. #### Usage ```bash + $ pulsar-admin sources available-sources + ``` ### `reload` @@ -297,7 +321,9 @@ Reload the available built-in connectors. #### Usage ```bash + $ pulsar-admin sources reload + ``` ## `sinks` @@ -305,7 +331,9 @@ $ pulsar-admin sources reload An interface for managing Pulsar IO sinks (egress data from Pulsar). ```bash + $ pulsar-admin sinks subcommands + ``` Subcommands are: @@ -342,7 +370,9 @@ Submit a Pulsar IO sink connector to run in a Pulsar cluster. #### Usage ```bash + $ pulsar-admin sinks create options + ``` #### Options @@ -378,7 +408,9 @@ Update a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks update options + ``` #### Options @@ -415,7 +447,9 @@ Delete a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks delete options + ``` #### Option @@ -433,7 +467,9 @@ Get the information about a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks get options + ``` #### Options @@ -451,7 +487,9 @@ Check the current status of a Pulsar sink. #### Usage ```bash + $ pulsar-admin sinks status options + ``` #### Options @@ -471,7 +509,9 @@ List all running Pulsar IO sink connectors. #### Usage ```bash + $ pulsar-admin sinks list options + ``` #### Options @@ -489,7 +529,9 @@ Stop a sink instance. #### Usage ```bash + $ pulsar-admin sinks stop options + ``` #### Options @@ -508,7 +550,9 @@ Start a sink instance. #### Usage ```bash + $ pulsar-admin sinks start options + ``` #### Options @@ -528,7 +572,9 @@ Restart a sink instance. #### Usage ```bash + $ pulsar-admin sinks restart options + ``` #### Options @@ -548,7 +594,9 @@ Run a Pulsar IO sink connector locally rather than deploying it to the Pulsar cl #### Usage ```bash + $ pulsar-admin sinks localrun options + ``` #### Options @@ -594,7 +642,9 @@ Get the list of Pulsar IO connector sinks supported by Pulsar cluster. #### Usage ```bash + $ pulsar-admin sinks available-sinks + ``` ### `reload` @@ -604,6 +654,8 @@ Reload the available built-in connectors. #### Usage ```bash + $ pulsar-admin sinks reload + ``` diff --git a/site2/website-next/docs/io-connectors.md b/site2/website-next/docs/io-connectors.md index dc8faaae76793..2c6affd6fcd21 100644 --- a/site2/website-next/docs/io-connectors.md +++ b/site2/website-next/docs/io-connectors.md @@ -1,7 +1,7 @@ --- id: io-connectors title: Built-in connector -sidebar_label: Built-in connector +sidebar_label: "Built-in connector" --- import Tabs from '@theme/Tabs'; @@ -49,6 +49,23 @@ Pulsar has various source connectors, which are sorted alphabetically as below. * [Java class](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/java/org/apache/pulsar/io/debezium/mongodb/DebeziumMongoDbSource.java) +### Debezium Oracle + +* [Configuration](io-debezium-source.md#configuration) + +* [Example](io-debezium-source.md#example-of-oracle) + +* [Java class](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/oracle/src/main/java/org/apache/pulsar/io/debezium/oracle/DebeziumOracleSource.java) + +### Debezium Microsoft SQL Server + +* [Configuration](io-debezium-source.md#configuration) + +* [Example](io-debezium-source.md#example-of-microsoft-sql) + +* [Java class](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mssql/src/main/java/org/apache/pulsar/io/debezium/mssql/DebeziumMsSqlSource.java) + + ### DynamoDB * [Configuration](io-dynamodb-source.md#configuration) diff --git a/site2/website-next/docs/io-debezium-source.md b/site2/website-next/docs/io-debezium-source.md index b3bef5b4b7c60..fbe0781364037 100644 --- a/site2/website-next/docs/io-debezium-source.md +++ b/site2/website-next/docs/io-debezium-source.md @@ -1,7 +1,7 @@ --- id: io-debezium-source title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" --- import Tabs from '@theme/Tabs'; @@ -24,11 +24,11 @@ The configuration of Debezium source connector has the following properties. | `database.password` | true | null | The password for a database user that has the required privileges. | | `database.server.id` | true | null | The connector’s identifier that must be unique within a database cluster and similar to the database’s server-id configuration property. | | `database.server.name` | true | null | The logical name of a database server/cluster, which forms a namespace and it is used in all the names of Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro Connector is used. | -| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | +| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | | `key.converter` | true | null | The converter provided by Kafka Connect to convert record key. | | `value.converter` | true | null | The converter provided by Kafka Connect to convert record value. | | `database.history` | true | null | The name of the database history class. | -| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | +| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | | `database.history.pulsar.service.url` | true | null | Pulsar cluster service URL for history topic. | | `offset.storage.topic` | true | null | Record the last committed offsets that the connector successfully completes. | | `json-with-envelope` | false | false | Present the message only consist of payload. @@ -70,57 +70,61 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "offset.storage.topic": "offset-topic" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" - - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - ``` + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -128,72 +132,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar`. + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - ``` + * Use the **YAML** configuration file as shown previously. - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -205,50 +223,54 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "changeme", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "plugin.name": "pgoutput", - "schema.whitelist": "public", - "table.whitelist": "public.users", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "changeme", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "plugin.name": "pgoutput", + "schema.whitelist": "public", + "table.whitelist": "public.users", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for postgres version 10+, official docker image: postgres:<10+> - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "changeme" - database.dbname: "postgres" - database.server.name: "dbserver1" - plugin.name: "pgoutput" - schema.whitelist: "public" - table.whitelist: "public.users" - - ## PULSAR_SERVICE_URL_CONFIG - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` - + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for postgres version 10+, official docker image: postgres:<10+> + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "changeme" + database.dbname: "postgres" + database.server.name: "dbserver1" + plugin.name: "pgoutput" + schema.whitelist: "public" + table.whitelist: "public.users" + + ## PULSAR_SERVICE_URL_CONFIG + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` + Notice that `pgoutput` is a standard plugin of Postgres introduced in version 10 - [see Postgres architecture docu](https://www.postgresql.org/docs/10/logical-replication-architecture.html). You don't need to install anything, just make sure the WAL level is set to `logical` (see docker command below and [Postgres docu](https://www.postgresql.org/docs/current/runtime-config-wal.html)). ### Usage @@ -258,135 +280,156 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=changeme \ - postgres:13.3 -c wal_level=logical - ``` + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=changeme \ + postgres:13.3 -c wal_level=logical + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - ``` + + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-users_ for the _public.users_ table. - ``` - $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 - ``` + ``` + + $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 + + ``` 5. Start a PostgreSQL client in docker. + + ```bash - ```bash - $ docker exec -it pulsar-postgresql /bin/bash - ``` + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to create sample data in the table _users_. - ``` - psql -U postgres -h localhost -p 5432 - Password for user postgres: - - CREATE TABLE users( - id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), - hash_firstname TEXT NOT NULL, - hash_lastname TEXT NOT NULL, - gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) - ); - - INSERT INTO users(hash_firstname, hash_lastname, gender) - SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); - - postgres=# select * from users; - - id | hash_firstname | hash_lastname | gender - -------+----------------------------------+----------------------------------+-------- - 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female - 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male - 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female - 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female + ``` + + psql -U postgres -h localhost -p 5432 + Password for user postgres: + + CREATE TABLE users( + id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), + hash_firstname TEXT NOT NULL, + hash_lastname TEXT NOT NULL, + gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) + ); + + INSERT INTO users(hash_firstname, hash_lastname, gender) + SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); + + postgres=# select * from users; + + id | hash_firstname | hash_lastname | gender + -------+----------------------------------+----------------------------------+-------- + 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female + 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male + 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female + 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female - - postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; - UPDATE 1 - ``` + + postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; + UPDATE 1 + + ``` + + In the terminal window of subscribing topic, you can receive the following messages. + + ```bash + + ----- got message ----- + {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} + ...many more + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} - ...many more - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for pg, docker image: debezium/example-mongodb:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-mongodb:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -395,78 +438,260 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 - ``` - Use the following commands to initialize the data. + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` + + Use the following commands to initialize the data. + + ``` bash + + ./usr/local/bin/init-inventory.sh - ``` bash - ./usr/local/bin/init-inventory.sh - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` + ``` + + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ``` + + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + ``` + + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml - ``` + + ``` + + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MongoDB client in docker. + + ``` + - ```bash - $ docker exec -it pulsar-mongodb /bin/bash - ``` + $ docker exec -it pulsar-mongodb /bin/bash + + ``` 6. A MongoDB client pops out. + + ``` + - ```bash - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) - ``` + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. - ``` + In the terminal window of subscribing topic, you can receive the following messages. + + ``` + + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + ``` + +## Example of Oracle + +### Packaging + +Oracle connector does not include Oracle JDBC driver and you need to package it with the connector. +Major reasons for not including the drivers are the variety of versions and Oracle licensing. It is recommended to use the driver provided with your Oracle DB installation, or you can [download](https://www.oracle.com/database/technologies/appdev/jdbc.html) one. +Integration test have an [example](https://github.com/apache/pulsar/blob/e2bc52d40450fa00af258c4432a5b71d50a5c6e0/tests/docker-images/latest-version-image/Dockerfile#L110-L122) of packaging the driver into the connector nar file. + +### Configuration + +Debezium [requires](https://debezium.io/documentation/reference/1.5/connectors/oracle.html#oracle-overview) Oracle DB with LogMiner or XStream API enabled. +Supported options and steps for enabling them vary from version to version of Oracle DB. +Steps outlined in the [documentation](https://debezium.io/documentation/reference/1.5/connectors/oracle.html#oracle-overview) and used in the [integration test](https://github.com/apache/pulsar/blob/master/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java) may or may not work for the version and edition of Oracle DB you are using. +Please refer to the [documentation for Oracle DB](https://docs.oracle.com/en/database/oracle/oracle-database/) as needed. + +Similarly to other connectors, you can use JSON or YAMl to configure the connector. +Using yaml as an example, you can create a debezium-oracle-source-config.yaml file like: + +* JSON + +``` + +{ + "database.hostname": "localhost", + "database.port": "1521", + "database.user": "dbzuser", + "database.password": "dbz", + "database.dbname": "XE", + "database.server.name": "XE", + "schema.exclude.list": "system,dbzuser", + "snapshot.mode": "initial", + "topic.namespace": "public/default", + "task.class": "io.debezium.connector.oracle.OracleConnectorTask", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "typeClassName": "org.apache.pulsar.common.schema.KeyValue", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.tcpKeepAlive": "true", + "decimal.handling.mode": "double", + "database.history.pulsar.topic": "debezium-oracle-source-history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" +} + +``` + +* YAML + +``` + +tenant: "public" +namespace: "default" +name: "debezium-oracle-source" +topicName: "debezium-oracle-topic" +parallelism: 1 + +className: "org.apache.pulsar.io.debezium.oracle.DebeziumOracleSource" +database.dbname: "XE" + +configs: + database.hostname: "localhost" + database.port: "1521" + database.user: "dbzuser" + database.password: "dbz" + database.dbname: "XE" + database.server.name: "XE" + schema.exclude.list: "system,dbzuser" + snapshot.mode: "initial" + topic.namespace: "public/default" + task.class: "io.debezium.connector.oracle.OracleConnectorTask" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + key.converter: "org.apache.kafka.connect.json.JsonConverter" + typeClassName: "org.apache.pulsar.common.schema.KeyValue" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.tcpKeepAlive: "true" + decimal.handling.mode: "double" + database.history.pulsar.topic: "debezium-oracle-source-history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + +``` + +For the full list of configuration properties supported by Debezium, see [Debezium Connector for Oracle](https://debezium.io/documentation/reference/1.5/connectors/oracle.html#oracle-connector-properties). + +## Example of Microsoft SQL + +### Configuration + +Debezium [requires](https://debezium.io/documentation/reference/1.5/connectors/sqlserver.html#sqlserver-overview) SQL Server with CDC enabled. +Steps outlined in the [documentation](https://debezium.io/documentation/reference/1.5/connectors/sqlserver.html#setting-up-sqlserver) and used in the [integration test](https://github.com/apache/pulsar/blob/master/tests/integration/src/test/java/org/apache/pulsar/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMsSqlSourceTester.java). +For more information, see [Enable and disable change data capture in Microsoft SQL Server](https://docs.microsoft.com/en-us/sql/relational-databases/track-changes/enable-and-disable-change-data-capture-sql-server). + +Similarly to other connectors, you can use JSON or YAMl to configure the connector. + +* JSON + +``` + +{ + "database.hostname": "localhost", + "database.port": "1433", + "database.user": "sa", + "database.password": "MyP@ssw0rd!", + "database.dbname": "MyTestDB", + "database.server.name": "mssql", + "snapshot.mode": "schema_only", + "topic.namespace": "public/default", + "task.class": "io.debezium.connector.sqlserver.SqlServerConnectorTask", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "typeClassName": "org.apache.pulsar.common.schema.KeyValue", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.tcpKeepAlive": "true", + "decimal.handling.mode": "double", + "database.history.pulsar.topic": "debezium-mssql-source-history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650" +} + +``` + +* YAML + +``` + +tenant: "public" +namespace: "default" +name: "debezium-mssql-source" +topicName: "debezium-mssql-topic" +parallelism: 1 + +className: "org.apache.pulsar.io.debezium.mssql.DebeziumMsSqlSource" +database.dbname: "mssql" + +configs: + database.hostname: "localhost" + database.port: "1433" + database.user: "sa" + database.password: "MyP@ssw0rd!" + database.dbname: "MyTestDB" + database.server.name: "mssql" + snapshot.mode: "schema_only" + topic.namespace: "public/default" + task.class: "io.debezium.connector.sqlserver.SqlServerConnectorTask" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + key.converter: "org.apache.kafka.connect.json.JsonConverter" + typeClassName: "org.apache.pulsar.common.schema.KeyValue" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.tcpKeepAlive: "true" + decimal.handling.mode: "double" + database.history.pulsar.topic: "debezium-mssql-source-history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + +``` + +For the full list of configuration properties supported by Debezium, see [Debezium Connector for MS SQL](https://debezium.io/documentation/reference/1.5/connectors/sqlserver.html#sqlserver-connector-properties). + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -476,15 +701,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -497,10 +722,14 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupJavaInstance(JavaInstanceRunnable.java:200) at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` + +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= + ``` + diff --git a/site2/website-next/docs/io-debug.md b/site2/website-next/docs/io-debug.md index aca5d261e1a04..93e8d47b3cf1a 100644 --- a/site2/website-next/docs/io-debug.md +++ b/site2/website-next/docs/io-debug.md @@ -1,7 +1,7 @@ --- id: io-debug title: How to debug Pulsar connectors -sidebar_label: Debug +sidebar_label: "Debug" --- import Tabs from '@theme/Tabs'; @@ -12,45 +12,63 @@ To better demonstrate how to debug Pulsar connectors, here takes a Mongo sink co **Deploy a Mongo sink environment** 1. Start a Mongo service. - ```bash - docker pull mongo:4 - docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 - ``` -2. Create a DB and a collection. - ```bash - docker exec -it pulsar-mongo /bin/bash - mongo - :::note - use pulsar - db.createCollection('messages') - exit + ```bash + + docker pull mongo:4 + docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 + + ``` + +2. Create a DB and a collection. - ::: + ```bash + + docker exec -it pulsar-mongo /bin/bash + mongo + > use pulsar + > db.createCollection('messages') + > exit + + ``` - ``` 3. Start Pulsar standalone. - ```bash - docker pull apachepulsar/pulsar:2.4.0 - docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone - ``` + + ```bash + + docker pull apachepulsar/pulsar:2.4.0 + docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + + ``` + 4. Configure the Mongo sink with the `mongo-sink-config.yaml` file. - ```bash - configs: - mongoUri: "mongodb://pulsar-mongo:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - ``` - ```bash - docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ - ``` + + ```bash + + configs: + mongoUri: "mongodb://pulsar-mongo:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + + ``` + + ```bash + + docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ + + ``` + 5. Download the Mongo sink nar package. - ```bash - docker exec -it pulsar-mongo-standalone /bin/bash - curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar - ``` + + ```bash + + docker exec -it pulsar-mongo-standalone /bin/bash + curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar + + ``` + ## Debug in localrun mode Start the Mongo sink in localrun mode using the `localrun` command. :::tip @@ -60,6 +78,7 @@ For more information about the `localrun` command, see [`localrun`](reference-co ::: ```bash + ./bin/pulsar-admin sinks localrun \ --archive pulsar-io-mongo-2.4.0.nar \ --tenant public --namespace default \ @@ -67,122 +86,146 @@ For more information about the `localrun` command, see [`localrun`](reference-co --name pulsar-mongo-sink \ --sink-config-file mongo-sink-config.yaml \ --parallelism 1 + ``` + ### Use connector log Use one of the following methods to get a connector log in localrun mode: * After executing the `localrun` command, the **log is automatically printed on the console**. * The log is located at: + + ```bash - ```bash - logs/functions/tenant/namespace/function-name/function-name-instance-id.log - ``` - - **Example** - - The path of the Mongo sink connector is: - ```bash - logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log - ``` + logs/functions/tenant/namespace/function-name/function-name-instance-id.log + + ``` + + **Example** + + The path of the Mongo sink connector is: + + ```bash + + logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log + + ``` + To clearly explain the log information, here breaks down the large block of information into small blocks and add descriptions for each block. * This piece of log information shows the storage path of the nar package after decompression. - ``` - 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, - ``` - :::tip - If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. + ``` + + 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, + + ``` + + :::tip - ::: + If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. + + ::: * This piece of log information illustrates the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, resources, and so on, which can be used to **check whether the Mongo sink connector is configured correctly or not**. - ```bash - 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" - namespace: "default" - name: "pulsar-mongo-sink" - className: "org.apache.pulsar.functions.api.utils.IdentityFunction" - autoAck: true - parallelism: 1 - source { - typeClassName: "[B" - inputSpecs { - key: "test-mongo" - value { - } - } - cleanupSubscription: true - } - sink { - className: "org.apache.pulsar.io.mongodb.MongoSink" - configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" - typeClassName: "[B" - } - resources { - cpu: 1.0 - ram: 1073741824 - disk: 10737418240 - } - componentType: SINK - , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) - ``` + + ```bash + + 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" + namespace: "default" + name: "pulsar-mongo-sink" + className: "org.apache.pulsar.functions.api.utils.IdentityFunction" + autoAck: true + parallelism: 1 + source { + typeClassName: "[B" + inputSpecs { + key: "test-mongo" + value { + } + } + cleanupSubscription: true + } + sink { + className: "org.apache.pulsar.io.mongodb.MongoSink" + configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" + typeClassName: "[B" + } + resources { + cpu: 1.0 + ram: 1073741824 + disk: 10737418240 + } + componentType: SINK + , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) + + ``` + * This piece of log information demonstrates the status of the connections to Mongo and configuration information. - ```bash - 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 - 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} - ``` + + ```bash + + 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 + 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} + + ``` + * This piece of log information explains the configuration of consumers and clients, including the topic name, subscription name, subscription type, and so on. - ```bash - 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { - "topicNames" : [ "test-mongo" ], - "topicsPattern" : null, - "subscriptionName" : "public/default/pulsar-mongo-sink", - "subscriptionType" : "Shared", - "receiverQueueSize" : 1000, - "acknowledgementsGroupTimeMicros" : 100000, - "negativeAckRedeliveryDelayMicros" : 60000000, - "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, - "consumerName" : null, - "ackTimeoutMillis" : 0, - "tickDurationMillis" : 1000, - "priorityLevel" : 0, - "cryptoFailureAction" : "CONSUME", - "properties" : { - "application" : "pulsar-sink", - "id" : "public/default/pulsar-mongo-sink", - "instance_id" : "0" - }, - "readCompacted" : false, - "subscriptionInitialPosition" : "Latest", - "patternAutoDiscoveryPeriod" : 1, - "regexSubscriptionMode" : "PersistentOnly", - "deadLetterPolicy" : null, - "autoUpdatePartitions" : true, - "replicateSubscriptionState" : false, - "resetIncludeHead" : false - } - 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { - "serviceUrl" : "pulsar://localhost:6650", - "authPluginClassName" : null, - "authParams" : null, - "operationTimeoutMs" : 30000, - "statsIntervalSeconds" : 60, - "numIoThreads" : 1, - "numListenerThreads" : 1, - "connectionsPerBroker" : 1, - "useTcpNoDelay" : true, - "useTls" : false, - "tlsTrustCertsFilePath" : null, - "tlsAllowInsecureConnection" : false, - "tlsHostnameVerificationEnable" : false, - "concurrentLookupRequest" : 5000, - "maxLookupRequest" : 50000, - "maxNumberOfRejectedRequestPerConnection" : 50, - "keepAliveIntervalSeconds" : 30, - "connectionTimeoutMs" : 10000, - "requestTimeoutMs" : 60000, - "defaultBackoffIntervalNanos" : 100000000, - "maxBackoffIntervalNanos" : 30000000000 - } - ``` + + ```bash + + 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { + "topicNames" : [ "test-mongo" ], + "topicsPattern" : null, + "subscriptionName" : "public/default/pulsar-mongo-sink", + "subscriptionType" : "Shared", + "receiverQueueSize" : 1000, + "acknowledgementsGroupTimeMicros" : 100000, + "negativeAckRedeliveryDelayMicros" : 60000000, + "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, + "consumerName" : null, + "ackTimeoutMillis" : 0, + "tickDurationMillis" : 1000, + "priorityLevel" : 0, + "cryptoFailureAction" : "CONSUME", + "properties" : { + "application" : "pulsar-sink", + "id" : "public/default/pulsar-mongo-sink", + "instance_id" : "0" + }, + "readCompacted" : false, + "subscriptionInitialPosition" : "Latest", + "patternAutoDiscoveryPeriod" : 1, + "regexSubscriptionMode" : "PersistentOnly", + "deadLetterPolicy" : null, + "autoUpdatePartitions" : true, + "replicateSubscriptionState" : false, + "resetIncludeHead" : false + } + 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { + "serviceUrl" : "pulsar://localhost:6650", + "authPluginClassName" : null, + "authParams" : null, + "operationTimeoutMs" : 30000, + "statsIntervalSeconds" : 60, + "numIoThreads" : 1, + "numListenerThreads" : 1, + "connectionsPerBroker" : 1, + "useTcpNoDelay" : true, + "useTls" : false, + "tlsTrustCertsFilePath" : null, + "tlsAllowInsecureConnection" : false, + "tlsHostnameVerificationEnable" : false, + "concurrentLookupRequest" : 5000, + "maxLookupRequest" : 50000, + "maxNumberOfRejectedRequestPerConnection" : 50, + "keepAliveIntervalSeconds" : 30, + "connectionTimeoutMs" : 10000, + "requestTimeoutMs" : 60000, + "defaultBackoffIntervalNanos" : 100000000, + "maxBackoffIntervalNanos" : 30000000000 + } + + ``` + ## Debug in cluster mode You can use the following methods to debug a connector in cluster mode: * [Use connector log](#use-connector-log) @@ -197,7 +240,9 @@ Pulsar admin CLI helps you debug Pulsar connectors with the following subcommand * [`topics stats`](#topics-stats) **Create a Mongo sink** + ```bash + ./bin/pulsar-admin sinks create \ --archive pulsar-io-mongo-2.4.0.nar \ --tenant public \ @@ -206,10 +251,14 @@ Pulsar admin CLI helps you debug Pulsar connectors with the following subcommand --name pulsar-mongo-sink \ --sink-config-file mongo-sink-config.yaml \ --parallelism 1 + ``` + ### `get` Use the `get` command to get the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, and so on. + ```bash + ./bin/pulsar-admin sinks get --tenant public --namespace default --name pulsar-mongo-sink { "tenant": "public", @@ -233,7 +282,9 @@ Use the `get` command to get the basic information about the Mongo sink connecto "retainOrdering": false, "autoAck": true } + ``` + :::tip For more information about the `get` command, see [`get`](reference-connector-admin.md/#get-1). @@ -242,7 +293,9 @@ For more information about the `get` command, see [`get`](reference-connector-ad ### `status` Use the `status` command to get the current status about the Mongo sink connector, such as the number of instance, the number of running instance, instanceId, workerId and so on. + ```bash + ./bin/pulsar-admin sinks status --tenant public \ --namespace default \ @@ -267,18 +320,21 @@ Use the `status` command to get the current status about the Mongo sink connecto } } ] } + ``` + :::tip For more information about the `status` command, see [`status`](reference-connector-admin.md/#stauts-1). - If there are multiple connectors running on a worker, `workerId` can locate the worker on which the specified connector is running. ::: ### `topics stats` Use the `topics stats` command to get the stats for a topic and its connected producer and consumer, such as whether the topic has received messages or not, whether there is a backlog of messages or not, the available permits and other key information. All rates are computed over a 1-minute window and are relative to the last completed 1-minute period. + ```bash + ./bin/pulsar-admin topics stats test-mongo { "msgRateIn" : 0.0, @@ -322,7 +378,9 @@ Use the `topics stats` command to get the stats for a topic and its connected pr "replication" : { }, "deduplicationStatus" : "Disabled" } + ``` + :::tip For more information about the `topic stats` command, see [`topic stats`](http://pulsar.apache.org/docs/en/pulsar-admin/#stats-1). diff --git a/site2/website-next/docs/io-develop.md b/site2/website-next/docs/io-develop.md index 3ab16013c9af6..316d164864fcc 100644 --- a/site2/website-next/docs/io-develop.md +++ b/site2/website-next/docs/io-develop.md @@ -1,7 +1,7 @@ --- id: io-develop title: How to develop Pulsar connectors -sidebar_label: Develop +sidebar_label: "Develop" --- import Tabs from '@theme/Tabs'; @@ -11,15 +11,15 @@ import TabItem from '@theme/TabItem'; This guide describes how to develop Pulsar connectors to move data between Pulsar and other systems. -Pulsar connectors are special [Pulsar Functions](functions-overview.md), so creating +Pulsar connectors are special [Pulsar Functions](functions-overview), so creating a Pulsar connector is similar to creating a Pulsar function. Pulsar connectors come in two types: | Type | Description | Example |---|---|--- -{@inject: github:Source:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java}|Import data from another system to Pulsar.|[RabbitMQ source connector](io-rabbitmq.md) imports the messages of a RabbitMQ queue to a Pulsar topic. -{@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java}|Export data from Pulsar to another system.|[Kinesis sink connector](io-kinesis.md) exports the messages of a Pulsar topic to a Kinesis stream. +{@inject: github:Source:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java}|Import data from another system to Pulsar.|[RabbitMQ source connector](io-rabbitmq) imports the messages of a RabbitMQ queue to a Pulsar topic. +{@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java}|Export data from Pulsar to another system.|[Kinesis sink connector](io-kinesis) exports the messages of a Pulsar topic to a Kinesis stream. ## Develop @@ -32,62 +32,66 @@ interface, which means you need to implement the {@inject: github:open:/pulsar-i 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java - /** - * Open connector with configuration - * - * @param config initialization config - * @param sourceContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SourceContext sourceContext) throws Exception; - ``` + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sourceContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SourceContext sourceContext) throws Exception; + + ``` - This method is called when the source connector is initialized. + This method is called when the source connector is initialized. - In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. - - For example, a Kafka connector can create a Kafka client in this `open` method. + In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. + + For example, a Kafka connector can create a Kafka client in this `open` method. - Besides, Pulsar runtime also provides a `SourceContext` for the - connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. + Besides, Pulsar runtime also provides a `SourceContext` for the + connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. 2. Implement the {@inject: github:read:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java - /** - * Reads the next message from source. - * If source does not have any new messages, this call should block. - * @return next message from source. The return result should never be null - * @throws Exception - */ - Record read() throws Exception; - ``` - - If nothing to return, the implementation should be blocking rather than returning `null`. - - The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: - - |Variable|Required|Description - |---|---|--- - `TopicName`|No|Pulsar topic name from which the record is originated from. - `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| - `Value`|Yes|Actual data of the record. - `EventTime`|No|Event time of the record from the source. - `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `Properties` |No| If the record carries user-defined properties, it returns those properties. - `DestinationTopic`|No|Topic to which message should be written. - `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: - - Method|Description - |---|--- - `ack` |Acknowledge that the record is fully processed. - `fail`|Indicate that the record fails to be processed. + ```java + + /** + * Reads the next message from source. + * If source does not have any new messages, this call should block. + * @return next message from source. The return result should never be null + * @throws Exception + */ + Record read() throws Exception; + + ``` + + If nothing to return, the implementation should be blocking rather than returning `null`. + + The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: + + |Variable|Required|Description + |---|---|--- + `TopicName`|No|Pulsar topic name from which the record is originated from. + `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| + `Value`|Yes|Actual data of the record. + `EventTime`|No|Event time of the record from the source. + `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `Properties` |No| If the record carries user-defined properties, it returns those properties. + `DestinationTopic`|No|Topic to which message should be written. + `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: + + Method|Description + |---|--- + `ack` |Acknowledge that the record is fully processed. + `fail`|Indicate that the record fails to be processed. ## Handle schema information @@ -95,13 +99,17 @@ Pulsar IO automatically handles the schema and provides a strongly typed API bas If you know the schema type that you are producing, you can declare the Java class relative to that type in your sink declaration. ``` + public class MySource implements Source { public Record read() {} } + ``` + If you want to implement a source that works with any schema, you can go with `byte[]` (of `ByteBuffer`) and use Schema.AUTO_PRODUCE_BYTES(). ``` + public class MySource implements Source { public Record read() { @@ -118,6 +126,7 @@ public class MySource implements Source { } } } + ``` To handle the `KeyValue` type properly, follow the guidelines for your record implementation: @@ -135,40 +144,43 @@ For more information about **how to create a source connector**, see {@inject: g ::: - ### Sink Developing a sink connector **is similar to** developing a source connector, that is, you need to implement the {@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} interface, which means implementing the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method and the {@inject: github:write:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - /** - * Open connector with configuration - * - * @param config initialization config - * @param sinkContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SinkContext sinkContext) throws Exception; - ``` + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sinkContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SinkContext sinkContext) throws Exception; + + ``` 2. Implement the {@inject: github:write:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - /** - * Write a message to Sink - * @param record record to write to sink - * @throws Exception - */ - void write(Record record) throws Exception; - ``` + ```java + + /** + * Write a message to Sink + * @param record record to write to sink + * @throws Exception + */ + void write(Record record) throws Exception; + + ``` - During the implementation, you can decide how to write the `Value` and - the `Key` to the actual source, and leverage all the provided information such as - `PartitionId` and `RecordSequence` to achieve different processing guarantees. + During the implementation, you can decide how to write the `Value` and + the `Key` to the actual source, and leverage all the provided information such as + `PartitionId` and `RecordSequence` to achieve different processing guarantees. - You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). + You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). ## Handling Schema information @@ -176,14 +188,17 @@ Pulsar IO handles automatically the Schema and provides a strongly typed API bas If you know the Schema type that you are consuming from you can declare the Java class relative to that type in your Sink declaration. ``` + public class MySink implements Sink { public void write(Record record) {} } + ``` If you want to implement a sink that works with any schema, you can you go with the special GenericObject interface. ``` + public class MySink implements Sink { public void write(Record record) { Schema schema = record.getSchema(); @@ -196,6 +211,7 @@ public class MySink implements Sink { .... } } + ``` In the case of AVRO, JSON, and Protobuf records (schemaType=AVRO,JSON,PROTOBUF_NATIVE), you can cast the @@ -205,6 +221,7 @@ You are able to access the native AVRO record using `genericObject.getNativeObj In the case of KeyValue type, you can access both the schema for the key and the schema for the value using this code. ``` + public class MySink implements Sink { public void write(Record record) { Schema schema = record.getSchema(); @@ -223,8 +240,8 @@ public class MySink implements Sink { .... } } -``` +``` ## Test @@ -244,8 +261,7 @@ You can create unit tests for your connector. Once you have written sufficient unit tests, you can add separate integration tests to verify end-to-end functionality. -Pulsar uses -[testcontainers](https://www.testcontainers.org/) **for all integration tests**. +Pulsar uses [testcontainers](https://www.testcontainers.org/) **for all integration tests**. :::tip @@ -253,11 +269,10 @@ For more information about **how to create integration tests for Pulsar connecto ::: - ## Package Once you've developed and tested your connector, you need to package it so that it can be submitted -to a [Pulsar Functions](functions-overview.md) cluster. +to a [Pulsar Functions](functions-overview) cluster. There are two methods to work with Pulsar Functions' runtime, that is, [NAR](#nar) and [uber JAR](#uber-jar). @@ -270,12 +285,8 @@ If you plan to package and distribute your connector for others to use, you are license and copyright your own code properly. Remember to add the license and copyright to all libraries your code uses and to your distribution. -:::note - -If you use the [NAR](#nar) method, the NAR plugin - -::: - +> +> If you use the [NAR](#nar) method, the NAR plugin automatically creates a `DEPENDENCIES` file in the generated NAR package, including the proper licensing and copyrights of all libraries of your connector. @@ -286,20 +297,18 @@ a bit of Java ClassLoader isolation. :::tip -For more information about **how NAR works**, see -[here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). +For more information about **how NAR works**, see [here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). ::: - Pulsar uses the same mechanism for packaging **all** [built-in connectors](io-connectors). -The easiest approach to package a Pulsar connector is to create a NAR package using -[nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin). +The easiest approach to package a Pulsar connector is to create a NAR package using [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin). Include this [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin) in your maven project for your connector as below. ```xml + org.apache.nifi @@ -307,15 +316,18 @@ Include this [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apac 1.2.0 + ``` You must also create a `resources/META-INF/services/pulsar-io.yaml` file with the following contents: ```yaml + name: connector name description: connector description sourceClass: fully qualified class name (only if source connector) sinkClass: fully qualified class name (only if sink connector) + ``` For Gradle users, there is a [Gradle Nar plugin available on the Gradle Plugin Portal](https://plugins.gradle.org/plugin/io.github.lhotari.gradle-nar-plugin). @@ -326,7 +338,6 @@ For more information about an **how to use NAR for Pulsar connectors**, see {@in ::: - ### Uber JAR An alternative approach is to create an **uber JAR** that contains all of the connector's JAR files @@ -335,6 +346,7 @@ and other resource files. No directory internal structure is necessary. You can use [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-plugin/examples/includes-excludes.html) to create a uber JAR as below: ```xml + org.apache.maven.plugins maven-shade-plugin @@ -355,6 +367,7 @@ You can use [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-pl + ``` ## Monitor @@ -363,7 +376,7 @@ Pulsar connectors enable you to move data in and out of Pulsar easily. It is imp - Check the metrics provided by Pulsar. - Pulsar connectors expose the metrics that can be collected and used for monitoring the health of **Java** connectors. You can check the metrics by following the [monitoring](deploy-monitoring.md) guide. + Pulsar connectors expose the metrics that can be collected and used for monitoring the health of **Java** connectors. You can check the metrics by following the [monitoring](deploy-monitoring) guide. - Set and check your customized metrics. @@ -382,6 +395,7 @@ Here is an example of how to customize metrics for a Java connector. ``` + public class TestMetricSink implements Sink { @Override @@ -399,6 +413,7 @@ public class TestMetricSink implements Sink { } } + ``` diff --git a/site2/website-next/docs/io-dynamodb-source.md b/site2/website-next/docs/io-dynamodb-source.md index bf171fabe5dab..a37b86590204b 100644 --- a/site2/website-next/docs/io-dynamodb-source.md +++ b/site2/website-next/docs/io-dynamodb-source.md @@ -1,7 +1,7 @@ --- id: io-dynamodb-source title: AWS DynamoDB source connector -sidebar_label: AWS DynamoDB source connector +sidebar_label: "AWS DynamoDB source connector" --- import Tabs from '@theme/Tabs'; @@ -23,19 +23,19 @@ The configuration of the DynamoDB source connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • +`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • `startAtTime`|Date|false|" " (empty string)|If set to `AT_TIMESTAMP`, it specifies the point in time to start consumption. -`applicationName`|String|false|Pulsar IO connector|The name of the KCL application. Must be unique, as it is used to define the table name for the dynamo table used for state tracking.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. +`applicationName`|String|false|Pulsar IO connector|The name of the KCL application. Must be unique, as it is used to define the table name for the dynamo table used for state tracking.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. `checkpointInterval`|long|false|60000|The frequency of the KCL checkpoint in milliseconds. `backoffTime`|long|false|3000|The amount of time to delay between requests when the connector encounters a throttling exception from AWS Kinesis in milliseconds. `numRetries`|int|false|3|The number of re-attempts when the connector encounters an exception while trying to set a checkpoint. -`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. +`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. `dynamoEndpoint`|String|false|" " (empty string)|The Dynamo end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `cloudwatchEndpoint`|String|false|" " (empty string)|The Cloudwatch end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `awsEndpoint`|String|false|" " (empty string)|The DynamoDB Streams end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsDynamodbStreamArn`|String|true|" " (empty string)|The DynamoDB stream arn. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Example @@ -44,36 +44,40 @@ Before using the DynamoDB source connector, you need to create a configuration f * JSON - ```json - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/docs/io-dynamodb.md b/site2/website-next/docs/io-dynamodb.md index 0c01ad1fceda0..5fba4c56ca772 100644 --- a/site2/website-next/docs/io-dynamodb.md +++ b/site2/website-next/docs/io-dynamodb.md @@ -1,7 +1,7 @@ --- id: io-dynamodb title: AWS DynamoDB Connector -sidebar_label: AWS DynamoDB Connector +sidebar_label: "AWS DynamoDB Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-elasticsearch-sink.md b/site2/website-next/docs/io-elasticsearch-sink.md index 2c39f87b1ac79..f2dd972ca7d71 100644 --- a/site2/website-next/docs/io-elasticsearch-sink.md +++ b/site2/website-next/docs/io-elasticsearch-sink.md @@ -1,7 +1,7 @@ --- id: io-elasticsearch-sink title: Elasticsearch sink connector -sidebar_label: Elasticsearch sink connector +sidebar_label: "Elasticsearch sink connector" --- import Tabs from '@theme/Tabs'; @@ -105,106 +105,141 @@ Before using the Elasticsearch sink connector, you need to create a configuratio * JSON - ```json - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "username": "scooby", - "password": "doobie" - } - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - username: "scooby" - password: "doobie" - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + username: "scooby" + password: "doobie" + + ``` #### For Elasticsearch Before 6.2 * JSON - ```json - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "typeName": "doc", - "username": "scooby", - "password": "doobie" - } - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "typeName": "doc", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - typeName: "doc" - username: "scooby" - password: "doobie" - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + typeName: "doc" + username: "scooby" + password: "doobie" + + ``` ### Usage 1. Start a single node Elasticsearch cluster. - ```bash - $ docker run -p 9200:9200 -p 9300:9300 \ - -e "discovery.type=single-node" \ - docker.elastic.co/elasticsearch/elasticsearch:7.13.3 - ``` + ```bash + + $ docker run -p 9200:9200 -p 9300:9300 \ + -e "discovery.type=single-node" \ + docker.elastic.co/elasticsearch/elasticsearch:7.13.3 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` - Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar`. + + ```bash + + $ bin/pulsar standalone + + ``` + + Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-@pulsar:version@.nar`. 3. Start the Pulsar Elasticsearch connector in local run mode using one of the following methods. - * Use the **JSON** configuration as shown previously. - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ - --inputs elasticsearch_test - ``` - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config-file elasticsearch-sink.yml \ - --inputs elasticsearch_test - ``` + * Use the **JSON** configuration as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ + --inputs elasticsearch_test + + ``` + + * Use the **YAML** configuration file as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config-file elasticsearch-sink.yml \ + --inputs elasticsearch_test + + ``` 4. Publish records to the topic. - ```bash - $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" - ``` + ```bash + + $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" + + ``` 5. Check documents in Elasticsearch. - - * refresh the index - ```bash - $ curl -s http://localhost:9200/my_index/_refresh - ``` - * search documents - ```bash - $ curl -s http://localhost:9200/my_index/_search - ``` - You can see the record that published earlier has been successfully written into Elasticsearch. - ```json - {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} - ``` + + * refresh the index + + ```bash + + $ curl -s http://localhost:9200/my_index/_refresh + + ``` + + + * search documents + + ```bash + + $ curl -s http://localhost:9200/my_index/_search + + ``` + + You can see the record that published earlier has been successfully written into Elasticsearch. + + ```json + + {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} + + ``` + diff --git a/site2/website-next/docs/io-elasticsearch.md b/site2/website-next/docs/io-elasticsearch.md index bb6440e477041..55ef0d9098286 100644 --- a/site2/website-next/docs/io-elasticsearch.md +++ b/site2/website-next/docs/io-elasticsearch.md @@ -1,7 +1,7 @@ --- id: io-elasticsearch title: ElasticSearch Connector -sidebar_label: ElasticSearch Connector +sidebar_label: "ElasticSearch Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-file-source.md b/site2/website-next/docs/io-file-source.md index 51582e8d47dc9..191d7a4a5e956 100644 --- a/site2/website-next/docs/io-file-source.md +++ b/site2/website-next/docs/io-file-source.md @@ -1,7 +1,7 @@ --- id: io-file-source title: File source connector -sidebar_label: File source connector +sidebar_label: "File source connector" --- import Tabs from '@theme/Tabs'; @@ -37,40 +37,44 @@ Before using the File source connector, you need to create a configuration file * JSON - ```json - { - "inputDirectory": "/Users/david", - "recurse": true, - "keepFile": true, - "fileFilter": "[^\\.].*", - "pathFilter": "*", - "minimumFileAge": 0, - "maximumFileAge": 9999999999, - "minimumSize": 1, - "maximumSize": 5000000, - "ignoreHiddenFiles": true, - "pollingInterval": 5000, - "numWorkers": 1 - } - ``` + ```json + + { + "inputDirectory": "/Users/david", + "recurse": true, + "keepFile": true, + "fileFilter": "[^\\.].*", + "pathFilter": "*", + "minimumFileAge": 0, + "maximumFileAge": 9999999999, + "minimumSize": 1, + "maximumSize": 5000000, + "ignoreHiddenFiles": true, + "pollingInterval": 5000, + "numWorkers": 1 + } + + ``` * YAML - ```yaml - configs: - inputDirectory: "/Users/david" - recurse: true - keepFile: true - fileFilter: "[^\\.].*" - pathFilter: "*" - minimumFileAge: 0 - maximumFileAge: 9999999999 - minimumSize: 1 - maximumSize: 5000000 - ignoreHiddenFiles: true - pollingInterval: 5000 - numWorkers: 1 - ``` + ```yaml + + configs: + inputDirectory: "/Users/david" + recurse: true + keepFile: true + fileFilter: "[^\\.].*" + pathFilter: "*" + minimumFileAge: 0 + maximumFileAge: 9999999999 + minimumSize: 1 + maximumSize: 5000000 + ignoreHiddenFiles: true + pollingInterval: 5000 + numWorkers: 1 + + ``` ## Usage @@ -78,64 +82,82 @@ Here is an example of using the File source connecter. 1. Pull a Pulsar image. - ```bash - $ docker pull apachepulsar/pulsar:{version} - ``` + ```bash + + $ docker pull apachepulsar/pulsar:{version} + + ``` 2. Start Pulsar standalone. + + ```bash - ```bash - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 3. Create a configuration file _file-connector.yaml_. - ```yaml - configs: - inputDirectory: "/opt" - ``` + ```yaml + + configs: + inputDirectory: "/opt" + + ``` 4. Copy the configuration file _file-connector.yaml_ to the container. - ```bash - $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ - ``` + ```bash + + $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ + + ``` 5. Download the File source connector. - ```bash - $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar - ``` + ```bash + + $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar + + ``` 6. Start the File source connector. - ```bash - $ docker exec -it pulsar-standalone /bin/bash + ```bash + + $ docker exec -it pulsar-standalone /bin/bash - $ ./bin/pulsar-admin sources localrun \ - --archive /pulsar/pulsar-io-file-{version}.nar \ - --name file-test \ - --destination-topic-name pulsar-file-test \ - --source-config-file /pulsar/file-connector.yaml - ``` + $ ./bin/pulsar-admin sources localrun \ + --archive /pulsar/pulsar-io-file-{version}.nar \ + --name file-test \ + --destination-topic-name pulsar-file-test \ + --source-config-file /pulsar/file-connector.yaml + + ``` 7. Start a consumer. - ```bash - ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test - ``` + ```bash + + ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test + + ``` 8. Write the message to the file _test.txt_. + + ```bash + + echo "hello world!" > /opt/test.txt - ```bash - echo "hello world!" > /opt/test.txt - ``` + ``` - The following information appears on the consumer terminal window. + The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello world! - ``` + ```bash + + ----- got message ----- + hello world! + + ``` - \ No newline at end of file + \ No newline at end of file diff --git a/site2/website-next/docs/io-file.md b/site2/website-next/docs/io-file.md index 94ed78d92a0a4..c1d405e789e65 100644 --- a/site2/website-next/docs/io-file.md +++ b/site2/website-next/docs/io-file.md @@ -1,7 +1,7 @@ --- id: io-file title: File Connector -sidebar_label: File Connector +sidebar_label: "File Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-flume-sink.md b/site2/website-next/docs/io-flume-sink.md index 1631b8b7ac697..5c6f8e2d0f8bd 100644 --- a/site2/website-next/docs/io-flume-sink.md +++ b/site2/website-next/docs/io-flume-sink.md @@ -1,7 +1,7 @@ --- id: io-flume-sink title: Flume sink connector -sidebar_label: Flume sink connector +sidebar_label: "Flume sink connector" --- import Tabs from '@theme/Tabs'; @@ -28,33 +28,32 @@ The configuration of the Flume sink connector has the following properties. Before using the Flume sink connector, you need to create a configuration file through one of the following methods. -:::note - -For more information about the `sink.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/sink.conf). - -::: - +> For more information about the `sink.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/sink.conf). * JSON - ```json - { - "name": "a1", - "confFile": "sink.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - ``` + ```json + + { + "name": "a1", + "confFile": "sink.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - configs: - name: a1 - confFile: sink.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - ``` + ```yaml + + configs: + name: a1 + confFile: sink.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/docs/io-flume-source.md b/site2/website-next/docs/io-flume-source.md index 5e886be682d4a..25a75cd8571f3 100644 --- a/site2/website-next/docs/io-flume-source.md +++ b/site2/website-next/docs/io-flume-source.md @@ -1,7 +1,7 @@ --- id: io-flume-source title: Flume source connector -sidebar_label: Flume source connector +sidebar_label: "Flume source connector" --- import Tabs from '@theme/Tabs'; @@ -28,33 +28,32 @@ The configuration of the Flume source connector has the following properties. Before using the Flume source connector, you need to create a configuration file through one of the following methods. -:::note - -For more information about the `source.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/source.conf). - -::: - +> For more information about the `source.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/source.conf). * JSON - ```json - { - "name": "a1", - "confFile": "source.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - ``` + ```json + + { + "name": "a1", + "confFile": "source.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - configs: - name: a1 - confFile: source.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - ``` + ```yaml + + configs: + name: a1 + confFile: source.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/docs/io-hbase-sink.md b/site2/website-next/docs/io-hbase-sink.md index 7a9885670de84..de9baf1c3a2d1 100644 --- a/site2/website-next/docs/io-hbase-sink.md +++ b/site2/website-next/docs/io-hbase-sink.md @@ -1,7 +1,7 @@ --- id: io-hbase-sink title: HBase sink connector -sidebar_label: HBase sink connector +sidebar_label: "HBase sink connector" --- import Tabs from '@theme/Tabs'; @@ -36,32 +36,35 @@ Before using the HBase sink connector, you need to create a configuration file t * JSON - ```json - { - "hbaseConfigResources": "hbase-site.xml", - "zookeeperQuorum": "localhost", - "zookeeperClientPort": "2181", - "zookeeperZnodeParent": "/hbase", - "tableName": "pulsar_hbase", - "rowKeyName": "rowKey", - "familyName": "info", - "qualifierNames": [ 'name', 'address', 'age'] - } - ``` - + ```json + + { + "hbaseConfigResources": "hbase-site.xml", + "zookeeperQuorum": "localhost", + "zookeeperClientPort": "2181", + "zookeeperZnodeParent": "/hbase", + "tableName": "pulsar_hbase", + "rowKeyName": "rowKey", + "familyName": "info", + "qualifierNames": [ 'name', 'address', 'age'] + } + + ``` * YAML - ```yaml - configs: - hbaseConfigResources: "hbase-site.xml" - zookeeperQuorum: "localhost" - zookeeperClientPort: "2181" - zookeeperZnodeParent: "/hbase" - tableName: "pulsar_hbase" - rowKeyName: "rowKey" - familyName: "info" - qualifierNames: [ 'name', 'address', 'age'] - ``` + ```yaml + + configs: + hbaseConfigResources: "hbase-site.xml" + zookeeperQuorum: "localhost" + zookeeperClientPort: "2181" + zookeeperZnodeParent: "/hbase" + tableName: "pulsar_hbase" + rowKeyName: "rowKey" + familyName: "info" + qualifierNames: [ 'name', 'address', 'age'] + + ``` - \ No newline at end of file + \ No newline at end of file diff --git a/site2/website-next/docs/io-hbase.md b/site2/website-next/docs/io-hbase.md index e3b128a8fece7..a54f2a70f9344 100644 --- a/site2/website-next/docs/io-hbase.md +++ b/site2/website-next/docs/io-hbase.md @@ -1,7 +1,7 @@ --- id: io-hbase title: hbase Connector -sidebar_label: hbase Connector +sidebar_label: "hbase Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-hdfs.md b/site2/website-next/docs/io-hdfs.md index c2a7e4e0af0f7..e4ef72e5a917e 100644 --- a/site2/website-next/docs/io-hdfs.md +++ b/site2/website-next/docs/io-hdfs.md @@ -1,7 +1,7 @@ --- id: io-hdfs title: Hdfs Connector -sidebar_label: Hdfs Connector +sidebar_label: "Hdfs Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-hdfs2-sink.md b/site2/website-next/docs/io-hdfs2-sink.md index 564a02f523043..db392867fda78 100644 --- a/site2/website-next/docs/io-hdfs2-sink.md +++ b/site2/website-next/docs/io-hdfs2-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs2-sink title: HDFS2 sink connector -sidebar_label: HDFS2 sink connector +sidebar_label: "HDFS2 sink connector" --- import Tabs from '@theme/Tabs'; @@ -19,18 +19,18 @@ The configuration of the HDFS2 sink connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | +| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | | `directory` | String | true | None|The HDFS directory where files read from or written to. | -| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | -| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | +| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | +| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | | `kerberosUserPrincipal` |String| false| None|The principal account of Kerberos user used for authentication. | | `keytab` | String|false|None| The full pathname of the Kerberos keytab file used for authentication. | -| `filenamePrefix` |String| true, if `compression` is set to `None`. | None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | -| `fileExtension` | String| true | None | The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | -| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | +| `filenamePrefix` |String| true, if `compression` is set to `None`. | None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | +| `fileExtension` | String| true | None | The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | +| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | | `syncInterval` | long| false |0| The interval between calls to flush data to HDFS disk in milliseconds. | -| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. -| `subdirectoryPattern` | String | false | None | A subdirectory associated with the created time of the sink.
    The pattern is the formatted pattern of `directory`'s subdirectory.

    See [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) for pattern's syntax. | +| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. +| `subdirectoryPattern` | String | false | None | A subdirectory associated with the created time of the sink.
    The pattern is the formatted pattern of `directory`'s subdirectory.

    See [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) for pattern's syntax. | ### Example @@ -38,25 +38,30 @@ Before using the HDFS2 sink connector, you need to create a configuration file t * JSON - ```json - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "fileExtension": ".log", - "compression": "SNAPPY", - "subdirectoryPattern": "yyyy-MM-dd" - } - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "fileExtension": ".log", + "compression": "SNAPPY", + "subdirectoryPattern": "yyyy-MM-dd" + } + + ``` * YAML - ```yaml - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - fileExtension: ".log" - compression: "SNAPPY" - subdirectoryPattern: "yyyy-MM-dd" - ``` + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + fileExtension: ".log" + compression: "SNAPPY" + subdirectoryPattern: "yyyy-MM-dd" + + ``` + diff --git a/site2/website-next/docs/io-hdfs3-sink.md b/site2/website-next/docs/io-hdfs3-sink.md index 01c3cdf6eb62d..fbbb3ee94b255 100644 --- a/site2/website-next/docs/io-hdfs3-sink.md +++ b/site2/website-next/docs/io-hdfs3-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs3-sink title: HDFS3 sink connector -sidebar_label: HDFS3 sink connector +sidebar_label: "HDFS3 sink connector" --- import Tabs from '@theme/Tabs'; @@ -19,17 +19,17 @@ The configuration of the HDFS3 sink connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | +| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | | `directory` | String | true | None|The HDFS directory where files read from or written to. | -| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | -| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | +| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | +| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | | `kerberosUserPrincipal` |String| false| None|The principal account of Kerberos user used for authentication. | | `keytab` | String|false|None| The full pathname of the Kerberos keytab file used for authentication. | -| `filenamePrefix` |String| false |None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | -| `fileExtension` | String| false | None| The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | -| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | +| `filenamePrefix` |String| false |None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | +| `fileExtension` | String| false | None| The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | +| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | | `syncInterval` | long| false |0| The interval between calls to flush data to HDFS disk in milliseconds. | -| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. +| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. ### Example @@ -37,21 +37,26 @@ Before using the HDFS3 sink connector, you need to create a configuration file t * JSON - ```json - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "compression": "SNAPPY" - } - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "compression": "SNAPPY" + } + + ``` * YAML - ```yaml - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - compression: "SNAPPY" - ``` + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + compression: "SNAPPY" + + ``` + diff --git a/site2/website-next/docs/io-influxdb-sink.md b/site2/website-next/docs/io-influxdb-sink.md index 74fb5aa4e21cb..f6912ed0478ef 100644 --- a/site2/website-next/docs/io-influxdb-sink.md +++ b/site2/website-next/docs/io-influxdb-sink.md @@ -1,7 +1,7 @@ --- id: io-influxdb-sink title: InfluxDB sink connector -sidebar_label: InfluxDB sink connector +sidebar_label: "InfluxDB sink connector" --- import Tabs from '@theme/Tabs'; @@ -49,63 +49,67 @@ The configuration of the InfluxDB sink connector has the following properties. Before using the InfluxDB sink connector, you need to create a configuration file through one of the following methods. #### InfluxDBv2 * JSON - ```json - { - "influxdbUrl": "http://localhost:9999", - "organization": "example-org", - "bucket": "example-bucket", - "token": "xxxx", - "precision": "ns", - "logLevel": "NONE", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - ``` + + ```json + + { + "influxdbUrl": "http://localhost:9999", + "organization": "example-org", + "bucket": "example-bucket", + "token": "xxxx", + "precision": "ns", + "logLevel": "NONE", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` + * YAML - ```yaml - { - influxdbUrl: "http://localhost:9999" - organization: "example-org" - bucket: "example-bucket" - token: "xxxx" - precision: "ns" - logLevel: "NONE" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } - ``` + ```yaml + configs: + influxdbUrl: "http://localhost:9999" + organization: "example-org" + bucket: "example-bucket" + token: "xxxx" + precision: "ns" + logLevel: "NONE" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + ``` #### InfluxDBv1 * JSON - ```json - { - "influxdbUrl": "http://localhost:8086", - "database": "test_db", - "consistencyLevel": "ONE", - "logLevel": "NONE", - "retentionPolicy": "autogen", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - ``` + ```json + + { + "influxdbUrl": "http://localhost:8086", + "database": "test_db", + "consistencyLevel": "ONE", + "logLevel": "NONE", + "retentionPolicy": "autogen", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` * YAML - ```yaml - { - influxdbUrl: "http://localhost:8086" - database: "test_db" - consistencyLevel: "ONE" - logLevel: "NONE" - retentionPolicy: "autogen" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } - ``` + ```yaml + configs: + influxdbUrl: "http://localhost:8086" + database: "test_db" + consistencyLevel: "ONE" + logLevel: "NONE" + retentionPolicy: "autogen" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + ``` diff --git a/site2/website-next/docs/io-influxdb.md b/site2/website-next/docs/io-influxdb.md index ea23ad32a7531..5a5f3d60e4420 100644 --- a/site2/website-next/docs/io-influxdb.md +++ b/site2/website-next/docs/io-influxdb.md @@ -1,7 +1,7 @@ --- id: io-influxdb title: InfluxDB Connector -sidebar_label: InfluxDB Connector +sidebar_label: "InfluxDB Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-jdbc-sink.md b/site2/website-next/docs/io-jdbc-sink.md index 7fa6afd8322a7..6b0672f634d5b 100644 --- a/site2/website-next/docs/io-jdbc-sink.md +++ b/site2/website-next/docs/io-jdbc-sink.md @@ -1,7 +1,7 @@ --- id: io-jdbc-sink title: JDBC sink connector -sidebar_label: JDBC sink connector +sidebar_label: "JDBC sink connector" --- import Tabs from '@theme/Tabs'; @@ -11,12 +11,7 @@ import TabItem from '@theme/TabItem'; The JDBC sink connectors allow pulling messages from Pulsar topics and persists the messages to ClickHouse, MariaDB, PostgreSQL, and SQLite. -:::note - -Currently, INSERT, DELETE and UPDATE operations are supported. - -::: - +> Currently, INSERT, DELETE and UPDATE operations are supported. ## Configuration @@ -39,57 +34,65 @@ The configuration of all JDBC sink connectors has the following properties. * JSON - ```json - { - "userName": "clickhouse", - "password": "password", - "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", - "tableName": "pulsar_clickhouse_jdbc_sink" - } - ``` + ```json + + { + "userName": "clickhouse", + "password": "password", + "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", + "tableName": "pulsar_clickhouse_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-clickhouse-sink" - topicName: "persistent://public/default/jdbc-clickhouse-topic" - sinkType: "jdbc-clickhouse" - configs: - userName: "clickhouse" - password: "password" - jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" - tableName: "pulsar_clickhouse_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-clickhouse-sink" + topicName: "persistent://public/default/jdbc-clickhouse-topic" + sinkType: "jdbc-clickhouse" + configs: + userName: "clickhouse" + password: "password" + jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" + tableName: "pulsar_clickhouse_jdbc_sink" + + ``` ### Example for MariaDB * JSON - ```json - { - "userName": "mariadb", - "password": "password", - "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", - "tableName": "pulsar_mariadb_jdbc_sink" - } - ``` + ```json + + { + "userName": "mariadb", + "password": "password", + "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", + "tableName": "pulsar_mariadb_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-mariadb-sink" - topicName: "persistent://public/default/jdbc-mariadb-topic" - sinkType: "jdbc-mariadb" - configs: - userName: "mariadb" - password: "password" - jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" - tableName: "pulsar_mariadb_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-mariadb-sink" + topicName: "persistent://public/default/jdbc-mariadb-topic" + sinkType: "jdbc-mariadb" + configs: + userName: "mariadb" + password: "password" + jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" + tableName: "pulsar_mariadb_jdbc_sink" + + ``` ### Example for PostgreSQL @@ -97,29 +100,33 @@ Before using the JDBC PostgreSQL sink connector, you need to create a configurat * JSON - ```json - { - "userName": "postgres", - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "tableName": "pulsar_postgres_jdbc_sink" - } - ``` + ```json + + { + "userName": "postgres", + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "tableName": "pulsar_postgres_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-postgres-sink" - topicName: "persistent://public/default/jdbc-postgres-topic" - sinkType: "jdbc-postgres" - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-postgres-sink" + topicName: "persistent://public/default/jdbc-postgres-topic" + sinkType: "jdbc-postgres" + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" + tableName: "pulsar_postgres_jdbc_sink" + + ``` For more information on **how to use this JDBC sink connector**, see [connect Pulsar to PostgreSQL](io-quickstart.md#connect-pulsar-to-postgresql). @@ -127,22 +134,27 @@ For more information on **how to use this JDBC sink connector**, see [connect Pu * JSON - ```json - { - "jdbcUrl": "jdbc:sqlite:db.sqlite", - "tableName": "pulsar_sqlite_jdbc_sink" - } - ``` + ```json + + { + "jdbcUrl": "jdbc:sqlite:db.sqlite", + "tableName": "pulsar_sqlite_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-sqlite-sink" - topicName: "persistent://public/default/jdbc-sqlite-topic" - sinkType: "jdbc-sqlite" - configs: - jdbcUrl: "jdbc:sqlite:db.sqlite" - tableName: "pulsar_sqlite_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-sqlite-sink" + topicName: "persistent://public/default/jdbc-sqlite-topic" + sinkType: "jdbc-sqlite" + configs: + jdbcUrl: "jdbc:sqlite:db.sqlite" + tableName: "pulsar_sqlite_jdbc_sink" + + ``` + diff --git a/site2/website-next/docs/io-jdbc.md b/site2/website-next/docs/io-jdbc.md index 6e4551cf1ae3f..ef09bcf215cf8 100644 --- a/site2/website-next/docs/io-jdbc.md +++ b/site2/website-next/docs/io-jdbc.md @@ -1,7 +1,7 @@ --- id: io-jdbc title: JDBC Connector -sidebar_label: JDBC Connector +sidebar_label: "JDBC Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-kafka-sink.md b/site2/website-next/docs/io-kafka-sink.md index 51a936896952f..2ca038dadffe0 100644 --- a/site2/website-next/docs/io-kafka-sink.md +++ b/site2/website-next/docs/io-kafka-sink.md @@ -1,7 +1,7 @@ --- id: io-kafka-sink title: Kafka sink connector -sidebar_label: Kafka sink connector +sidebar_label: "Kafka sink connector" --- import Tabs from '@theme/Tabs'; @@ -22,13 +22,13 @@ The configuration of the Kafka sink connector has the following parameters. | Name | Type| Required | Default | Description |------|----------|---------|-------------|-------------| | `bootstrapServers` |String| true | " " (empty string) | A comma-separated list of host and port pairs for establishing the initial connection to the Kafka cluster. | -|`acks`|String|true|" " (empty string) |The number of acknowledgments that the producer requires the leader to receive before a request completes.
    This controls the durability of the sent records. +|`acks`|String|true|" " (empty string) |The number of acknowledgments that the producer requires the leader to receive before a request completes.
    This controls the durability of the sent records. |`batchsize`|long|false|16384L|The batch size that a Kafka producer attempts to batch records together before sending them to brokers. |`maxRequestSize`|long|false|1048576L|The maximum size of a Kafka request in bytes. |`topic`|String|true|" " (empty string) |The Kafka topic which receives messages from Pulsar. | `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringSerializer | The serializer class for Kafka producers to serialize keys. -| `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArraySerializer | The serializer class for Kafka producers to serialize values.

    The serializer is set by a specific implementation of [`KafkaAbstractSink`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java). -|`producerConfigProperties`|Map|false|" " (empty string)|The producer configuration properties to be passed to producers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. +| `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArraySerializer | The serializer class for Kafka producers to serialize values.

    The serializer is set by a specific implementation of [`KafkaAbstractSink`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java). +|`producerConfigProperties`|Map|false|" " (empty string)|The producer configuration properties to be passed to producers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. ### Example @@ -37,36 +37,39 @@ Before using the Kafka sink connector, you need to create a configuration file t * JSON - ```json - { - "bootstrapServers": "localhost:6667", - "topic": "test", - "acks": "1", - "batchSize": "16384", - "maxRequestSize": "1048576", - "producerConfigProperties": - { - "client.id": "test-pulsar-producer", - "security.protocol": "SASL_PLAINTEXT", - "sasl.mechanism": "GSSAPI", - "sasl.kerberos.service.name": "kafka", - "acks": "all" - } - } + ```json + + { + "bootstrapServers": "localhost:6667", + "topic": "test", + "acks": "1", + "batchSize": "16384", + "maxRequestSize": "1048576", + "producerConfigProperties": + { + "client.id": "test-pulsar-producer", + "security.protocol": "SASL_PLAINTEXT", + "sasl.mechanism": "GSSAPI", + "sasl.kerberos.service.name": "kafka", + "acks": "all" + } + } * YAML - ```yaml - configs: - bootstrapServers: "localhost:6667" - topic: "test" - acks: "1" - batchSize: "16384" - maxRequestSize: "1048576" - producerConfigProperties: - client.id: "test-pulsar-producer" - security.protocol: "SASL_PLAINTEXT" - sasl.mechanism: "GSSAPI" - sasl.kerberos.service.name: "kafka" - acks: "all" - ``` + ``` + +yaml + configs: + bootstrapServers: "localhost:6667" + topic: "test" + acks: "1" + batchSize: "16384" + maxRequestSize: "1048576" + producerConfigProperties: + client.id: "test-pulsar-producer" + security.protocol: "SASL_PLAINTEXT" + sasl.mechanism: "GSSAPI" + sasl.kerberos.service.name: "kafka" + acks: "all" + ``` diff --git a/site2/website-next/docs/io-kafka-source.md b/site2/website-next/docs/io-kafka-source.md index b7a7993172a18..ce54394737c29 100644 --- a/site2/website-next/docs/io-kafka-source.md +++ b/site2/website-next/docs/io-kafka-source.md @@ -1,7 +1,7 @@ --- id: io-kafka-source title: Kafka source connector -sidebar_label: Kafka source connector +sidebar_label: "Kafka source connector" --- import Tabs from '@theme/Tabs'; @@ -24,13 +24,13 @@ The configuration of the Kafka source connector has the following properties. | `bootstrapServers` |String| true | " " (empty string) | A comma-separated list of host and port pairs for establishing the initial connection to the Kafka cluster. | | `groupId` |String| true | " " (empty string) | A unique string that identifies the group of consumer processes to which this consumer belongs. | | `fetchMinBytes` | long|false | 1 | The minimum byte expected for each fetch response. | -| `autoCommitEnabled` | boolean |false | true | If set to true, the consumer's offset is periodically committed in the background.

    This committed offset is used when the process fails as the position from which a new consumer begins. | +| `autoCommitEnabled` | boolean |false | true | If set to true, the consumer's offset is periodically committed in the background.

    This committed offset is used when the process fails as the position from which a new consumer begins. | | `autoCommitIntervalMs` | long|false | 5000 | The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if `autoCommitEnabled` is set to true. | -| `heartbeatIntervalMs` | long| false | 3000 | The interval between heartbeats to the consumer when using Kafka's group management facilities.

    **Note: `heartbeatIntervalMs` must be smaller than `sessionTimeoutMs`**.| +| `heartbeatIntervalMs` | long| false | 3000 | The interval between heartbeats to the consumer when using Kafka's group management facilities.

    **Note: `heartbeatIntervalMs` must be smaller than `sessionTimeoutMs`**.| | `sessionTimeoutMs` | long|false | 30000 | The timeout used to detect consumer failures when using Kafka's group management facility. | | `topic` | String|true | " " (empty string)| The Kafka topic which sends messages to Pulsar. | -| `consumerConfigProperties` | Map| false | " " (empty string) | The consumer configuration properties to be passed to consumers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. | -| `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringDeserializer | The deserializer class for Kafka consumers to deserialize keys.
    The deserializer is set by a specific implementation of [`KafkaAbstractSource`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java). +| `consumerConfigProperties` | Map| false | " " (empty string) | The consumer configuration properties to be passed to consumers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. | +| `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringDeserializer | The deserializer class for Kafka consumers to deserialize keys.
    The deserializer is set by a specific implementation of [`KafkaAbstractSource`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java). | `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArrayDeserializer | The deserializer class for Kafka consumers to deserialize values. | `autoOffsetReset` | String | false | "earliest" | The default offset reset policy. | @@ -70,26 +70,30 @@ Before using the Kafka source connector, you need to create a configuration file * JSON - ```json - { - "bootstrapServers": "pulsar-kafka:9092", - "groupId": "test-pulsar-io", - "topic": "my-topic", - "sessionTimeoutMs": "10000", - "autoCommitEnabled": false - } - ``` + ```json + + { + "bootstrapServers": "pulsar-kafka:9092", + "groupId": "test-pulsar-io", + "topic": "my-topic", + "sessionTimeoutMs": "10000", + "autoCommitEnabled": false + } + + ``` * YAML - ```yaml - configs: - bootstrapServers: "pulsar-kafka:9092" - groupId: "test-pulsar-io" - topic: "my-topic" - sessionTimeoutMs: "10000" - autoCommitEnabled: false - ``` + ```yaml + + configs: + bootstrapServers: "pulsar-kafka:9092" + groupId: "test-pulsar-io" + topic: "my-topic" + sessionTimeoutMs: "10000" + autoCommitEnabled: false + + ``` ## Usage @@ -97,106 +101,129 @@ Here is an example of using the Kafka source connector with the configuration fi 1. Download a Kafka client and a Kafka connector. - ```bash - $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar + ```bash + + $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar - ``` + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar + + ``` 2. Create a network. - + ```bash + $ docker network create kafka-pulsar + ``` 3. Pull a ZooKeeper image and start ZooKeeper. - + ```bash + $ docker pull wurstmeister/zookeeper $ docker run -d -it -p 2181:2181 --name pulsar-kafka-zookeeper --network kafka-pulsar wurstmeister/zookeeper + ``` 4. Pull a Kafka image and start Kafka. - + ```bash + $ docker pull wurstmeister/kafka:2.11-1.0.2 $ docker run -d -it --network kafka-pulsar -p 6667:6667 -p 9092:9092 -e KAFKA_ADVERTISED_HOST_NAME=pulsar-kafka -e KAFKA_ZOOKEEPER_CONNECT=pulsar-kafka-zookeeper:2181 --name pulsar-kafka wurstmeister/kafka:2.11-1.0.2 + ``` 5. Pull a Pulsar image and start Pulsar standalone. - + ```bash - $ docker pull apachepulsar/pulsar:{{pulsar:version}} + + $ docker pull apachepulsar/pulsar:@pulsar:version@ $ docker run -d -it --network kafka-pulsar -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-kafka-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + ``` 6. Create a producer file _kafka-producer.py_. - + ```python + from kafka import KafkaProducer producer = KafkaProducer(bootstrap_servers='pulsar-kafka:9092') future = producer.send('my-topic', b'hello world') future.get() + ``` 7. Create a consumer file _pulsar-client.py_. - ```python - import pulsar + ```python + + import pulsar - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', subscription_name='my-aa') + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', subscription_name='my-aa') - while True: - msg = consumer.receive() - print msg - print dir(msg) - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + while True: + msg = consumer.receive() + print msg + print dir(msg) + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) - client.close() - ``` + client.close() + + ``` 8. Copy the following files to Pulsar. + + ```bash + + $ docker cp pulsar-io-kafka-@pulsar:version@.nar pulsar-kafka-standalone:/pulsar + $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf + $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ + $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ - ```bash - $ docker cp pulsar-io-kafka-{{pulsar:version}}.nar pulsar-kafka-standalone:/pulsar - $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf - $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ - $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ - ``` + ``` 9. Open a new terminal window and start the Kafka source connector in local run mode. - ```bash - $ docker exec -it pulsar-kafka-standalone /bin/bash - - $ ./bin/pulsar-admin source localrun \ - --archive ./pulsar-io-kafka-{{pulsar:version}}.nar \ - --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ - --tenant public \ - --namespace default \ - --name kafka \ - --destination-topic-name my-topic \ - --source-config-file ./conf/kafkaSourceConfig.yaml \ - --parallelism 1 - ``` + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash + + $ ./bin/pulsar-admin source localrun \ + --archive ./pulsar-io-kafka-@pulsar:version@.nar \ + --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ + --tenant public \ + --namespace default \ + --name kafka \ + --destination-topic-name my-topic \ + --source-config-file ./conf/kafkaSourceConfig.yaml \ + --parallelism 1 + + ``` 10. Open a new terminal window and run the consumer. - ```bash - $ docker exec -it pulsar-kafka-standalone /bin/bash + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash - $ pip install kafka-python + $ pip install kafka-python - $ python3 kafka-producer.py - ``` + $ python3 kafka-producer.py + + ``` - The following information appears on the consumer terminal window. + The following information appears on the consumer terminal window. + + ```bash + + Received message: 'hello world' + + ``` - ```bash - Received message: 'hello world' - ``` diff --git a/site2/website-next/docs/io-kafka.md b/site2/website-next/docs/io-kafka.md index 3afed1fc38a09..6a212990e2418 100644 --- a/site2/website-next/docs/io-kafka.md +++ b/site2/website-next/docs/io-kafka.md @@ -1,7 +1,7 @@ --- id: io-kafka title: Kafka Connector -sidebar_label: Kafka Connector +sidebar_label: "Kafka Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-kinesis-sink.md b/site2/website-next/docs/io-kinesis-sink.md index c9cfa4439be6b..13523efcee5c8 100644 --- a/site2/website-next/docs/io-kinesis-sink.md +++ b/site2/website-next/docs/io-kinesis-sink.md @@ -1,7 +1,7 @@ --- id: io-kinesis-sink title: Kinesis sink connector -sidebar_label: Kinesis sink connector +sidebar_label: "Kinesis sink connector" --- import Tabs from '@theme/Tabs'; @@ -18,12 +18,12 @@ The configuration of the Kinesis sink connector has the following property. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`messageFormat`|MessageFormat|true|ONLY_RAW_PAYLOAD|Message format in which Kinesis sink converts Pulsar messages and publishes to Kinesis streams.

    Below are the available options:

  • `ONLY_RAW_PAYLOAD`: Kinesis sink directly publishes Pulsar message payload as a message into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_JSON`: Kinesis sink creates a JSON payload with Pulsar message payload, properties and encryptionCtx, and publishes JSON payload into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_FB`: Kinesis sink creates a flatbuffer serialized payload with Pulsar message payload, properties and encryptionCtx, and publishes flatbuffer payload into the configured Kinesis stream.
  • +`messageFormat`|MessageFormat|true|ONLY_RAW_PAYLOAD|Message format in which Kinesis sink converts Pulsar messages and publishes to Kinesis streams.

    Below are the available options:

  • `ONLY_RAW_PAYLOAD`: Kinesis sink directly publishes Pulsar message payload as a message into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_JSON`: Kinesis sink creates a JSON payload with Pulsar message payload, properties and encryptionCtx, and publishes JSON payload into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_FB`: Kinesis sink creates a flatbuffer serialized payload with Pulsar message payload, properties and encryptionCtx, and publishes flatbuffer payload into the configured Kinesis stream.
  • `retainOrdering`|boolean|false|false|Whether Pulsar connectors to retain ordering when moving messages from Pulsar to Kinesis or not. `awsEndpoint`|String|false|" " (empty string)|The Kinesis end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsKinesisStreamName`|String|true|" " (empty string)|The Kinesis stream name. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    It is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If it is empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`. +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    It is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If it is empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`. `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Built-in plugins @@ -32,19 +32,21 @@ The following are built-in `AwsCredentialProviderPlugin` plugins: * `org.apache.pulsar.io.aws.AwsDefaultProviderChainPlugin` - This plugin takes no configuration, it uses the default AWS provider chain. - - For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). + This plugin takes no configuration, it uses the default AWS provider chain. + + For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). * `org.apache.pulsar.io.aws.STSAssumeRoleProviderPlugin` - This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. + This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. - This configuration takes the form of a small json document like: + This configuration takes the form of a small json document like: - ```json - {"roleArn": "arn...", "roleSessionName": "name"} - ``` + ```json + + {"roleArn": "arn...", "roleSessionName": "name"} + + ``` ### Example @@ -52,25 +54,30 @@ Before using the Kinesis sink connector, you need to create a configuration file * JSON - ```json - { - "awsEndpoint": "some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "messageFormat": "ONLY_RAW_PAYLOAD", - "retainOrdering": "true" - } - ``` + ```json + + { + "awsEndpoint": "some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "messageFormat": "ONLY_RAW_PAYLOAD", + "retainOrdering": "true" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - messageFormat: "ONLY_RAW_PAYLOAD" - retainOrdering: "true" - ``` + ```yaml + + configs: + awsEndpoint: "some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + messageFormat: "ONLY_RAW_PAYLOAD" + retainOrdering: "true" + + ``` + diff --git a/site2/website-next/docs/io-kinesis-source.md b/site2/website-next/docs/io-kinesis-source.md index f380eaf86e656..1402839d1b975 100644 --- a/site2/website-next/docs/io-kinesis-source.md +++ b/site2/website-next/docs/io-kinesis-source.md @@ -1,7 +1,7 @@ --- id: io-kinesis-source title: Kinesis source connector -sidebar_label: Kinesis source connector +sidebar_label: "Kinesis source connector" --- import Tabs from '@theme/Tabs'; @@ -12,12 +12,7 @@ The Kinesis source connector pulls data from Amazon Kinesis and persists data in This connector uses the [Kinesis Consumer Library](https://github.com/awslabs/amazon-kinesis-client) (KCL) to do the actual consuming of messages. The KCL uses DynamoDB to track state for consumers. -:::note - -: currently, the Kinesis source connector only supports raw messages. If you use KMS encrypted messages, the encrypted messages are sent to downstream. This connector will support decrypting messages in the future release. - -::: - +> Note: currently, the Kinesis source connector only supports raw messages. If you use KMS encrypted messages, the encrypted messages are sent to downstream. This connector will support decrypting messages in the future release. ## Configuration @@ -28,20 +23,20 @@ The configuration of the Kinesis source connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • +`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • `startAtTime`|Date|false|" " (empty string)|If set to `AT_TIMESTAMP`, it specifies the point in time to start consumption. -`applicationName`|String|false|Pulsar IO connector|The name of the Amazon Kinesis application.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. +`applicationName`|String|false|Pulsar IO connector|The name of the Amazon Kinesis application.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. `checkpointInterval`|long|false|60000|The frequency of the Kinesis stream checkpoint in milliseconds. `backoffTime`|long|false|3000|The amount of time to delay between requests when the connector encounters a throttling exception from AWS Kinesis in milliseconds. `numRetries`|int|false|3|The number of re-attempts when the connector encounters an exception while trying to set a checkpoint. -`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. +`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. `dynamoEndpoint`|String|false|" " (empty string)|The Dynamo end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `cloudwatchEndpoint`|String|false|" " (empty string)|The Cloudwatch end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `useEnhancedFanOut`|boolean|false|true|If set to true, it uses Kinesis enhanced fan-out.

    If set to false, it uses polling. `awsEndpoint`|String|false|" " (empty string)|The Kinesis end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsKinesisStreamName`|String|true|" " (empty string)|The Kinesis stream name. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Example @@ -50,36 +45,40 @@ Before using the Kinesis source connector, you need to create a configuration fi * JSON - ```json - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/docs/io-kinesis.md b/site2/website-next/docs/io-kinesis.md index 5d96f3bd70999..1b18b6526deab 100644 --- a/site2/website-next/docs/io-kinesis.md +++ b/site2/website-next/docs/io-kinesis.md @@ -1,7 +1,7 @@ --- id: io-kinesis title: AWS Kinesis Connector -sidebar_label: AWS Kinesis Connector +sidebar_label: "AWS Kinesis Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-mongo-sink.md b/site2/website-next/docs/io-mongo-sink.md index fa7d43f8e53d1..aea2ad05de2ca 100644 --- a/site2/website-next/docs/io-mongo-sink.md +++ b/site2/website-next/docs/io-mongo-sink.md @@ -1,7 +1,7 @@ --- id: io-mongo-sink title: MongoDB sink connector -sidebar_label: MongoDB sink connector +sidebar_label: "MongoDB sink connector" --- import Tabs from '@theme/Tabs'; @@ -31,25 +31,30 @@ The configuration of the MongoDB sink connector has the following properties. Before using the Mongo sink connector, you need to create a configuration file through one of the following methods. * JSON + + ```json + + { + "mongoUri": "mongodb://localhost:27017", + "database": "pulsar", + "collection": "messages", + "batchSize": "2", + "batchTimeMs": "500" + } - ```json - { - "mongoUri": "mongodb://localhost:27017", - "database": "pulsar", - "collection": "messages", - "batchSize": "2", - "batchTimeMs": "500" - } - ``` + ``` * YAML + + ```yaml - ```yaml - { - mongoUri: "mongodb://localhost:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - } - ``` + { + mongoUri: "mongodb://localhost:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + } + + ``` + diff --git a/site2/website-next/docs/io-mongo.md b/site2/website-next/docs/io-mongo.md index bc5079277df36..f0b5f0bc69357 100644 --- a/site2/website-next/docs/io-mongo.md +++ b/site2/website-next/docs/io-mongo.md @@ -1,7 +1,7 @@ --- id: io-mongo title: MongoDB Connector -sidebar_label: MongoDB Connector +sidebar_label: "MongoDB Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-netty-source.md b/site2/website-next/docs/io-netty-source.md index b0a25fe17910d..5f045658a2284 100644 --- a/site2/website-next/docs/io-netty-source.md +++ b/site2/website-next/docs/io-netty-source.md @@ -1,7 +1,7 @@ --- id: io-netty-source title: Netty source connector -sidebar_label: Netty source connector +sidebar_label: "Netty source connector" --- import Tabs from '@theme/Tabs'; @@ -33,25 +33,28 @@ Before using the Netty source connector, you need to create a configuration file * JSON - ```json - { - "type": "tcp", - "host": "127.0.0.1", - "port": "10911", - "numberOfThreads": "1" - } - ``` + ```json + + { + "type": "tcp", + "host": "127.0.0.1", + "port": "10911", + "numberOfThreads": "1" + } + + ``` * YAML - ```yaml - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` - + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` ## Usage @@ -61,148 +64,181 @@ The following examples show how to use the Netty source connector with TCP and H 1. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:{version} + ```bash + + $ docker pull apachepulsar/pulsar:{version} - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 2. Create a configuration file _netty-source-config.yaml_. + + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 - ```yaml - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` + ``` 3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. - ```bash - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ - ``` + ```bash + + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar - ``` - -5. Start the Netty source connector. + ```bash + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + + ``` + +5. Start the Netty source connector. + ```bash + $ ./bin/pulsar-admin sources localrun \ - --archive pulsar-io-{{pulsar:version}}.nar \ + --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ --namespace default \ --name netty \ --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 + ``` 6. Consume data. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ apt-get update - - $ apt-get -y install telnet - - $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 - Trying 127.0.0.1... - Connected to 127.0.0.1. - Escape character is '^]'. - hello - world - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ apt-get update + + $ apt-get -y install telnet + + $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 + Trying 127.0.0.1... + Connected to 127.0.0.1. + Escape character is '^]'. + hello + world + + ``` 8. The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello + ```bash + + ----- got message ----- + hello - ----- got message ----- - world - ``` + ----- got message ----- + world + + ``` ### HTTP 1. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:{version} + ```bash + + $ docker pull apachepulsar/pulsar:{version} - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 2. Create a configuration file _netty-source-config.yaml_. + + ```yaml - ```yaml - configs: - type: "http" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` + configs: + type: "http" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` 3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. + + ```bash - ```bash - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ - ``` + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar - ``` - -5. Start the Netty source connector. + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + ``` + +5. Start the Netty source connector. + ```bash + $ ./bin/pulsar-admin sources localrun \ - --archive pulsar-io-{{pulsar:version}}.nar \ + --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ --namespace default \ --name netty \ --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 + ``` 6. Consume data. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ + + ``` 8. The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello, world! - ``` + ```bash + + ----- got message ----- + hello, world! + + ``` + diff --git a/site2/website-next/docs/io-netty.md b/site2/website-next/docs/io-netty.md index 4d11d6bcabd40..928a480a644ba 100644 --- a/site2/website-next/docs/io-netty.md +++ b/site2/website-next/docs/io-netty.md @@ -1,7 +1,7 @@ --- id: io-netty title: Netty Tcp or Udp Connector -sidebar_label: Netty Tcp or Udp Connector +sidebar_label: "Netty Tcp or Udp Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-nsq-source.md b/site2/website-next/docs/io-nsq-source.md index f7e59964a4fcc..7c39ea92d9d8d 100644 --- a/site2/website-next/docs/io-nsq-source.md +++ b/site2/website-next/docs/io-nsq-source.md @@ -1,7 +1,7 @@ --- id: io-nsq-source title: NSQ source connector -sidebar_label: NSQ source connector +sidebar_label: "NSQ source connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-nsq.md b/site2/website-next/docs/io-nsq.md index 301f6d54eaa37..8bb98affed8dc 100644 --- a/site2/website-next/docs/io-nsq.md +++ b/site2/website-next/docs/io-nsq.md @@ -1,7 +1,7 @@ --- id: io-nsq title: NSQ Connector -sidebar_label: NSQ Connector +sidebar_label: "NSQ Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-overview.md b/site2/website-next/docs/io-overview.md index e6b541dd91d81..640a84fa6683a 100644 --- a/site2/website-next/docs/io-overview.md +++ b/site2/website-next/docs/io-overview.md @@ -1,7 +1,7 @@ --- id: io-overview title: Pulsar connector overview -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -24,12 +24,7 @@ This diagram illustrates the relationship between source, Pulsar, and sink: ### Source -:::note - -Sources **feed data from external systems into Pulsar**. - -::: - +> Sources **feed data from external systems into Pulsar**. Common sources include other messaging systems and firehose-style data pipeline APIs. @@ -37,12 +32,7 @@ For the complete list of Pulsar built-in source connectors, see [source connecto ### Sink -:::note - -Sinks **feed data from Pulsar into external systems**. - -::: - +> Sinks **feed data from Pulsar into external systems**. Common sinks include other messaging systems and SQL and NoSQL databases. @@ -52,12 +42,7 @@ For the complete list of Pulsar built-in sink connectors, see [sink connector](i Processing guarantees are used to handle errors when writing messages to Pulsar topics. -:::note - -Pulsar connectors and Functions use the **same** processing guarantees as below. - -::: - +> Pulsar connectors and Functions use the **same** processing guarantees as below. Delivery semantic | Description :------------------|:------- @@ -65,12 +50,7 @@ Delivery semantic | Description `at-least-once` | Each message sent to a connector is to be **processed once** or **more than once**. `effectively-once` | Each message sent to a connector has **one output associated** with it. -:::note - -Processing guarantees for connectors not just rely on Pulsar guarantee but also **relate to external systems**, that is, **the implementation of source and sink**. - -::: - +> Processing guarantees for connectors not just rely on Pulsar guarantee but also **relate to external systems**, that is, **the implementation of source and sink**. * Source: Pulsar ensures that writing messages to Pulsar topics respects to the processing guarantees. It is within Pulsar's control. @@ -86,12 +66,7 @@ When creating a connector, you can set the processing guarantee with the followi * EFFECTIVELY_ONCE -:::note - -If `--processing-guarantees` is not specified when creating a connector, the default semantic is `ATLEAST_ONCE`. - -::: - +> If `--processing-guarantees` is not specified when creating a connector, the default semantic is `ATLEAST_ONCE`. Here takes **Admin CLI** as an example. For more information about **REST API** or **JAVA Admin API**, see [here](io-use.md#create). @@ -111,9 +86,11 @@ Here takes **Admin CLI** as an example. For more information about **REST API** ```bash + $ bin/pulsar-admin sources create \ --processing-guarantees ATMOST_ONCE \ # Other source configs + ``` For more information about the options of `pulsar-admin sources create`, see [here](reference-connector-admin.md#create). @@ -122,9 +99,11 @@ For more information about the options of `pulsar-admin sources create`, see [he ```bash + $ bin/pulsar-admin sinks create \ --processing-guarantees EFFECTIVELY_ONCE \ # Other sink configs + ``` For more information about the options of `pulsar-admin sinks create`, see [here](reference-connector-admin.md#create-1). @@ -161,9 +140,11 @@ Here takes **Admin CLI** as an example. For more information about **REST API** ```bash + $ bin/pulsar-admin sources update \ --processing-guarantees EFFECTIVELY_ONCE \ # Other source configs + ``` For more information about the options of `pulsar-admin sources update`, see [here](reference-connector-admin.md#update). @@ -172,9 +153,11 @@ For more information about the options of `pulsar-admin sources update`, see [he ```bash + $ bin/pulsar-admin sinks update \ --processing-guarantees ATMOST_ONCE \ # Other sink configs + ``` For more information about the options of `pulsar-admin sinks update`, see [here](reference-connector-admin.md#update-1). @@ -186,7 +169,7 @@ For more information about the options of `pulsar-admin sinks update`, see [here ## Work with connector -You can manage Pulsar connectors (for example, create, update, start, stop, restart, reload, delete and perform other operations on connectors) via the [Connector Admin CLI](reference-connector-admin.md) with [sources](io-cli.md#sources) and [sinks](io-cli.md#sinks) subcommands. +You can manage Pulsar connectors (for example, create, update, start, stop, restart, reload, delete and perform other operations on connectors) via the [Connector Admin CLI](reference-connector-admin) with [sources](io-cli.md#sources) and [sinks](io-cli.md#sinks) subcommands. -Connectors (sources and sinks) and Functions are components of instances, and they all run on Functions workers. When managing a source, sink or function via [Connector Admin CLI](reference-connector-admin.md) or [Functions Admin CLI](functions-cli.md), an instance is started on a worker. For more information, see [Functions worker](functions-worker.md#run-functions-worker-separately). +Connectors (sources and sinks) and Functions are components of instances, and they all run on Functions workers. When managing a source, sink or function via [Connector Admin CLI](reference-connector-admin.md) or [Functions Admin CLI](functions-cli), an instance is started on a worker. For more information, see [Functions worker](functions-worker.md#run-functions-worker-separately). diff --git a/site2/website-next/docs/io-quickstart.md b/site2/website-next/docs/io-quickstart.md index d2fbf4599c98c..7f1efb6d54dfc 100644 --- a/site2/website-next/docs/io-quickstart.md +++ b/site2/website-next/docs/io-quickstart.md @@ -1,7 +1,7 @@ --- id: io-quickstart title: How to connect Pulsar to database -sidebar_label: Get started +sidebar_label: "Get started" --- import Tabs from '@theme/Tabs'; @@ -10,7 +10,7 @@ import TabItem from '@theme/TabItem'; This tutorial provides a hands-on look at how you can move data out of Pulsar without writing a single line of code. -It is helpful to review the [concepts](io-overview.md) for Pulsar I/O with running the steps in this guide to gain a deeper understanding. +It is helpful to review the [concepts](io-overview) for Pulsar I/O with running the steps in this guide to gain a deeper understanding. At the end of this tutorial, you are able to: @@ -20,14 +20,12 @@ At the end of this tutorial, you are able to: :::tip -* These instructions assume you are running Pulsar in [standalone mode](getting-started-standalone.md). However, all +* These instructions assume you are running Pulsar in [standalone mode](getting-started-standalone). However, all the commands used in this tutorial can be used in a multi-nodes Pulsar cluster without any changes. - * All the instructions are assumed to run at the root directory of a Pulsar binary distribution. ::: - ## Install Pulsar and built-in connector Before connecting Pulsar to a database, you need to install Pulsar and the desired built-in connector. @@ -38,56 +36,74 @@ For more information about **how to install a standalone Pulsar and built-in con 1. Start Pulsar locally. - ```bash - bin/pulsar standalone - ``` + ```bash + + bin/pulsar standalone + + ``` - All the components of a Pulsar service are start in order. - - You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. + All the components of a Pulsar service are start in order. + + You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. 2. Check Pulsar binary protocol port. - ```bash - telnet localhost 6650 - ``` + ```bash + + telnet localhost 6650 + + ``` 3. Check Pulsar Function cluster. - ```bash - curl -s http://localhost:8080/admin/v2/worker/cluster - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/worker/cluster + + ``` + + **Example output** - **Example output** - ```json - [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] - ``` + ```json + + [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] + + ``` 4. Make sure a public tenant and a default namespace exist. - ```bash - curl -s http://localhost:8080/admin/v2/namespaces/public - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/namespaces/public + + ``` - **Example output** - ```json - ["public/default","public/functions"] - ``` + **Example output** + + ```json + + ["public/default","public/functions"] + + ``` 5. All built-in connectors should be listed as available. - ```bash - curl -s http://localhost:8080/admin/v2/functions/connectors - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/functions/connectors + + ``` - **Example output** + **Example output** - ```json - [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] - ``` + ```json + + [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] + + ``` - If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, - or you can navigate to the `logs` directory under the Pulsar directory to view the logs. + If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, + or you can navigate to the `logs` directory under the Pulsar directory to view the logs. ## Connect Pulsar to Cassandra @@ -96,80 +112,93 @@ This section demonstrates how to connect Pulsar to Cassandra. :::tip * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - -* The Cassandra sink connector reads messages from Pulsar topics and writes the messages into Cassandra tables. For more information, see [Cassandra sink connector](io-cassandra-sink.md). +* The Cassandra sink connector reads messages from Pulsar topics and writes the messages into Cassandra tables. For more information, see [Cassandra sink connector](io-cassandra-sink). ::: - ### Setup a Cassandra cluster This example uses `cassandra` Docker image to start a single-node Cassandra cluster in Docker. 1. Start a Cassandra cluster. - ```bash - docker run -d --rm --name=cassandra -p 9042:9042 cassandra - ``` - - :::note + ```bash + + docker run -d --rm --name=cassandra -p 9042:9042 cassandra + + ``` - Before moving to the next steps, make sure the Cassandra cluster is running. + :::note - ::: + Before moving to the next steps, make sure the Cassandra cluster is running. + ::: 2. Make sure the Docker process is running. - ```bash - docker ps - ``` + ```bash + + docker ps + + ``` 3. Check the Cassandra logs to make sure the Cassandra process is running as expected. - ```bash - docker logs cassandra - ``` + ```bash + + docker logs cassandra + + ``` 4. Check the status of the Cassandra cluster. - ```bash - docker exec cassandra nodetool status - ``` + ```bash + + docker exec cassandra nodetool status + + ``` - **Example output** + **Example output** - ``` - Datacenter: datacenter1 - ======================= - Status=Up/Down - |/ State=Normal/Leaving/Joining/Moving - -- Address Load Tokens Owns (effective) Host ID Rack - UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 - ``` + ``` + + Datacenter: datacenter1 + ======================= + Status=Up/Down + |/ State=Normal/Leaving/Joining/Moving + -- Address Load Tokens Owns (effective) Host ID Rack + UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 + + ``` 5. Use `cqlsh` to connect to the Cassandra cluster. - ```bash - $ docker exec -ti cassandra cqlsh localhost - Connected to Test Cluster at localhost:9042. - [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] - Use HELP for help. - cqlsh> - ``` + ```bash + + $ docker exec -ti cassandra cqlsh localhost + Connected to Test Cluster at localhost:9042. + [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] + Use HELP for help. + cqlsh> + + ``` 6. Create a keyspace `pulsar_test_keyspace`. - ```bash - cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; - ``` + ```bash + + cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; + + ``` 7. Create a table `pulsar_test_table`. - ```bash - cqlsh> USE pulsar_test_keyspace; - cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); - ``` + ```bash + + cqlsh> USE pulsar_test_keyspace; + cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); + + ``` ### Configure a Cassandra sink @@ -185,44 +214,45 @@ You can create a configuration file through one of the following methods. * JSON - ```json - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - ``` + ```json + + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` * YAML + + ```yaml - ```yaml - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" - ``` + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" -For more information, see [Cassandra sink connector](io-cassandra-sink.md). + ``` + +For more information, see [Cassandra sink connector](io-cassandra-sink). ### Create a Cassandra sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to create a sink connector and perform other operations on them. Run the following command to create a Cassandra sink connector with sink type _cassandra_ and the config file _examples/cassandra-sink.yml_ created previously. #### Note -:::note - -The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file. - -::: +> The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file. ```bash + bin/pulsar-admin sinks create \ --tenant public \ --namespace default \ @@ -230,6 +260,7 @@ bin/pulsar-admin sinks create \ --sink-type cassandra \ --sink-config-file examples/cassandra-sink.yml \ --inputs test_cassandra + ``` Once the command is executed, Pulsar creates the sink connector _cassandra-test-sink_. @@ -239,160 +270,180 @@ as a Pulsar Function and writes the messages produced in the topic _test_cassand ### Inspect a Cassandra sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to monitor a connector and perform other operations on it. * Get the information of a Cassandra sink. ```bash + bin/pulsar-admin sinks get \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json + { - "tenant": "public", - "namespace": "default", - "name": "cassandra-test-sink", - "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", - "inputSpecs": { - "test_cassandra": { - "isRegexPattern": false - } - }, - "configs": { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true, - "archive": "builtin://cassandra" + "tenant": "public", + "namespace": "default", + "name": "cassandra-test-sink", + "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", + "inputSpecs": { + "test_cassandra": { + "isRegexPattern": false + } + }, + "configs": { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true, + "archive": "builtin://cassandra" } + ``` * Check the status of a Cassandra sink. ```bash + bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] } + ``` ### Verify a Cassandra sink 1. Produce some messages to the input topic of the Cassandra sink _test_cassandra_. - ```bash - for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done - ``` + ```bash + + for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done + + ``` 2. Inspect the status of the Cassandra sink _test_cassandra_. - ```bash - bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink - ``` - - You can see 10 messages are processed by the Cassandra sink _test_cassandra_. - - **Example output** - - ```json - { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 10, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 10, - "lastReceivedTime" : 1551685489136, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] - } - ``` + ```bash + + bin/pulsar-admin sinks status \ + --tenant public \ + --namespace default \ + --name cassandra-test-sink + + ``` + + You can see 10 messages are processed by the Cassandra sink _test_cassandra_. + + **Example output** + + ```json + + { + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 10, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 10, + "lastReceivedTime" : 1551685489136, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] + } + + ``` 3. Use `cqlsh` to connect to the Cassandra cluster. ```bash + docker exec -ti cassandra cqlsh localhost + ``` 4. Check the data of the Cassandra table _pulsar_test_table_. ```bash + cqlsh> use pulsar_test_keyspace; cqlsh:pulsar_test_keyspace> select * from pulsar_test_table; key | col --------+-------- - key-5 | key-5 - key-0 | key-0 - key-9 | key-9 - key-2 | key-2 - key-1 | key-1 - key-3 | key-3 - key-6 | key-6 - key-7 | key-7 - key-4 | key-4 - key-8 | key-8 + key-5 | key-5 + key-0 | key-0 + key-9 | key-9 + key-2 | key-2 + key-1 | key-1 + key-3 | key-3 + key-6 | key-6 + key-7 | key-7 + key-4 | key-4 + key-8 | key-8 + ``` ### Delete a Cassandra Sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to delete a connector and perform other operations on it. ```bash + bin/pulsar-admin sinks delete \ --tenant public \ --namespace default \ --name cassandra-test-sink + ``` ## Connect Pulsar to PostgreSQL @@ -402,18 +453,12 @@ This section demonstrates how to connect Pulsar to PostgreSQL. :::tip * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - * The JDBC sink connector pulls messages from Pulsar topics ::: and persists the messages to ClickHouse, MariaDB, PostgreSQL, or SQlite. -:::note - -For more information, see [JDBC sink connector](io-jdbc-sink.md). - -::: - +>For more information, see [JDBC sink connector](io-jdbc-sink). ### Setup a PostgreSQL cluster @@ -422,73 +467,84 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS 1. Pull the PostgreSQL 12 image from Docker. - ```bash - $ docker pull postgres:12 - ``` + ```bash + + $ docker pull postgres:12 + + ``` 2. Start PostgreSQL. - ```bash - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=password \ - -e POSTGRES_USER=postgres \ - postgres:12 - ``` - - #### Tip - - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=password \ + -e POSTGRES_USER=postgres \ + postgres:12 + + ``` + + #### Tip + + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. :::tip - For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). ::: - 3. Check if PostgreSQL has been started successfully. - ```bash - $ docker logs -f pulsar-postgres - ``` + ```bash + + $ docker logs -f pulsar-postgres + + ``` - PostgreSQL has been started successfully if the following message appears. + PostgreSQL has been started successfully if the following message appears. - ```text - 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 - 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" - 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC - 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections - ``` + ```text + + 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 + 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" + 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC + 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections + + ``` 4. Access to PostgreSQL. - ```bash - $ docker exec -it pulsar-postgres /bin/bash - ``` + ```bash + + $ docker exec -it pulsar-postgres /bin/bash + + ``` 5. Create a PostgreSQL table _pulsar_postgres_jdbc_sink_. - ```bash - $ psql -U postgres postgres - - postgres=# create table if not exists pulsar_postgres_jdbc_sink - ( - id serial PRIMARY KEY, - name VARCHAR(255) NOT NULL - ); - ``` + ```bash + + $ psql -U postgres postgres + + postgres=# create table if not exists pulsar_postgres_jdbc_sink + ( + id serial PRIMARY KEY, + name VARCHAR(255) NOT NULL + ); + + ``` ### Configure a JDBC sink @@ -498,74 +554,84 @@ In this section, you need to configure a JDBC sink connector. 1. Add a configuration file. - To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. - - For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. + To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. + + For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. - Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. + Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - ```yaml - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" - ``` + ```yaml + + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/postgres" + tableName: "pulsar_postgres_jdbc_sink" + + ``` 2. Create a schema. - Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. + Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - ```json - { - "type": "AVRO", - "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", - "properties": {} - } - ``` - - :::tip - - For more information about AVRO, see [Apache Avro](https://avro.apache.org/docs/1.9.1/). + ```json + + { + "type": "AVRO", + "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", + "properties": {} + } + + ``` - ::: + :::tip + For more information about AVRO, see [Apache Avro](https://avro.apache.org/docs/1.9.1/). + ::: 3. Upload a schema to a topic. - This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. + This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. - ```bash - $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema - ``` + ```bash + + $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema + + ``` 4. Check if the schema has been uploaded successfully. - ```bash - $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic - ``` + ```bash + + $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic + + ``` - The schema has been uploaded successfully if the following message appears. + The schema has been uploaded successfully if the following message appears. - ```json - {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} - ``` + ```json + + {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} + + ``` ### Create a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to create a sink connector and perform other operations on it. This example creates a sink connector and specifies the desired information. ```bash + $ bin/pulsar-admin sinks create \ ---archive ./connectors/pulsar-io-jdbc-postgres-{{pulsar:version}}.nar \ +--archive ./connectors/pulsar-io-jdbc-postgres-@pulsar:version@.nar \ --inputs pulsar-postgres-jdbc-sink-topic \ --name pulsar-postgres-jdbc-sink \ --sink-config-file ./connectors/pulsar-postgres-jdbc-sink.yaml \ --parallelism 1 + ``` Once the command is executed, Pulsar creates a sink connector _pulsar-postgres-jdbc-sink_. @@ -576,7 +642,7 @@ This sink connector runs as a Pulsar Function and writes the messages produced i Flag | Description | This example ---|---|---| - `--archive` | The path to the archive file for the sink. | _pulsar-io-jdbc-postgres-{{pulsar:version}}.nar_ | + `--archive` | The path to the archive file for the sink. | _pulsar-io-jdbc-postgres-@pulsar:version@.nar_ | `--inputs` | The input topic(s) of the sink.

    Multiple topics can be specified as a comma-separated list.|| `--name` | The name of the sink. | _pulsar-postgres-jdbc-sink_ | `--sink-config-file` | The path to a YAML config file specifying the configuration of the sink. | _pulsar-postgres-jdbc-sink.yaml_ | @@ -588,24 +654,27 @@ For more information about `pulsar-admin sinks create options`, see [here](io-cl ::: - The sink has been created successfully if the following message appears. ```bash + "Created successfully" + ``` ### Inspect a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to monitor a connector and perform other operations on it. * List all running JDBC sink(s). ```bash + $ bin/pulsar-admin sinks list \ --tenant public \ --namespace default + ``` :::tip @@ -614,22 +683,25 @@ to monitor a connector and perform other operations on it. ::: - The result shows that only the _postgres-jdbc-sink_ sink is running. ```json + [ "pulsar-postgres-jdbc-sink" ] + ``` * Get the information of a JDBC sink. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -638,40 +710,43 @@ to monitor a connector and perform other operations on it. ::: - The result shows the information of the sink connector, including tenant, namespace, topic and so on. ```json + { - "tenant": "public", - "namespace": "default", - "name": "pulsar-postgres-jdbc-sink", - "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", - "inputSpecs": { - "pulsar-postgres-jdbc-sink-topic": { - "isRegexPattern": false - } - }, - "configs": { - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "userName": "postgres", - "tableName": "pulsar_postgres_jdbc_sink" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true + "tenant": "public", + "namespace": "default", + "name": "pulsar-postgres-jdbc-sink", + "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", + "inputSpecs": { + "pulsar-postgres-jdbc-sink-topic": { + "isRegexPattern": false + } + }, + "configs": { + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "userName": "postgres", + "tableName": "pulsar_postgres_jdbc_sink" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true } + ``` * Get the status of a JDBC sink ```bash + $ bin/pulsar-admin sinks status \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -680,42 +755,45 @@ to monitor a connector and perform other operations on it. ::: - The result shows the current status of sink connector, including the number of instance, running status, worker ID and so on. ```json + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-192.168.2.52-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-192.168.2.52-8080" + } + } ] } + ``` ### Stop a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to stop a connector and perform other operations on it. ```bash + $ bin/pulsar-admin sinks stop \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -724,23 +802,26 @@ For more information about `pulsar-admin sinks stop options`, see [here](io-cli. ::: - The sink instance has been stopped successfully if the following message disappears. ```bash + "Stopped successfully" + ``` ### Restart a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to restart a connector and perform other operations on it. ```bash + $ bin/pulsar-admin sinks restart \ --tenant public \ --namespace default \ ---name pulsar-postgres-jdbc-sink +--name pulsar-postgres-jdbc-sink + ``` :::tip @@ -749,35 +830,35 @@ For more information about `pulsar-admin sinks restart options`, see [here](io-c ::: - The sink instance has been started successfully if the following message disappears. ```bash + "Started successfully" + ``` :::tip * Optionally, you can run a standalone sink connector using `pulsar-admin sinks localrun options`. - Note that `pulsar-admin sinks localrun options` **runs a sink connector locally**, while `pulsar-admin sinks start options` **starts a sink connector in a cluster**. - * For more information about `pulsar-admin sinks localrun options`, see [here](io-cli.md#localrun-1). ::: - ### Update a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to update a connector and perform other operations on it. This example updates the parallelism of the _pulsar-postgres-jdbc-sink_ sink connector to 2. ```bash + $ bin/pulsar-admin sinks update \ --name pulsar-postgres-jdbc-sink \ --parallelism 2 + ``` :::tip @@ -786,25 +867,29 @@ For more information about `pulsar-admin sinks update options`, see [here](io-cl ::: - The sink connector has been updated successfully if the following message disappears. ```bash + "Updated successfully" + ``` This example double-checks the information. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` The result shows that the parallelism is 2. ```json + { "tenant": "public", "namespace": "default", @@ -826,20 +911,23 @@ The result shows that the parallelism is 2. "retainOrdering": false, "autoAck": true } + ``` ### Delete a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to delete a connector and perform other operations on it. This example deletes the _pulsar-postgres-jdbc-sink_ sink connector. ```bash + $ bin/pulsar-admin sinks delete \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -848,26 +936,32 @@ For more information about `pulsar-admin sinks delete options`, see [here](io-cl ::: - The sink connector has been deleted successfully if the following message appears. ```text + "Deleted successfully" + ``` This example double-checks the status of the sink connector. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` The result shows that the sink connector does not exist. ```text + HTTP 404 Not Found Reason: Sink pulsar-postgres-jdbc-sink doesn't exist + ``` + diff --git a/site2/website-next/docs/io-rabbitmq-sink.md b/site2/website-next/docs/io-rabbitmq-sink.md index 215a64dce0f70..131d015219496 100644 --- a/site2/website-next/docs/io-rabbitmq-sink.md +++ b/site2/website-next/docs/io-rabbitmq-sink.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-sink title: RabbitMQ sink connector -sidebar_label: RabbitMQ sink connector +sidebar_label: "RabbitMQ sink connector" --- import Tabs from '@theme/Tabs'; @@ -43,42 +43,46 @@ Before using the RabbitMQ sink connector, you need to create a configuration fil * JSON - ```json - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "exchangeName": "test-exchange", - "routingKey": "test-key" - } - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "exchangeName": "test-exchange", + "routingKey": "test-key" + } + + ``` * YAML - ```yaml - configs: - host: "localhost" - port: 5672 - virtualHost: "/", - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - exchangeName: "test-exchange" - routingKey: "test-key" - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/", + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + exchangeName: "test-exchange" + routingKey: "test-key" + + ``` diff --git a/site2/website-next/docs/io-rabbitmq-source.md b/site2/website-next/docs/io-rabbitmq-source.md index 190e03649a905..9fcf2c0f417a7 100644 --- a/site2/website-next/docs/io-rabbitmq-source.md +++ b/site2/website-next/docs/io-rabbitmq-source.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-source title: RabbitMQ source connector -sidebar_label: RabbitMQ source connector +sidebar_label: "RabbitMQ source connector" --- import Tabs from '@theme/Tabs'; @@ -41,44 +41,48 @@ Before using the RabbitMQ source connector, you need to create a configuration f * JSON - ```json - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "prefetchCount": "0", - "prefetchGlobal": "false", - "passive": "false" - } - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "prefetchCount": "0", + "prefetchGlobal": "false", + "passive": "false" + } + + ``` * YAML - ```yaml - configs: - host: "localhost" - port: 5672 - virtualHost: "/" - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - prefetchCount: 0 - prefetchGlobal: "false" - passive: "false" - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/" + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + prefetchCount: 0 + prefetchGlobal: "false" + passive: "false" + + ``` diff --git a/site2/website-next/docs/io-rabbitmq.md b/site2/website-next/docs/io-rabbitmq.md index a8a179da46dfa..96345cc2ea17b 100644 --- a/site2/website-next/docs/io-rabbitmq.md +++ b/site2/website-next/docs/io-rabbitmq.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq title: RabbitMQ Connector -sidebar_label: RabbitMQ Connector +sidebar_label: "RabbitMQ Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-redis-sink.md b/site2/website-next/docs/io-redis-sink.md index c52489dc73660..e2638c4eb14ca 100644 --- a/site2/website-next/docs/io-redis-sink.md +++ b/site2/website-next/docs/io-redis-sink.md @@ -1,7 +1,7 @@ --- id: io-redis-sink title: Redis sink connector -sidebar_label: Redis sink connector +sidebar_label: "Redis sink connector" --- import Tabs from '@theme/Tabs'; @@ -43,97 +43,117 @@ Before using the Redis sink connector, you need to create a configuration file i * JSON - ```json - { - "redisHosts": "localhost:6379", - "redisPassword": "mypassword", - "redisDatabase": "0", - "clientMode": "Standalone", - "operationTimeout": "2000", - "batchSize": "1", - "batchTimeMs": "1000", - "connectTimeout": "3000" - } - ``` + ```json + + { + "redisHosts": "localhost:6379", + "redisPassword": "mypassword", + "redisDatabase": "0", + "clientMode": "Standalone", + "operationTimeout": "2000", + "batchSize": "1", + "batchTimeMs": "1000", + "connectTimeout": "3000" + } + + ``` * YAML - ```yaml - configs: - redisHosts: "localhost:6379" - redisPassword: "mypassword" - redisDatabase: 0 - clientMode: "Standalone" - operationTimeout: 2000 - batchSize: 1 - batchTimeMs: 1000 - connectTimeout: 3000 - ``` + ```yaml + + configs: + redisHosts: "localhost:6379" + redisPassword: "mypassword" + redisDatabase: 0 + clientMode: "Standalone" + operationTimeout: 2000 + batchSize: 1 + batchTimeMs: 1000 + connectTimeout: 3000 + ``` + ### Usage This example shows how to write records to a Redis database using the Pulsar Redis connector. 1. Start a Redis server. - ```bash - $ docker pull redis:5.0.5 - $ docker run -d -p 6379:6379 --name my-redis redis:5.0.5 --requirepass "mypassword" - ``` + ```bash + + $ docker pull redis:5.0.5 + $ docker run -d -p 6379:6379 --name my-redis redis:5.0.5 --requirepass "mypassword" + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` - Make sure the NAR file is available at `connectors/pulsar-io-redis-{{pulsar:version}}.nar`. + ```bash + + $ bin/pulsar standalone + + ``` + + Make sure the NAR file is available at `connectors/pulsar-io-redis-@pulsar:version@.nar`. 3. Start the Pulsar Redis connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-redis-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name my-redis-sink \ - --sink-config '{"redisHosts": "localhost:6379","redisPassword": "mypassword","redisDatabase": "0","clientMode": "Standalone","operationTimeout": "3000","batchSize": "1"}' \ - --inputs my-redis-topic - ``` - + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-redis-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name my-redis-sink \ + --sink-config '{"redisHosts": "localhost:6379","redisPassword": "mypassword","redisDatabase": "0","clientMode": "Standalone","operationTimeout": "3000","batchSize": "1"}' \ + --inputs my-redis-topic + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-redis-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name my-redis-sink \ - --sink-config-file redis-sink-config.yaml \ - --inputs my-redis-topic - ``` - + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-redis-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name my-redis-sink \ + --sink-config-file redis-sink-config.yaml \ + --inputs my-redis-topic + + ``` + 4. Publish records to the topic. - ```bash - $ bin/pulsar-client produce \ - persistent://public/default/my-redis-topic \ - -k "streaming" \ - -m "Pulsar" - ``` + ```bash + + $ bin/pulsar-client produce \ + persistent://public/default/my-redis-topic \ + -k "streaming" \ + -m "Pulsar" + + ``` 5. Start a Redis client in Docker. + + ```bash - ```bash - $ docker exec -it my-redis redis-cli -a "mypassword" - ``` + $ docker exec -it my-redis redis-cli -a "mypassword" + ``` + 6. Check the key/value in Redis. - ``` - 127.0.0.1:6379> keys * - 1) "streaming" - 127.0.0.1:6379> get "streaming" - "Pulsar" - ``` \ No newline at end of file + ``` + + 127.0.0.1:6379> keys * + 1) "streaming" + 127.0.0.1:6379> get "streaming" + "Pulsar" + + ``` + diff --git a/site2/website-next/docs/io-redis.md b/site2/website-next/docs/io-redis.md index af9057b404195..ff6fafd103a38 100644 --- a/site2/website-next/docs/io-redis.md +++ b/site2/website-next/docs/io-redis.md @@ -1,7 +1,7 @@ --- id: io-redis title: redis Connector -sidebar_label: redis Connector +sidebar_label: "redis Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-solr-sink.md b/site2/website-next/docs/io-solr-sink.md index 6bc3e40c12222..89e1289c1fc73 100644 --- a/site2/website-next/docs/io-solr-sink.md +++ b/site2/website-next/docs/io-solr-sink.md @@ -1,7 +1,7 @@ --- id: io-solr-sink title: Solr sink connector -sidebar_label: Solr sink connector +sidebar_label: "Solr sink connector" --- import Tabs from '@theme/Tabs'; @@ -38,27 +38,31 @@ Before using the Solr sink connector, you need to create a configuration file th * JSON - ```json - { - "solrUrl": "localhost:2181,localhost:2182/chroot", - "solrMode": "SolrCloud", - "solrCollection": "techproducts", - "solrCommitWithinMs": 100, - "username": "fakeuser", - "password": "fake@123" - } - ``` + ```json + + { + "solrUrl": "localhost:2181,localhost:2182/chroot", + "solrMode": "SolrCloud", + "solrCollection": "techproducts", + "solrCommitWithinMs": 100, + "username": "fakeuser", + "password": "fake@123" + } + + ``` * YAML - ```yaml - { - solrUrl: "localhost:2181,localhost:2182/chroot" - solrMode: "SolrCloud" - solrCollection: "techproducts" - solrCommitWithinMs: 100 - username: "fakeuser" - password: "fake@123" - } - ``` + ```yaml + + { + solrUrl: "localhost:2181,localhost:2182/chroot" + solrMode: "SolrCloud" + solrCollection: "techproducts" + solrCommitWithinMs: 100 + username: "fakeuser" + password: "fake@123" + } + + ``` diff --git a/site2/website-next/docs/io-solr.md b/site2/website-next/docs/io-solr.md index 2a194c18153f7..1405e699b6a16 100644 --- a/site2/website-next/docs/io-solr.md +++ b/site2/website-next/docs/io-solr.md @@ -1,7 +1,7 @@ --- id: io-solr title: solr Connector -sidebar_label: solr Connector +sidebar_label: "solr Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-tcp.md b/site2/website-next/docs/io-tcp.md index 174bc93b75e24..8cd32078d538e 100644 --- a/site2/website-next/docs/io-tcp.md +++ b/site2/website-next/docs/io-tcp.md @@ -1,7 +1,7 @@ --- id: io-tcp title: Netty Tcp Connector -sidebar_label: Netty Tcp Connector +sidebar_label: "Netty Tcp Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-twitter-source.md b/site2/website-next/docs/io-twitter-source.md index f179ea3bfeb27..3fa0db4784785 100644 --- a/site2/website-next/docs/io-twitter-source.md +++ b/site2/website-next/docs/io-twitter-source.md @@ -1,7 +1,7 @@ --- id: io-twitter-source title: Twitter Firehose source connector -sidebar_label: Twitter Firehose source connector +sidebar_label: "Twitter Firehose source connector" --- import Tabs from '@theme/Tabs'; @@ -28,9 +28,4 @@ The configuration of the Twitter Firehose source connector has the following pro | `clientHosts` |String| false | Constants.STREAM_HOST | The twitter firehose hosts to which client connects. | | `clientBufferSize` | int|false | 50000 | The buffer size for buffering tweets fetched from twitter firehose. | -:::note - -For more information about OAuth credentials, see [Twitter developers portal](https://developer.twitter.com/en.html). - -::: - +> For more information about OAuth credentials, see [Twitter developers portal](https://developer.twitter.com/en.html). diff --git a/site2/website-next/docs/io-twitter.md b/site2/website-next/docs/io-twitter.md index 7476d8fa1d3cc..57b8b3df5c008 100644 --- a/site2/website-next/docs/io-twitter.md +++ b/site2/website-next/docs/io-twitter.md @@ -1,7 +1,7 @@ --- id: io-twitter title: Twitter Firehose Connector -sidebar_label: Twitter Firehose Connector +sidebar_label: "Twitter Firehose Connector" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/io-use.md b/site2/website-next/docs/io-use.md index 594f68974a40f..4957184c123d9 100644 --- a/site2/website-next/docs/io-use.md +++ b/site2/website-next/docs/io-use.md @@ -1,7 +1,7 @@ --- id: io-use title: How to use Pulsar connectors -sidebar_label: Use +sidebar_label: "Use" --- import Tabs from '@theme/Tabs'; @@ -12,7 +12,7 @@ This guide describes how to use Pulsar connectors. ## Install a connector -Pulsar bundles several [builtin connectors](io-connectors.md) used to move data in and out of commonly used systems (such as database and messaging system). Optionally, you can create and use your desired non-builtin connectors. +Pulsar bundles several [builtin connectors](io-connectors) used to move data in and out of commonly used systems (such as database and messaging system). Optionally, you can create and use your desired non-builtin connectors. :::note @@ -20,7 +20,6 @@ When using a non-builtin connector, you need to specify the path of a archive fi ::: - To set up a builtin connector, follow the instructions [here](getting-started-standalone.md#installing-builtin-connectors). @@ -43,11 +42,13 @@ To configure a default folder for builtin connectors, set the `connectorsDirecto Set the `./connectors` folder as the default storage location for builtin connectors. ``` + ######################## # Connectors ######################## connectorsDirectory: ./connectors + ``` ### Configure a connector with a YAML file @@ -67,6 +68,7 @@ Below is a YAML configuration file of a Cassandra sink, which tells Pulsar: * How to map Pulsar messages into Cassandra table key and columns ```shell + tenant: public namespace: default name: cassandra-test-sink @@ -78,6 +80,7 @@ configs: columnFamily: "pulsar_test_table" keyname: "key" columnName: "col" + ``` **Example 2** @@ -85,12 +88,14 @@ configs: Below is a YAML configuration file of a Kafka source. ```shell + configs: bootstrapServers: "pulsar-kafka:9092" groupId: "test-pulsar-io" topic: "my-topic" sessionTimeoutMs: "10000" autoCommitEnabled: "false" + ``` **Example 3** @@ -98,11 +103,13 @@ configs: Below is a YAML configuration file of a PostgreSQL JDBC sink. ```shell + configs: userName: "postgres" password: "password" jdbcUrl: "jdbc:postgresql://localhost:5432/test_jdbc" tableName: "test_jdbc" + ``` ## Get available connectors @@ -122,7 +129,9 @@ If you add or delete a nar file in a connector folder, reload the available buil Use the `reload` subcommand. ```shell + $ pulsar-admin sources reload + ``` For more information, see [`here`](io-cli.md#reload). @@ -132,7 +141,9 @@ For more information, see [`here`](io-cli.md#reload). Use the `reload` subcommand. ```shell + $ pulsar-admin sinks reload + ``` For more information, see [`here`](io-cli.md#reload-1). @@ -146,7 +157,9 @@ After reloading connectors (optional), you can get a list of available connector Use the `available-sources` subcommand. ```shell + $ pulsar-admin sources available-sources + ``` #### Sink @@ -154,7 +167,9 @@ $ pulsar-admin sources available-sources Use the `available-sinks` subcommand. ```shell + $ pulsar-admin sinks available-sinks + ``` ## Run a connector @@ -197,7 +212,9 @@ Create a source connector. Use the `create` subcommand. ``` + $ pulsar-admin sources create options + ``` For more information, see [here](io-cli.md#create). @@ -212,56 +229,60 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/source * Create a source connector with a **local file**. - ```java - void createSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void createSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sourceConfig` | The source configuration object + |Name|Description + |---|--- + `sourceConfig` | The source configuration object **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). + For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). * Create a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void createSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void createSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sourceConfig` | The source configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sourceConfig` | The source configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-). + For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-).
    @@ -293,7 +314,9 @@ Create a sink connector. Use the `create` subcommand. ``` + $ pulsar-admin sinks create options + ``` For more information, see [here](io-cli.md#create-1). @@ -307,57 +330,61 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/ * Create a sink connector with a **local file**. + + ```java - ```java - void createSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - ``` + void createSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sinkConfig` | The sink configuration object + |Name|Description + |---|--- + `sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). + For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). * Create a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void createSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void createSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sinkConfig` | The sink configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sinkConfig` | The sink configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). + For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). @@ -389,7 +416,9 @@ Start a source connector. Use the `start` subcommand. ``` + $ pulsar-admin sources start options + ``` For more information, see [here](io-cli.md#start). @@ -399,11 +428,11 @@ For more information, see [here](io-cli.md#start). * Start **all** source connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} * Start a **specified** source connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@}
    @@ -431,7 +460,9 @@ Start a sink connector. Use the `start` subcommand. ``` + $ pulsar-admin sinks start options + ``` For more information, see [here](io-cli.md#start-1). @@ -441,11 +472,11 @@ For more information, see [here](io-cli.md#start-1). * Start **all** sink connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} * Start a **specified** sink connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@}
    @@ -473,7 +504,9 @@ Run a source connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sources localrun options + ``` For more information, see [here](io-cli.md#localrun). @@ -500,7 +533,9 @@ Run a sink connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sinks localrun options + ``` For more information, see [here](io-cli.md#localrun-1). @@ -549,7 +584,9 @@ Get the information of a source connector. Use the `get` subcommand. ``` + $ pulsar-admin sources get options + ``` For more information, see [here](io-cli.md#get). @@ -563,10 +600,12 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sources/ ```java + SourceConfig getSource(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Example** @@ -574,6 +613,7 @@ SourceConfig getSource(String tenant, This is a sourceConfig. ```java + { "tenant": "tenantName", "namespace": "namespaceName", @@ -589,11 +629,13 @@ This is a sourceConfig. "disk": 10737418240 } } + ``` This is a sourceConfig example. ``` + { "tenant": "public", "namespace": "default", @@ -623,6 +665,7 @@ This is a sourceConfig example. "disk": 10737418240 } } + ``` **Exception** @@ -665,7 +708,9 @@ Get the information of a sink connector. Use the `get` subcommand. ``` + $ pulsar-admin sinks get options + ``` For more information, see [here](io-cli.md#get-1). @@ -679,10 +724,12 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sinks/:t ```java + SinkConfig getSink(String tenant, String namespace, String sink) throws PulsarAdminException + ``` **Example** @@ -690,6 +737,7 @@ SinkConfig getSink(String tenant, This is a sinkConfig. ```json + { "tenant": "tenantName", "namespace": "namespaceName", @@ -706,11 +754,13 @@ This is a sinkConfig. "retainOrdering": false, "autoAck": true } + ``` This is a sinkConfig example. ```json + { "tenant": "public", "namespace": "default", @@ -732,6 +782,7 @@ This is a sinkConfig example. "retainOrdering": false, "autoAck": true } + ``` **Parameter description** @@ -778,7 +829,9 @@ Get the list of all running source connectors. Use the `list` subcommand. ``` + $ pulsar-admin sources list options + ``` For more information, see [here](io-cli.md#list). @@ -792,15 +845,19 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sources/ ```java + List listSources(String tenant, String namespace) throws PulsarAdminException + ``` **Response example** ```java + ["f1", "f2", "f3"] + ``` **Exception** @@ -842,7 +899,9 @@ Get the list of all running sink connectors. Use the `list` subcommand. ``` + $ pulsar-admin sinks list options + ``` For more information, see [here](io-cli.md#list-1). @@ -856,15 +915,19 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sinks/:t ```java + List listSinks(String tenant, String namespace) throws PulsarAdminException + ``` **Response example** ```java + ["f1", "f2", "f3"] + ``` **Exception** @@ -910,7 +973,9 @@ Get the current status of a source connector. Use the `status` subcommand. ``` + $ pulsar-admin sources status options + ``` For more information, see [here](io-cli.md#status). @@ -931,55 +996,59 @@ For more information, see [here](io-cli.md#status). * Get the current status of **all** source connectors. - ```java - SourceStatus getSourceStatus(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + SourceStatus getSourceStatus(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, - String namespace, - String source, - int id) - throws PulsarAdminException - ``` + ```java + + SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, + String namespace, + String source, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Source instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Source instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1011,7 +1080,9 @@ Get the current status of a Pulsar sink connector. Use the `status` subcommand. ``` + $ pulsar-admin sinks status options + ``` For more information, see [here](io-cli.md#status-1). @@ -1032,55 +1103,59 @@ For more information, see [here](io-cli.md#status-1). * Get the current status of **all** sink connectors. - ```java - SinkStatus getSinkStatus(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + SinkStatus getSinkStatus(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, - String namespace, - String sink, - int id) - throws PulsarAdminException - ``` + ```java + + SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, + String namespace, + String sink, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Sink instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Sink instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1118,7 +1193,9 @@ Update a running Pulsar source connector. Use the `update` subcommand. ``` + $ pulsar-admin sources update options + ``` For more information, see [here](io-cli.md#update). @@ -1133,58 +1210,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sources/ * Update a running source connector with a **local file**. - ```java - void updateSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void updateSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sourceConfig` | The source configuration object + | Name | Description + |---|--- + |`sourceConfig` | The source configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). + For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). * Update a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void updateSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void updateSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sourceConfig` | The source configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sourceConfig` | The source configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSourceWithUrl-SourceConfig-java.lang.String-). @@ -1218,7 +1299,9 @@ Update a running Pulsar sink connector. Use the `update` subcommand. ``` + $ pulsar-admin sinks update options + ``` For more information, see [here](io-cli.md#update-1). @@ -1233,58 +1316,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sinks/:t * Update a running sink connector with a **local file**. - ```java - void updateSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void updateSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sinkConfig` | The sink configuration object + | Name | Description + |---|--- + |`sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). + For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). * Update a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void updateSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void updateSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sinkConfig` | The sink configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sinkConfig` | The sink configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - |`PulsarAdminException.NotFoundException` | Cluster doesn't exist - |`PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + |`PulsarAdminException.NotFoundException` | Cluster doesn't exist + |`PulsarAdminException` | Unexpected error For more information, see [`updateSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSinkWithUrl-SinkConfig-java.lang.String-). @@ -1324,7 +1411,9 @@ Stop a source connector. Use the `stop` subcommand. ``` + $ pulsar-admin sources stop options + ``` For more information, see [here](io-cli.md#stop). @@ -1345,55 +1434,59 @@ For more information, see [here](io-cli.md#stop). * Stop **all** source connectors. - ```java - void stopSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** source connector. - ```java - void stopSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1425,7 +1518,9 @@ Stop a sink connector. Use the `stop` subcommand. ``` + $ pulsar-admin sinks stop options + ``` For more information, see [here](io-cli.md#stop-1). @@ -1446,55 +1541,59 @@ For more information, see [here](io-cli.md#stop-1). * Stop **all** sink connectors. - ```java - void stopSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** sink connector. - ```java - void stopSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1532,7 +1631,9 @@ Restart a source connector. Use the `restart` subcommand. ``` + $ pulsar-admin sources restart options + ``` For more information, see [here](io-cli.md#restart). @@ -1553,55 +1654,59 @@ For more information, see [here](io-cli.md#restart). * Restart **all** source connectors. - ```java - void restartSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** source connector. - ```java - void restartSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1633,7 +1738,9 @@ Restart a sink connector. Use the `restart` subcommand. ``` + $ pulsar-admin sinks restart options + ``` For more information, see [here](io-cli.md#restart-1). @@ -1654,55 +1761,59 @@ For more information, see [here](io-cli.md#restart-1). * Restart all Pulsar sink connectors. - ```java - void restartSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Sink name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Sink name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** sink connector. - ```java - void restartSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Sink instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Sink instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1740,7 +1851,9 @@ Delete a source connector. Use the `delete` subcommand. ``` + $ pulsar-admin sources delete options + ``` For more information, see [here](io-cli.md#delete). @@ -1758,10 +1871,12 @@ Send a `DELETE` request to this endpoint: {@inject: endpoint|DELETE|/admin/v3/so Delete a source connector. ```java + void deleteSource(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Parameter** @@ -1813,7 +1928,9 @@ Delete a sink connector. Use the `delete` subcommand. ``` + $ pulsar-admin sinks delete options + ``` For more information, see [here](io-cli.md#delete-1). @@ -1831,10 +1948,12 @@ Send a `DELETE` request to this endpoint: {@inject: endpoint|DELETE|/admin/v3/si Delete a Pulsar sink connector. ```java + void deleteSink(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Parameter** diff --git a/site2/website-next/docs/performance-pulsar-perf.md b/site2/website-next/docs/performance-pulsar-perf.md index 3e7f3321a8c42..1d9c0f9cfb1fd 100644 --- a/site2/website-next/docs/performance-pulsar-perf.md +++ b/site2/website-next/docs/performance-pulsar-perf.md @@ -1,7 +1,7 @@ --- id: performance-pulsar-perf title: Pulsar Perf -sidebar_label: Pulsar Perf +sidebar_label: "Pulsar Perf" --- import Tabs from '@theme/Tabs'; @@ -15,6 +15,7 @@ The Pulsar Perf is a built-in performance test tool for Apache Pulsar. You can u This example shows how the Pulsar Perf produces messages with default options. For all configuration options available for the `pulsar-perf produce` command, see [configuration options](#configuration-options-for-pulsar-perf-produce). ``` + bin/pulsar-perf produce my-topic ``` @@ -24,6 +25,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 19:53:31.459 [pulsar-perf-producer-exec-1-1] INFO org.apache.pulsar.testclient.PerformanceProducer - Created 1 producers 19:53:31.482 [pulsar-timer-5-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 19:53:40.861 [main] INFO org.apache.pulsar.testclient.PerformanceProducer - Throughput produced: 93.7 msg/s --- 0.7 Mbit/s --- failure 0.0 msg/s --- Latency: mean: 3.575 ms - med: 3.460 - 95pct: 4.790 - 99pct: 5.308 - 99.9pct: 5.834 - 99.99pct: 6.609 - Max: 6.609 @@ -51,7 +53,7 @@ The following table lists configuration options available for the `pulsar-perf p | access-mode | Set the producer access mode. Valid values are `Shared`, `Exclusive` and `WaitForExclusive`. | Shared | | admin-url | Set the Pulsar admin URL. | N/A | | auth-params | Set the authentication parameters, whose format is determined by the implementation of the `configure` method in the authentication plugin class, such as "key1:val1,key2:val2" or "{"key1":"val1","key2":"val2"}". | N/A | -| auth_plugin | Set the authentication plugin class name. | N/A | +| auth-plugin | Set the authentication plugin class name. | N/A | | listener-name | Set the listener name for the broker. | N/A | | batch-max-bytes | Set the maximum number of bytes for each batch. | 4194304 | | batch-max-messages | Set the maximum number of messages for each batch. | 1000 | @@ -96,6 +98,7 @@ The following table lists configuration options available for the `pulsar-perf p This example shows how the Pulsar Perf consumes messages with default options. ``` + bin/pulsar-perf consume my-topic ``` @@ -105,6 +108,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 20:35:37.071 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Start receiving from 1 consumers on 1 topics 20:35:41.150 [pulsar-client-io-1-9] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 20:35:47.092 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Throughput received: 59.572 msg/s -- 0.465 Mbit/s --- Latency: mean: 11.298 ms - med: 10 - 95pct: 15 - 99pct: 98 - 99.9pct: 137 - 99.99pct: 152 - Max: 152 @@ -130,7 +134,7 @@ The following table lists configuration options available for the `pulsar-perf c |----|----|----| | acks-delay-millis | Set the acknowledgment grouping delay in milliseconds. | 100 ms | | auth-params | Set the authentication parameters, whose format is determined by the implementation of the `configure` method in the authentication plugin class, such as "key1:val1,key2:val2" or "{"key1":"val1","key2":"val2"}". | N/A | -| auth_plugin | Set the authentication plugin class name. | N/A | +| auth-plugin | Set the authentication plugin class name. | N/A | | auto_ack_chunk_q_full | Configure whether to automatically ack for the oldest message in receiver queue if the queue is full. | false | | listener-name | Set the listener name for the broker. | N/A | | batch-index-ack | Enable or disable the batch index acknowledgment. | false | @@ -167,6 +171,7 @@ By default, the Pulsar Perf uses `conf/client.conf` as the default configuration You can use the following commands to change the configuration file and the Log4j configuration file. ``` + export PULSAR_CLIENT_CONF= export PULSAR_LOG_CONF= @@ -175,6 +180,7 @@ export PULSAR_LOG_CONF= In addition, you can use the following command to configure the JVM configuration through environment variables: ``` + export PULSAR_EXTRA_OPTS='-Xms4g -Xmx4g -XX:MaxDirectMemorySize=4g' ``` @@ -187,32 +193,38 @@ To check test results through the HdrHistogram Plotter, follow these steps: 1. Clone the HdrHistogram repository from GitHub to the local. - ``` - git clone https://github.com/HdrHistogram/HdrHistogram.git - - ``` + ``` + + git clone https://github.com/HdrHistogram/HdrHistogram.git + + ``` 2. Switch to the HdrHistogram folder. - ``` - cd HdrHistogram + ``` + + cd HdrHistogram + + ``` - ``` 3. Install the HdrHistogram Plotter. - ``` - mvn clean install -DskipTests + ``` + + mvn clean install -DskipTests + + ``` - ``` 4. Transform the file generated by the Pulsar Perf. - ``` - ./HistogramLogProcessor -i -o - - ``` + ``` + + ./HistogramLogProcessor -i -o + + ``` 5. You will get two output files. Upload the output file with the filename extension of .hgrm to the [HdrHistogram Plotter](https://hdrhistogram.github.io/HdrHistogram/plotFiles.html). 6. Check the test result through the Graphical User Interface of the HdrHistogram Plotter, as shown blow. - ![](/assets/perf-produce.png) + ![](/assets/perf-produce.png) diff --git a/site2/website-next/docs/reference-cli-tools.md b/site2/website-next/docs/reference-cli-tools.md index cbdf7173ba927..af1cd0ff1f645 100644 --- a/site2/website-next/docs/reference-cli-tools.md +++ b/site2/website-next/docs/reference-cli-tools.md @@ -1,7 +1,7 @@ --- id: reference-cli-tools title: Pulsar command-line tools -sidebar_label: Pulsar CLI tools +sidebar_label: "Pulsar CLI tools" --- import Tabs from '@theme/Tabs'; @@ -21,9 +21,12 @@ All Pulsar command-line tools can be run from the `bin` directory of your [insta > ### Getting help > You can get help for any CLI tool, command, or subcommand using the `--help` flag, or `-h` for short. Here's an example: + > ```shell -> +> > $ bin/pulsar broker --help +> +> > ``` ## `pulsar` @@ -39,6 +42,7 @@ Usage: $ pulsar command ``` + Commands: * `bookie` * `broker` @@ -146,6 +150,7 @@ Usage $ pulsar compact-topic options ``` + Options |Flag|Description|Default| @@ -509,6 +514,7 @@ Run a consumer Usage ``` + $ pulsar-perf consume options ``` @@ -518,7 +524,7 @@ Options |Flag|Description|Default| |---|---|---| |`--auth-params`|Authentication parameters, whose format is determined by the implementation of method `configure` in authentication plugin class. For example, `key1:val1,key2:val2` or `{"key1":"val1","key2":"val2"}`.|| -|`--auth_plugin`|Authentication plugin class name|| +|`--auth-plugin`|Authentication plugin class name|| |`-ac`, `--auto_ack_chunk_q_full`|Auto ack for the oldest message in consumer's receiver queue if the queue full|false| |`--listener-name`|Listener name for the broker|| |`--acks-delay-millis`|Acknowledgements grouping delay in millis|100| @@ -569,7 +575,7 @@ Options |`-am`, `--access-mode`|Producer access mode. Valid values are `Shared`, `Exclusive` and `WaitForExclusive`|Shared| |`-au`, `--admin-url`|Pulsar admin URL|| |`--auth-params`|Authentication parameters, whose format is determined by the implementation of method `configure` in authentication plugin class. For example, `key1:val1,key2:val2` or `{"key1":"val1","key2":"val2"}`.|| -|`--auth_plugin`|Authentication plugin class name|| +|`--auth-plugin`|Authentication plugin class name|| |`--listener-name`|Listener name for the broker|| |`-b`, `--batch-time-window`|Batch messages in a window of the specified number of milliseconds|1| |`-bb`, `--batch-max-bytes`|Maximum number of bytes per batch|4194304| @@ -626,7 +632,7 @@ Options |Flag|Description|Default| |---|---|---| |`--auth-params`|Authentication parameters, whose format is determined by the implementation of method `configure` in authentication plugin class. For example, `key1:val1,key2:val2` or `{"key1":"val1","key2":"val2"}`.|| -|`--auth_plugin`|Authentication plugin class name|| +|`--auth-plugin`|Authentication plugin class name|| |`--listener-name`|Listener name for the broker|| |`--conf-file`|Configuration file|| |`-h`, `--help`|Help message|false| @@ -660,12 +666,15 @@ Options |Flag|Description|Default| |---|---|---| |`--auth-params`|Authentication parameters, whose format is determined by the implementation of method `configure` in authentication plugin class. For example, `key1:val1,key2:val2` or `{"key1":"val1","key2":"val2"}`.|| -|`--auth_plugin`|Authentication plugin class name|| +|`--auth-plugin`|Authentication plugin class name|| |`--conf-file`|Configuration file|| |`-h`, `--help`|Help message|false| |`-m`, `--num-messages`|Number of messages to publish in total. If this value is less than or equal to 0, it keeps publishing messages.|0| |`-t`, `--num-topic`|The number of topics|1| |`-f`, `--payload-file`|Use payload from a file instead of empty buffer|| +|`-e`, `--payload-delimiter`|The delimiter used to split lines when using payload from a file|\n| +|`-fp`, `--format-payload`|Format %i as a message index in the stream from producer and/or %t as the timestamp nanoseconds|false| +|`-fc`, `--format-class`|Custom formatter class name|`org.apache.pulsar.testclient.DefaultMessageFormatter`| |`-u`, `--proxy-url`|Pulsar Proxy URL, e.g., "ws://localhost:8080/"|| |`-r`, `--rate`|Publish rate msg/s across topics|100| |`-s`, `--size`|Message size in byte|1024| @@ -905,6 +914,7 @@ Usage $ broker-tool command ``` + Commands * `load-report` * `help` diff --git a/site2/website-next/docs/reference-configuration.md b/site2/website-next/docs/reference-configuration.md index ca565a04b6f3c..53c4cbfd6d064 100644 --- a/site2/website-next/docs/reference-configuration.md +++ b/site2/website-next/docs/reference-configuration.md @@ -1,7 +1,7 @@ --- id: reference-configuration title: Pulsar configuration -sidebar_label: Pulsar configuration +sidebar_label: "Pulsar configuration" --- import Tabs from '@theme/Tabs'; @@ -161,6 +161,7 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |exposeConsumerLevelMetricsInPrometheus|Whether to enable consumer level metrics.|false| |jvmGCMetricsLoggerClassName|Classname of Pluggable JVM GC metrics logger that can log GC specific metrics.|N/A| |bindAddress| Hostname or IP address the service binds on, default is 0.0.0.0. |0.0.0.0| +|bindAddresses| Additional Hostname or IP addresses the service binds on: `listener_name:scheme://host:port,...`. || |advertisedAddress| Hostname or IP address the service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostName()` is used. || |clusterName| Name of the cluster to which this broker belongs to || |brokerDeduplicationEnabled| Sets the default behavior for message deduplication in the broker. If enabled, the broker will reject messages that were already stored in the topic. This setting can be overridden on a per-namespace basis. |false| @@ -203,7 +204,8 @@ brokerServiceCompactionThresholdInBytes|If the estimated backlog size is greater |tlsTrustCertsFilePath| Path for the trusted TLS certificate file. This cert is used to verify that any certs presented by connecting clients are signed by a certificate authority. If this verification fails, then the certs are untrusted and the connections are dropped. || |tlsAllowInsecureConnection| Accept untrusted TLS certificate from client. If it is set to `true`, a client with a cert which cannot be verified with the 'tlsTrustCertsFilePath' cert will be allowed to connect to the server, though the cert will not be used for client authentication. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.3```, ```TLSv1.2``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + |tlsEnabledWithKeyStore| Enable TLS with KeyStore type configuration in broker |false| |tlsProvider| TLS Provider for KeyStore type || |tlsKeyStoreType| LS KeyStore type configuration in broker: JKS, PKCS12 |JKS| @@ -431,6 +433,7 @@ You can set the log level and configuration in the [log4j2.yaml](https://github |brokerServicePort| The port on which the standalone broker listens for connections |6650| |webServicePort| The port used by the standalone broker for HTTP requests |8080| |bindAddress| The hostname or IP address on which the standalone service binds |0.0.0.0| +|bindAddresses| Additional Hostname or IP addresses the service binds on: `listener_name:scheme://host:port,...`. || |advertisedAddress| The hostname or IP address that the standalone service advertises to the outside world. If not set, the value of `InetAddress.getLocalHost().getHostName()` is used. || | numAcceptorThreads | Number of threads to use for Netty Acceptor | 1 | | numIOThreads | Number of threads to use for Netty IO | 2 * Runtime.getRuntime().availableProcessors() | @@ -720,7 +723,8 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tlsHostnameVerificationEnabled| Whether the hostname is validated when the proxy creates a TLS connection with brokers |false| |tlsRequireTrustedClientCertOnConnect| Whether client certificates are required for TLS. Connections are rejected if the client certificate isn’t trusted. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.3```, ```TLSv1.2``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + | httpReverseProxyConfigs | HTTP directs to redirect to non-pulsar services | | | httpOutputBufferSize | HTTP output buffer size. The amount of data that will be buffered for HTTP requests before it is flushed to the channel. A larger buffer size may result in higher HTTP throughput though it may take longer for the client to see data. If using HTTP streaming via the reverse proxy, this should be set to the minimum value (1) so that clients see the data as soon as possible. | 32768 | | httpNumThreads | Number of threads to use for HTTP requests processing| 2 * Runtime.getRuntime().availableProcessors() | diff --git a/site2/website-next/docs/reference-connector-admin.md b/site2/website-next/docs/reference-connector-admin.md index 4cc1d50e7d33c..c4fe71c64cf33 100644 --- a/site2/website-next/docs/reference-connector-admin.md +++ b/site2/website-next/docs/reference-connector-admin.md @@ -1,7 +1,7 @@ --- id: reference-connector-admin title: Connector Admin CLI -sidebar_label: Connector Admin CLI +sidebar_label: "Connector Admin CLI" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/reference-metrics.md b/site2/website-next/docs/reference-metrics.md index 7b4414cda27a3..cd4609f623d37 100644 --- a/site2/website-next/docs/reference-metrics.md +++ b/site2/website-next/docs/reference-metrics.md @@ -1,7 +1,7 @@ --- id: reference-metrics title: Pulsar Metrics -sidebar_label: Pulsar Metrics +sidebar_label: "Pulsar Metrics" --- import Tabs from '@theme/Tabs'; @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -106,20 +106,21 @@ The following metrics are available for broker: - [Storage metrics](#storage-metrics) - [Broker](#broker) - [Namespace metrics](#namespace-metrics) - - [Replication metrics](#replication-metrics) + - [Replication metrics](#replication-metrics) - [Topic metrics](#topic-metrics) - - [Replication metrics](#replication-metrics-1) + - [Replication metrics](#replication-metrics-1) - [ManagedLedgerCache metrics](#managedledgercache-metrics) - [ManagedLedger metrics](#managedledger-metrics) - [LoadBalancing metrics](#loadbalancing-metrics) - - [BundleUnloading metrics](#bundleunloading-metrics) - - [BundleSplit metrics](#bundlesplit-metrics) + - [BundleUnloading metrics](#bundleunloading-metrics) + - [BundleSplit metrics](#bundlesplit-metrics) - [Subscription metrics](#subscription-metrics) - [Consumer metrics](#consumer-metrics) - [Managed ledger bookie client metrics](#managed-ledger-bookie-client-metrics) - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -290,7 +291,7 @@ All the cursor acknowledgment state metrics are labelled with the following labe Name |Type |Description |---|---|--- -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to a ledger.| +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:")|Gauge|The number of acknowledgment states that is persistent to a ledger.| brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to the ledger.| brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to ZooKeeper. brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to ZooKeeper. @@ -303,20 +304,20 @@ brk_ml_cursor_readLedgerSize(namespace="", ledger_name="", cursor_name:"")|Gauge All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -326,7 +327,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -437,6 +438,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -489,10 +519,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/website-next/docs/reference-pulsar-admin.md b/site2/website-next/docs/reference-pulsar-admin.md index 206380cea6cbf..cf4f45b8e8bc0 100644 --- a/site2/website-next/docs/reference-pulsar-admin.md +++ b/site2/website-next/docs/reference-pulsar-admin.md @@ -1,7 +1,7 @@ --- id: pulsar-admin title: Pulsar admin CLI -sidebar_label: Pulsar Admin CLI +sidebar_label: "Pulsar Admin CLI" --- import Tabs from '@theme/Tabs'; @@ -585,6 +585,7 @@ Create a Pulsar Function in cluster mode (i.e. deploy it on a Pulsar cluster) Usage ``` + $ pulsar-admin functions create options ``` @@ -2130,6 +2131,7 @@ Usage $ pulsar-admin namespaces get-max-subscriptions-per-topic tenant/namespace ``` + ### `remove-max-subscriptions-per-topic` Remove the maximum subscription per topic for a namespace. @@ -2350,6 +2352,7 @@ Run compaction on the specified topic (persistent topics only) Usage ``` + $ pulsar-admin topics compact persistent://tenant/namespace/topic ``` @@ -2410,13 +2413,9 @@ Create a partitioned topic. A partitioned topic must be created before producers :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2462,13 +2461,9 @@ Creates a non-partitioned topic. A non-partitioned topic must explicitly be crea :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2525,6 +2520,7 @@ Get the list of topics under a namespace Usage ``` + $ pulsar-admin topics list tenant/cluster/namespace ``` @@ -2975,6 +2971,7 @@ Usage $ pulsar-admin topics remove-persistence tenant/namespace/topic ``` + ### `get-message-ttl` Get the message TTL for a topic. @@ -3009,7 +3006,7 @@ Usage ```bash -$ pulsar-admin topics remove-message-ttl tenant/namespace/topic +$ pulsar-admin topics remove-message-ttl tenant/namespace/topic ``` @@ -3228,6 +3225,7 @@ Operations related to Schemas associated with Pulsar topics. Usage ``` + $ pulsar-admin schemas subcommand ``` diff --git a/site2/website-next/docs/reference-terminology.md b/site2/website-next/docs/reference-terminology.md index c03395427040b..5bdcc27561ed2 100644 --- a/site2/website-next/docs/reference-terminology.md +++ b/site2/website-next/docs/reference-terminology.md @@ -1,7 +1,7 @@ --- id: reference-terminology title: Pulsar Terminology -sidebar_label: Terminology +sidebar_label: "Terminology" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/schema-evolution-compatibility.md b/site2/website-next/docs/schema-evolution-compatibility.md index 4a52533b34161..531435e48c1fb 100644 --- a/site2/website-next/docs/schema-evolution-compatibility.md +++ b/site2/website-next/docs/schema-evolution-compatibility.md @@ -1,12 +1,13 @@ --- id: schema-evolution-compatibility title: Schema evolution and compatibility -sidebar_label: Schema evolution and compatibility +sidebar_label: "Schema evolution and compatibility" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + Normally, schemas do not stay the same over a long period of time. Instead, they undergo evolutions to satisfy new needs. This chapter examines how Pulsar schema evolves and what Pulsar schema compatibility check strategies are. @@ -35,9 +36,9 @@ For more information, see [Schema compatibility check strategy](#schema-compatib 1. When a producer/consumer/reader connects to a broker, the broker deploys the schema compatibility checker configured by `schemaRegistryCompatibilityCheckers` to enforce schema compatibility check. - The schema compatibility checker is one instance per schema type. - - Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. + The schema compatibility checker is one instance per schema type. + + Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. 2. The producer/consumer/reader sends its client `SchemaInfo` to the broker. @@ -53,328 +54,47 @@ Pulsar has 8 schema compatibility check strategies, which are summarized in the Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is the oldest and V3 is the latest: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Changes allowed - - - -Check against which schema - - - -Upgrade first - -
    -`ALWAYS_COMPATIBLE` - -Disable schema compatibility check. - -All changes are allowed - -All previous versions - -Any order -
    -`ALWAYS_INCOMPATIBLE` - -Disable schema evolution. - -All changes are disabled - -None - -None -
    -`BACKWARD` - -Consumers using the schema V3 can process data written by producers using the schema V3 or V2. - -* Add optional fields -* Delete fields - -Latest version - -Consumers -
    -`BACKWARD_TRANSITIVE` - -Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. - -* Add optional fields -* Delete fields - -All previous versions - -Consumers -
    -`FORWARD` - -Consumers using the schema V3 or V2 can process data written by producers using the schema V3. - -* Add fields -* Delete optional fields - -Latest version - -Producers -
    -`FORWARD_TRANSITIVE` - -Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. - -* Add fields -* Delete optional fields - -All previous versions - -Producers -
    -`FULL` - -Backward and forward compatible between the schema V3 and V2. - -* Modify optional fields - -Latest version - -Any order -
    -`FULL_TRANSITIVE` - -Backward and forward compatible among the schema V3, V2, and V1. - -* Modify optional fields - -All previous versions - -Any order -
    +| Compatibility check strategy | Definition | Changes allowed | Check against which schema | Upgrade first | +| --- | --- | --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | All changes are allowed | All previous versions | Any order | +| `ALWAYS_INCOMPATIBLE` | Disable schema evolution. | All changes are disabled | None | None | +| `BACKWARD` | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. |
  • Add optional fields
  • Delete fields
  • | Latest version | Consumers | +| `BACKWARD_TRANSITIVE` | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. |
  • Add optional fields
  • Delete fields
  • | All previous versions | Consumers | +| `FORWARD` | Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | Latest version | Producers | +| `FORWARD_TRANSITIVE` | Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | All previous versions | Producers | +| `FULL` | Backward and forward compatible between the schema V3 and V2. |
  • Modify optional fields
  • | Latest version | Any order | +| `FULL_TRANSITIVE` | Backward and forward compatible among the schema V3, V2, and V1. |
  • Modify optional fields
  • | All previous versions | Any order | ### ALWAYS_COMPATIBLE and ALWAYS_INCOMPATIBLE - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Note - -
    -`ALWAYS_COMPATIBLE` - -Disable schema compatibility check. - -None -
    -`ALWAYS_INCOMPATIBLE` - -Disable schema evolution, that is, any schema change is rejected. - -* For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. -* For Avro and JSON, the default schema compatibility check strategy is `FULL`. -
    +| Compatibility check strategy | Definition | Note | +| --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | None | +| `ALWAYS_INCOMPATIBLE` | Disable schema evolution, that is, any schema change is rejected. |
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • | #### Example * Example 1 - In some situations, an application needs to store events of several different types in the same Pulsar topic. + In some situations, an application needs to store events of several different types in the same Pulsar topic. - In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. + In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. - For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. + For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. - Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. + Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. * Example 2 - Sometimes we also make incompatible changes. + Sometimes we also make incompatible changes. - For example, you are modifying a field type from `string` to `int`. + For example, you are modifying a field type from `string` to `int`. - In this case, you need to: + In this case, you need to: - * Upgrade all producers and consumers to the new schema versions at the same time. + * Upgrade all producers and consumers to the new schema versions at the same time. - * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. + * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. ### BACKWARD and BACKWARD_TRANSITIVE @@ -389,15 +109,15 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t * Example 1 - Remove a field. - - A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. + Remove a field. + + A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. * Example 2 - You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. + You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. - Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. + Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. ### FORWARD and FORWARD_TRANSITIVE @@ -428,87 +148,10 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is the oldest and V3 is the latest: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Description - - - -Note - -
    -`FULL` - -Schemas are both backward and forward compatible, which means: -Consumers using the last schema can process data written by producers using the new schema. -AND -Consumers using the new schema can process data written by producers using the last schema. - -Consumers using the schema V3 can process data written by producers using the schema V3 or V2. -AND -Consumers using the schema V3 or V2 can process data written by producers using the schema V3. - -* For Avro and JSON, the default schema compatibility check strategy is `FULL`. -* For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. -
    -`FULL_TRANSITIVE` - -The new schema is backward and forward compatible with all previously registered schemas. - -Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. -AND -Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. - -None -
    +| Compatibility check strategy | Definition | Description | Note | +| --- | --- | --- | --- | +| `FULL` | Schemas are both backward and forward compatible, which means: Consumers using the last schema can process data written by producers using the new schema. AND Consumers using the new schema can process data written by producers using the last schema. | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. AND Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • | +| `FULL_TRANSITIVE` | The new schema is backward and forward compatible with all previously registered schemas. | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. AND Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. | None | #### Example @@ -524,130 +167,23 @@ When a producer tries to connect to a topic (suppose ignore the schema auto crea * Check if the schema carried by the producer exists in the schema registry or not. - * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. - - * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. - + * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. + + * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. + ### Consumer When a consumer tries to connect to a topic, a broker checks if a carried schema is compatible with a registered schema based on the configured schema compatibility check strategy. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Check logic - -
    -`ALWAYS_COMPATIBLE` - -All pass -
    -`ALWAYS_INCOMPATIBLE` - -No pass -
    -`BACKWARD` - -Can read the last schema -
    -`BACKWARD_TRANSITIVE` - -Can read all schemas -
    -`FORWARD` - -Can read the last schema -
    -`FORWARD_TRANSITIVE` - -Can read the last schema -
    -`FULL` - -Can read the last schema -
    -`FULL_TRANSITIVE` - -Can read all schemas -
    +| Compatibility check strategy | Check logic | +| --- | --- | +| `ALWAYS_COMPATIBLE` | All pass | +| `ALWAYS_INCOMPATIBLE` | No pass | +| `BACKWARD` | Can read the last schema | +| `BACKWARD_TRANSITIVE` | Can read all schemas | +| `FORWARD` | Can read the last schema | +| `FORWARD_TRANSITIVE` | Can read the last schema | +| `FULL` | Can read the last schema | +| `FULL_TRANSITIVE` | Can read all schemas | ## Order of upgrading clients @@ -655,120 +191,13 @@ The order of upgrading client applications is determined by the compatibility ch For example, the producers using schemas to write data to Pulsar and the consumers using schemas to read data from Pulsar. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Upgrade first - - - -Description - -
    -`ALWAYS_COMPATIBLE` - -Any order - -The compatibility check is disabled. -Consequently, you can upgrade the producers and consumers in **any order**. -
    -`ALWAYS_INCOMPATIBLE` - -None - -The schema evolution is disabled. -
    -* `BACKWARD` -* `BACKWARD_TRANSITIVE` - -Consumers - -There is no guarantee that consumers using the old schema can read data produced using the new schema. -Consequently, **upgrade all consumers first**, and then start producing new data. -
    -* `FORWARD` -* `FORWARD_TRANSITIVE` - -Producers - -There is no guarantee that consumers using the new schema can read data produced using the old schema. -Consequently, **upgrade all producers first** to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers. -
    -* `FULL` -* `FULL_TRANSITIVE` - -Any order - -There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. -Consequently, you can upgrade the producers and consumers in **any order**. -
    +| Compatibility check strategy | Upgrade first | Description | +| --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Any order | The compatibility check is disabled. Consequently, you can upgrade the producers and consumers in **any order**. | +| `ALWAYS_INCOMPATIBLE` | None | The schema evolution is disabled. | +|
  • `BACKWARD`
  • `BACKWARD_TRANSITIVE`
  • | Consumers | There is no guarantee that consumers using the old schema can read data produced using the new schema. Consequently, **upgrade all consumers first**, and then start producing new data. | +|
  • `FORWARD`
  • `FORWARD_TRANSITIVE`
  • | Producers | There is no guarantee that consumers using the new schema can read data produced using the old schema. Consequently, **upgrade all producers first**
  • to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers.
  • | +|
  • `FULL`
  • `FULL_TRANSITIVE`
  • | Any order | There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. Consequently, you can upgrade the producers and consumers in **any order**. | diff --git a/site2/website-next/docs/schema-get-started.md b/site2/website-next/docs/schema-get-started.md index 5bc2e8580b8ed..ec77bf121ec2a 100644 --- a/site2/website-next/docs/schema-get-started.md +++ b/site2/website-next/docs/schema-get-started.md @@ -1,12 +1,13 @@ --- id: schema-get-started title: Get started -sidebar_label: Get started +sidebar_label: "Get started" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This chapter introduces Pulsar schemas and explains why they are important. ## Schema Registry @@ -19,7 +20,7 @@ Applications typically adopt one of the following approaches to guarantee type s #### Note > -> Currently, the Pulsar schema registry is only available for the [Java client](client-libraries-java.md), [CGo client](client-libraries-cgo.md), [Python client](client-libraries-python.md), and [C++ client](client-libraries-cpp.md). +> Currently, the Pulsar schema registry is only available for the [Java client](client-libraries-java.md), [CGo client](client-libraries-cgo.md), [Python client](client-libraries-python.md), and [C++ client](client-libraries-cpp). ### Client-side approach @@ -52,11 +53,14 @@ Pulsar schema enables you to use language-specific types of data when constructi **Example** You can use the _User_ class to define the messages sent to Pulsar topics. + ``` + public class User { String name; int age; } + ``` When constructing a producer with the _User_ class, you can specify a schema or not as below. @@ -66,14 +70,18 @@ When constructing a producer with the _User_ class, you can specify a schema or If you construct a producer without specifying a schema, then the producer can only produce messages of type `byte[]`. If you have a POJO class, you need to serialize the POJO into bytes before sending messages. **Example** + ``` + Producer producer = client.newProducer() .topic(topic) .create(); User user = new User("Tom", 28); byte[] message = … // serialize the `user` by yourself; producer.send(message); + ``` + ### With schema If you construct a producer with specifying a schema, then you can send a class to a topic directly without worrying about how to serialize POJOs into bytes. @@ -81,12 +89,15 @@ If you construct a producer with specifying a schema, then you can send a class **Example** This example constructs a producer with the _JSONSchema_, and you can send the _User_ class to topics directly without worrying about how to serialize it into bytes. + ``` + Producer producer = client.newProducer(JSONSchema.of(User.class)) .topic(topic) .create(); User user = new User("Tom", 28); producer.send(user); + ``` ### Summary diff --git a/site2/website-next/docs/schema-manage.md b/site2/website-next/docs/schema-manage.md index 4a6670687636d..c614a96e6d97d 100644 --- a/site2/website-next/docs/schema-manage.md +++ b/site2/website-next/docs/schema-manage.md @@ -1,12 +1,13 @@ --- id: schema-manage title: Manage schema -sidebar_label: Manage schema +sidebar_label: "Manage schema" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This guide demonstrates the ways to manage schemas: * Automatically @@ -33,25 +34,25 @@ For a producer, the `AutoUpdate` happens in the following cases: * If a **producer doesn’t carry a schema**: - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. - - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. + + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. * If a **producer carries a schema**: - A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. - - * If the schema is registered, a producer is connected to a broker. - - * If the schema is not registered: - - * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. - - * If `isAllowAutoUpdateSchema` sets to **true**: - - * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. - - * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. + A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. + + * If the schema is registered, a producer is connected to a broker. + + * If the schema is not registered: + + * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. + + * If `isAllowAutoUpdateSchema` sets to **true**: + + * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. + + * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. ![AutoUpdate Producer](/assets/schema-producer.png) @@ -63,18 +64,18 @@ For a consumer, the `AutoUpdate` happens in the following cases: * If a **consumer connects to a topic with a schema**. - * If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - - * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. - + * If a topic does not have all of them (a schema/data/a local consumer and a local producer): + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + + * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + ![AutoUpdate Consumer](/assets/schema-consumer.png) @@ -92,18 +93,20 @@ You can use the `pulsar-admin` command to manage the `AutoUpdate` strategy as be To enable `AutoUpdate` on a namespace, you can use the `pulsar-admin` command. - ```bash + bin/pulsar-admin namespaces set-is-allow-auto-update-schema --enable tenant/namespace + ``` #### Disable AutoUpdate To disable `AutoUpdate` on a namespace, you can use the `pulsar-admin` command. - ```bash + bin/pulsar-admin namespaces set-is-allow-auto-update-schema --disable tenant/namespace + ``` Once the `AutoUpdate` is disabled, you can only register a new schema using the `pulsar-admin` command. @@ -112,9 +115,10 @@ Once the `AutoUpdate` is disabled, you can only register a new schema using the To adjust the schema compatibility level on a namespace, you can use the `pulsar-admin` command. - ```bash + bin/pulsar-admin namespaces set-schema-compatibility-strategy --compatibility tenant/namespace + ``` ### Schema validation @@ -131,56 +135,31 @@ However, if you want a stronger guarantee on the topics with schemas, you can en To enable `schemaValidationEnforced` on a namespace, you can use the `pulsar-admin` command. - ```bash + bin/pulsar-admin namespaces set-schema-validation-enforce --enable tenant/namespace + ``` #### Disable schema validation To disable `schemaValidationEnforced` on a namespace, you can use the `pulsar-admin` command. - ```bash + bin/pulsar-admin namespaces set-schema-validation-enforce --disable tenant/namespace + ``` ## Schema manual management To manage schemas, you can use one of the following methods. - - - - - - - - - - - - - - - - - - - -
    MethodDescription
    -**Admin CLI** - -You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. -For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin.md). -
    -**REST API** - -Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. -For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). -
    -**Java Admin API** - Pulsar provides Java admin library.
    +| Method | Description | +| --- | --- | +| **Admin CLI**
  • | You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin). | +| **REST API**
  • | Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). | +| **Java Admin API**
  • | Pulsar provides Java admin library. | ### Upload a schema @@ -207,73 +186,50 @@ To upload (register) a new schema for a topic, you can use one of the following Use the `upload` subcommand. - ```bash + $ pulsar-admin schemas upload --filename + ``` The `schema-definition-file` is in JSON format. - ```json + { "type": "", "schema": "", "properties": {} // the properties associated with the schema } + ``` The `schema-definition-file` includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | Here are examples of the `schema-definition-file` for a JSON schema. **Example 1** - ```json + { "type": "JSON", "schema": "{\"type\":\"record\",\"name\":\"User\",\"namespace\":\"com.foo\",\"fields\":[{\"name\":\"file1\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file2\",\"type\":\"string\",\"default\":null},{\"name\":\"file3\",\"type\":[\"null\",\"string\"],\"default\":\"dfdf\"}]}", "properties": {} } + ``` **Example 2** - ```json + { "type": "STRING", "schema": "", @@ -281,6 +237,7 @@ Here are examples of the `schema-definition-file` for a JSON schema. "key1": "value1" } } + ```
    @@ -290,103 +247,55 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v2/schema The post payload is in JSON format. - ```json + { "type": "", "schema": "", "properties": {} // the properties associated with the schema } + ``` The post payload includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. |
    - ```java + void createSchema(String topic, PostSchemaPayload schemaPayload) + ``` The `PostSchemaPayload` includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `PostSchemaPayload`: - ```java + PulsarAdmin admin = …; PostSchemaPayload payload = new PostSchemaPayload(); payload.setType("INT8"); payload.setSchema(""); -admin.createSchema("my-tenant/my-ns/my-topic", payload); +admin.createSchema("my-tenant/my-ns/my-topic", payload); + ``` +
    @@ -416,8 +325,8 @@ To get the latest schema for a topic, you can use one of the following methods. Use the `get` subcommand. - ```bash + $ pulsar-admin schemas get { @@ -430,6 +339,7 @@ $ pulsar-admin schemas get "property2": "string" } } + ``` @@ -439,8 +349,8 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v2/schemas/ Here is an example of a response, which is returned in JSON format. - ```json + { "version": "", "type": "", @@ -448,112 +358,45 @@ Here is an example of a response, which is returned in JSON format. "data": "", "properties": {} // the properties associated with the schema } + ``` The response includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`version` - - The schema version, which is a long number.
    -`type` - - The schema type.
    -`timestamp` - - The timestamp of creating this version of schema.
    -`data` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `version` | The schema version, which is a long number. | +| `type` | The schema type. | +| `timestamp` | The timestamp of creating this version of schema. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | - ```java + SchemaInfo createSchema(String topic) + ``` The `SchemaInfo` includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`name` - - The schema name.
    -`type` - - The schema type.
    -`schema` - -A byte array of the schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this byte array should be empty. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `name` | The schema name. | +| `type` | The schema type. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: - ```java + PulsarAdmin admin = …; -SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); +SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); + ```
    @@ -585,9 +428,10 @@ To get a specific version of a schema, you can use one of the following methods. Use the `get` subcommand. - ```bash -$ pulsar-admin schemas get --version= + +$ pulsar-admin schemas get --version= + ``` @@ -597,8 +441,8 @@ Send a `GET` request to a schema endpoint: {@inject: endpoint|GET|/admin/v2/sche Here is an example of a response, which is returned in JSON format. - ```json + { "version": "", "type": "", @@ -606,110 +450,45 @@ Here is an example of a response, which is returned in JSON format. "data": "", "properties": {} // the properties associated with the schema } + ``` The response includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`version` - - The schema version, which is a long number.
    -`type` - - The schema type.
    -`timestamp` - - The timestamp of creating this version of schema.
    -`data` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `version` | The schema version, which is a long number. | +| `type` | The schema type. | +| `timestamp` | The timestamp of creating this version of schema. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | - ```java + SchemaInfo createSchema(String topic, long version) + ``` The `SchemaInfo` includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`name` - The schema name.
    -`type` - The schema type.
    -`schema` - -A byte array of the schema definition data, which is encoded in UTF 8. -* If the schema is a **primitive** schema, this byte array should be empty. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `name` | The schema name. | +| `type` | The schema type. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: - ```java + PulsarAdmin admin = …; SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic", 1L); + ```
    @@ -733,9 +512,10 @@ To provide a schema via a topic, you can use the following method. Use the `extract` subcommand. - ```bash + $ pulsar-admin schemas extract --classname --jar --type + ``` @@ -746,9 +526,11 @@ $ pulsar-admin schemas extract --classname --jar --type To delete a schema for a topic, you can use one of the following methods. -> #### Note -> -> In any case, the **delete** action deletes **all versions** of a schema registered for a topic. +:::note + +In any case, the **delete** action deletes **all versions** of a schema registered for a topic. + +::: + ``` @@ -783,11 +566,12 @@ Send a `DELETE` request to a schema endpoint: {@inject: endpoint|DELETE|/admin/v Here is an example of a response, which is returned in JSON format. - ```json + { "version": "", } + ``` The response includes the following field: @@ -799,18 +583,20 @@ Field | Description | - ```java + void deleteSchema(String topic) + ``` Here is an example of deleting a schema. - ```java + PulsarAdmin admin = …; -admin.deleteSchema("my-tenant/my-ns/my-topic"); +admin.deleteSchema("my-tenant/my-ns/my-topic"); + ``` @@ -835,8 +621,8 @@ To use a non-default (non-BookKeeper) storage system for Pulsar schemas, you nee The `SchemaStorage` interface has the following methods: - ```java + public interface SchemaStorage { // How schemas are updated CompletableFuture put(String key, byte[] value, byte[] hash); @@ -856,27 +642,33 @@ public interface SchemaStorage { // Shutdown behavior for the schema storage client void close() throws Exception; } + ``` -> #### Tip -> -> For a complete example of **schema storage** implementation, see [BookKeeperSchemaStorage](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. +:::tip + +For a complete example of **schema storage** implementation, see [BookKeeperSchemaStorage](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. + +::: #### SchemaStorageFactory interface The `SchemaStorageFactory` interface has the following method: - ```java + public interface SchemaStorageFactory { @NotNull SchemaStorage create(PulsarService pulsar) throws Exception; } + ``` -> Tip -> -> For a complete example of **schema storage factory** implementation, see [BookKeeperSchemaStorageFactory](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. +:::tip + +For a complete example of **schema storage factory** implementation, see [BookKeeperSchemaStorageFactory](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. + +::: ### Deploy diff --git a/site2/website-next/docs/schema-understand.md b/site2/website-next/docs/schema-understand.md index 777a3aa667e6d..9d134cd386f89 100644 --- a/site2/website-next/docs/schema-understand.md +++ b/site2/website-next/docs/schema-understand.md @@ -1,12 +1,13 @@ --- id: schema-understand title: Understand schema -sidebar_label: Understand schema +sidebar_label: "Understand schema" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This chapter explains the basic concepts of Pulsar schema, focuses on the topics of particular importance, and provides additional background. ## SchemaInfo @@ -17,92 +18,26 @@ The `SchemaInfo` is stored and enforced on a per-topic basis and cannot be store A `SchemaInfo` consists of the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Field - - - -Description - -
    -`name` - -Schema name (a string). -
    -`type` - -Schema type, which determines how to interpret the schema data. -* Predefined schema: see [here](schema-understand.md#schema-type). -* Customized schema: it is left as an empty string. -
    -`schema`(`payload`) - -Schema data, which is a sequence of 8-bit unsigned bytes and schema-type specific. -
    -`properties` - -It is a user defined properties as a string/string map. -Applications can use this bag for carrying any application specific logics. -Possible properties might be the Git hash associated with the schema, an environment string like `dev` or `prod`. -
    +| Field | Description | +| --- | --- | +| `name` | Schema name (a string). | +| `type` | Schema type, which determines how to interpret the schema data.
  • Predefined schema: see [here](schema-understand.md#schema-type).
  • Customized schema: it is left as an empty string.
  • | +| `schema`(`payload`) | Schema data, which is a sequence of 8-bit unsigned bytes and schema-type specific. | +| `properties` | It is a user defined properties as a string/string map. Applications can use this bag for carrying any application specific logics. Possible properties might be the Git hash associated with the schema, an environment string like `dev` or `prod`. | **Example** This is the `SchemaInfo` of a string. - ```json + { "name": "test-string-schema", "type": "STRING", "schema": "", "properties": {} } + ``` ## Schema type @@ -165,19 +100,21 @@ This example demonstrates how to use a string schema. 1. Create a producer with a string schema and send messages. - -```java - Producer producer = client.newProducer(Schema.STRING).create(); - producer.newMessage().value("Hello Pulsar!").send(); -``` + ```java + + Producer producer = client.newProducer(Schema.STRING).create(); + producer.newMessage().value("Hello Pulsar!").send(); + + ``` 2. Create a consumer with a string schema and receive messages. - -```java - Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); - consumer.receive(); -``` + ```java + + Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); + consumer.receive(); + + ``` ### Complex type @@ -230,68 +167,72 @@ This example shows how to construct a key/value schema and then use it to produc 1. Construct a key/value schema with `INLINE` encoding type. - -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.INLINE - ); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.INLINE + ); + + ``` 2. Optionally, construct a key/value schema with `SEPARATED` encoding type. - -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + ``` 3. Produce messages using a key/value schema. - -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Producer> producer = client.newProducer(kvSchema) - .topic(TOPIC) - .create(); - - final int key = 100; - final String value = "value-100"; - - // send the key/value message - producer.newMessage() - .value(new KeyValue<>(key, value)) - .send(); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Producer> producer = client.newProducer(kvSchema) + .topic(TOPIC) + .create(); + + final int key = 100; + final String value = "value-100"; + + // send the key/value message + producer.newMessage() + .value(new KeyValue(key, value)) + .send(); + + ``` 4. Consume messages using a key/value schema. - -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Consumer> consumer = client.newConsumer(kvSchema) - ... - .topic(TOPIC) - .subscriptionName(SubscriptionName).subscribe(); - - // receive key/value pair - Message> msg = consumer.receive(); - KeyValue kv = msg.getValue(); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Consumer> consumer = client.newConsumer(kvSchema) + ... + .topic(TOPIC) + .subscriptionName(SubscriptionName).subscribe(); + + // receive key/value pair + Message> msg = consumer.receive(); + KeyValue kv = msg.getValue(); + + ``` @@ -347,32 +288,35 @@ Pulsar gets the schema definition from the predefined `struct` using an Avro lib 1. Create the _User_ class to define the messages sent to Pulsar topics. - -```java - @Builder - @AllArgsConstructor - @NoArgsConstructor - public static class User { - String name; - int age; - } -``` + ```java + + @Builder + @AllArgsConstructor + @NoArgsConstructor + public static class User { + String name; + int age; + } + + ``` 2. Create a producer with a `struct` schema and send messages. - -```java - Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); - producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); -``` + ```java + + Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); + producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); + + ``` 3. Create a consumer with a `struct` schema and receive messages - -```java - Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); - User user = consumer.receive(); -``` + ```java + + Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); + User user = consumer.receive(); + + ``` @@ -385,23 +329,25 @@ You can define the `struct` schema using the `GenericSchemaBuilder`, generate a 1. Use `RecordSchemaBuilder` to build a schema. - -```java - RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); - recordSchemaBuilder.field("intField").type(SchemaType.INT32); - SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); + ```java + + RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); + recordSchemaBuilder.field("intField").type(SchemaType.INT32); + SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); -``` + Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + + ``` 2. Use `RecordBuilder` to build the struct records. - -```java - producer.newMessage().value(schema.newRecordBuilder() - .set("intField", 32) - .build()).send(); -``` + ```java + + producer.newMessage().value(schema.newRecordBuilder() + .set("intField", 32) + .build()).send(); + + ``` @@ -412,34 +358,37 @@ You can define the `schemaDefinition` to generate a `struct` schema. 1. Create the _User_ class to define the messages sent to Pulsar topics. - -```java - @Builder - @AllArgsConstructor - @NoArgsConstructor - public static class User { - String name; - int age; - } -``` + ```java + + @Builder + @AllArgsConstructor + @NoArgsConstructor + public static class User { + String name; + int age; + } + + ``` 2. Create a producer with a `SchemaDefinition` and send messages. - -```java - SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); - Producer producer = client.newProducer(Schema.AVRO(schemaDefinition)).create(); - producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); -``` + ```java + + SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); + Producer producer = client.newProducer(Schema.AVRO(schemaDefinition)).create(); + producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); + + ``` 3. Create a consumer with a `SchemaDefinition` schema and receive messages - -```java - SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); - Consumer consumer = client.newConsumer(Schema.AVRO(schemaDefinition)).subscribe(); - User user = consumer.receive().getValue(); -``` + ```java + + SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); + Consumer consumer = client.newConsumer(Schema.AVRO(schemaDefinition)).subscribe(); + User user = consumer.receive().getValue(); + + ``` @@ -470,8 +419,8 @@ Suppose that: In this case, you can use `AUTO_PRODUCE` to verify whether the bytes produced by _K_ can be sent to _P_ or not. - ```java + Produce pulsarProducer = client.newProducer(Schema.AUTO_PRODUCE()) … .create(); @@ -479,6 +428,7 @@ Produce pulsarProducer = client.newProducer(Schema.AUTO_PRODUCE()) byte[] kafkaMessageBytes = … ; pulsarProducer.produce(kafkaMessageBytes); + ``` #### AUTO_CONSUME @@ -499,14 +449,15 @@ Suppose that: In this case, you can use `AUTO_CONSUME` to verify whether the bytes produced by _P_ can be sent to MySQL or not. - ```java + Consumer pulsarConsumer = client.newConsumer(Schema.AUTO_CONSUME()) … .subscribe(); Message msg = consumer.receive() ; GenericRecord record = msg.getValue(); + ``` ### Native Avro Schema @@ -517,8 +468,8 @@ Hence, we provide `Schema.NATIVE_AVRO` to wrap a native Avro schema of type `org **Example** - ```java + org.apache.avro.Schema nativeAvroSchema = … ; Producer producer = pulsarClient.newProducer().topic("ingress").create(); @@ -526,6 +477,7 @@ Producer producer = pulsarClient.newProducer().topic("ingress").create() byte[] content = … ; producer.newMessage(Schema.NATIVE_AVRO(nativeAvroSchema)).value(content).send(); + ``` ## Schema version @@ -542,10 +494,10 @@ Once a version is assigned/fetched to/for a schema, all subsequent messages prod The following example illustrates how the schema version works. -Suppose that a Pulsar [Java client](client-libraries-java.md) created using the code below attempts to connect to Pulsar and begins to send messages: - +Suppose that a Pulsar [Java client](client-libraries-java) created using the code below attempts to connect to Pulsar and begins to send messages: ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .build(); @@ -554,65 +506,15 @@ Producer producer = client.newProducer(JSONSchema.of(SensorReadin .topic("sensor-data") .sendTimeout(3, TimeUnit.SECONDS) .create(); + ``` The table below lists the possible scenarios when this connection attempt occurs and what happens in each scenario: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    ScenarioWhat happens
    -* No schema exists for the topic. - -(1) The producer is created using the given schema. -(2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. -(3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. -
    -* A schema already exists. -* The producer connects using the same schema that is already stored. - -(1) The schema is transmitted to the broker. -(2) The broker determines that the schema is compatible. -(3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. -
    -* A schema already exists. -* The producer connects using a new schema that is compatible. - -(1) The schema is transmitted to the broker. -(2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). -
    +| Scenario | What happens | +| --- | --- | +|
  • No schema exists for the topic.
  • | (1) The producer is created using the given schema. (2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. (3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. | +|
  • A schema already exists.
  • The producer connects using the same schema that is already stored.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible. (3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. |
  • A schema already exists.
  • The producer connects using a new schema that is compatible.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). | ## How does schema work @@ -628,9 +530,9 @@ This diagram illustrates how does schema work on the Producer side. 1. The application uses a schema instance to construct a producer instance. - The schema instance defines the schema for the data being produced using the producer instance. + The schema instance defines the schema for the data being produced using the producer instance. - Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. + Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. 2. The producer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. @@ -654,7 +556,7 @@ For how to set `isAllowAutoUpdateSchema` via Pulsar admin API, see [Manage AutoU * If the schema is compatible, the broker stores it and returns the schema version to the producer. - All the messages produced by this producer are tagged with the schema version. + All the messages produced by this producer are tagged with the schema version. * If the schema is incompatible, the broker rejects it. @@ -666,24 +568,24 @@ This diagram illustrates how does Schema work on the consumer side. 1. The application uses a schema instance to construct a consumer instance. - The schema instance defines the schema that the consumer uses for decoding messages received from a broker. + The schema instance defines the schema that the consumer uses for decoding messages received from a broker. 2. The consumer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. 3. The broker determines whether the topic has one of them (a schema/data/a local consumer and a local producer). 4. If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + 5. If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. 6. The consumer receives messages from the broker. - If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. + If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. diff --git a/site2/website-next/docs/security-athenz.md b/site2/website-next/docs/security-athenz.md index d42e97b712e66..d7d26fd5f739f 100644 --- a/site2/website-next/docs/security-athenz.md +++ b/site2/website-next/docs/security-athenz.md @@ -1,7 +1,7 @@ --- id: security-athenz title: Authentication using Athenz -sidebar_label: Authentication using Athenz +sidebar_label: "Authentication using Athenz" --- import Tabs from '@theme/Tabs'; @@ -12,7 +12,7 @@ import TabItem from '@theme/TabItem'; ## Athenz authentication settings -A [decentralized Athenz system](https://github.com/AthenZ/athenz/blob/master/docs/decent_authz_flow.md) contains an [authori**Z**ation **M**anagement **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zms.md) (ZMS) server and an [authori**Z**ation **T**oken **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zts.md) (ZTS) server. +A [decentralized Athenz system](https://github.com/AthenZ/athenz/blob/master/docs/decent_authz_flow.md) contains an [authori**Z**ation **M**anagement **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zms.md) (ZMS) server and an [authori**Z**ation **T**oken **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zts) (ZTS) server. To begin, you need to set up Athenz service access control. You need to create domains for the *provider* (which provides some resources to other services with some authentication/authorization policies) and the *tenant* (which is provisioned to access some resources in a provider). In this case, the provider corresponds to the Pulsar service itself and the tenant corresponds to each application using Pulsar (typically, a [tenant](reference-terminology.md#tenant) in Pulsar). @@ -45,7 +45,7 @@ For more specific steps involving UI, refer to [Example Service Access Control S > ### TLS encryption > > Note that when you are using Athenz as an authentication provider, you had better use TLS encryption -> as it can protect role tokens from being intercepted and reused. (for more details involving TLS encryption see [Architecture - Data Model](https://github.com/AthenZ/athenz/blob/master/docs/data_model.md)). +> as it can protect role tokens from being intercepted and reused. (for more details involving TLS encryption see [Architecture - Data Model](https://github.com/AthenZ/athenz/blob/master/docs/data_model)). In the `conf/broker.conf` configuration file in your Pulsar installation, you need to provide the class name of the Athenz authentication provider as well as a comma-separated list of provider domain names. @@ -79,7 +79,7 @@ For more information on Pulsar client authentication using Athenz, see the follo ## Configure CLI tools for Athenz -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following authentication parameters to the `conf/client.conf` config file to use Athenz with CLI tools of Pulsar: @@ -98,3 +98,4 @@ tlsAllowInsecureConnection=false tlsTrustCertsFilePath=/path/to/cacert.pem ``` + diff --git a/site2/website-next/docs/security-authorization.md b/site2/website-next/docs/security-authorization.md index 6175484fbf237..cdeb99a088fa5 100644 --- a/site2/website-next/docs/security-authorization.md +++ b/site2/website-next/docs/security-authorization.md @@ -1,7 +1,7 @@ --- id: security-authorization title: Authentication and authorization in Pulsar -sidebar_label: Authorization and ACLs +sidebar_label: "Authorization and ACLs" --- import Tabs from '@theme/Tabs'; @@ -30,7 +30,7 @@ superUserRoles=my-super-user-1,my-super-user-2 > A full list of parameters is available in the `conf/broker.conf` file. > You can also find the default values for those parameters in [Broker Configuration](reference-configuration.md#broker). -Typically, you use superuser roles for administrators, clients as well as broker-to-broker authorization. When you use [geo-replication](concepts-replication.md), every broker needs to be able to publish to all the other topics of clusters. +Typically, you use superuser roles for administrators, clients as well as broker-to-broker authorization. When you use [geo-replication](concepts-replication), every broker needs to be able to publish to all the other topics of clusters. You can also enable the authorization for the proxy in the proxy configuration file (`conf/proxy.conf`). Once you enable the authorization on the proxy, the proxy does an additional authorization check before forwarding the request to a broker. If you enable authorization on the broker, the broker checks the authorization of the request when the broker receives the forwarded request. @@ -49,7 +49,7 @@ Another approach is to make the proxy role a superuser. This allows the proxy to You can specify the roles as proxy roles in [`conf/broker.conf`](reference-configuration.md#broker). -```properties +``` proxyRoles=my-proxy-role @@ -62,13 +62,13 @@ superUserRoles=my-super-user-1,my-super-user-2,my-proxy-role Pulsar [instance](reference-terminology.md#instance) administrators or some kind of self-service portal typically provisions a Pulsar [tenant](reference-terminology.md#tenant). -You can manage tenants using the [`pulsar-admin`](reference-pulsar-admin.md) tool. +You can manage tenants using the [`pulsar-admin`](reference-pulsar-admin) tool. ### Create a new tenant The following is an example tenant creation command: -```shell +``` $ bin/pulsar-admin tenants create my-tenant \ --admin-roles my-admin-role \ @@ -82,7 +82,7 @@ A client that successfully identifies itself as having the role `my-admin-role` The structure of topic names in Pulsar reflects the hierarchy between tenants, clusters, and namespaces: -```shell +``` persistent://tenant/namespace/topic @@ -90,11 +90,11 @@ persistent://tenant/namespace/topic ### Manage permissions -You can use [Pulsar Admin Tools](admin-api-permissions.md) for managing permission in Pulsar. +You can use [Pulsar Admin Tools](admin-api-permissions) for managing permission in Pulsar. ### Pulsar admin authentication -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("http://broker:8080") @@ -105,7 +105,7 @@ PulsarAdmin admin = PulsarAdmin.builder() To use TLS: -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("https://broker:8080") @@ -114,3 +114,4 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + diff --git a/site2/website-next/docs/security-bouncy-castle.md b/site2/website-next/docs/security-bouncy-castle.md index 5c7263aac24ad..b5d9f18a9f548 100644 --- a/site2/website-next/docs/security-bouncy-castle.md +++ b/site2/website-next/docs/security-bouncy-castle.md @@ -1,7 +1,7 @@ --- id: security-bouncy-castle title: Bouncy Castle Providers -sidebar_label: Bouncy Castle Providers +sidebar_label: "Bouncy Castle Providers" --- import Tabs from '@theme/Tabs'; @@ -19,7 +19,7 @@ In Pulsar, security and crypto have dependencies on BouncyCastle Jars. For the d `Bouncy Castle` provides both [FIPS](https://www.bouncycastle.org/fips_faq.html) and non-FIPS version. But in a JVM, you can not include both of the 2 versions, and you need to exclude the current version before include the other. -In Pulsar, the security and crypto methods also depends on `Bouncy Castle`, especially in [TLS Authentication](security-tls-authentication.md) and [Transport Encryption](security-encryption.md). This document contains the configuration between BouncyCastle FIPS(BC-FIPS) and non-FIPS(BC-non-FIPS) version while using Pulsar. +In Pulsar, the security and crypto methods also depends on `Bouncy Castle`, especially in [TLS Authentication](security-tls-authentication.md) and [Transport Encryption](security-encryption). This document contains the configuration between BouncyCastle FIPS(BC-FIPS) and non-FIPS(BC-non-FIPS) version while using Pulsar. ## How BouncyCastle modules packaged in Pulsar @@ -34,11 +34,13 @@ Usually, You will meet error like `java.lang.SecurityException: Invalid signatur You could exclude these signatures in mvn pom file to avoid above error, by ```access transformers + META-INF/*.SF META-INF/*.DSA META-INF/*.RSA ``` + But it can also lead to new, cryptic errors, e.g. `java.security.NoSuchAlgorithmException: PBEWithSHA256And256BitAES-CBC-BC SecretKeyFactory not available` By explicitly specifying where to find the algorithm like this: `SecretKeyFactory.getInstance("PBEWithSHA256And256BitAES-CBC-BC","BC")` It will get the real error: `java.security.NoSuchProviderException: JCE cannot authenticate the provider BC` @@ -152,6 +154,7 @@ If you want to switch from BC-non-FIPS to BC-FIPS version, Here is an example fo
    ``` + For more example, you can reference module `bcfips-include-test`. diff --git a/site2/website-next/docs/security-encryption.md b/site2/website-next/docs/security-encryption.md index 7c03e1d56b300..04c32c2233515 100644 --- a/site2/website-next/docs/security-encryption.md +++ b/site2/website-next/docs/security-encryption.md @@ -1,7 +1,7 @@ --- id: security-encryption title: Pulsar Encryption -sidebar_label: End-to-End Encryption +sidebar_label: "End-to-End Encryption" --- import Tabs from '@theme/Tabs'; @@ -102,6 +102,7 @@ producer.close(); pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -179,6 +180,7 @@ If you need to encrypt the messages using 2 keys (myapp.messagekey1 and myapp.me PulsarClient.newProducer().addEncryptionKey("myapp.messagekey1").addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypt encrypted messages at the consumer application Consumers require access one of the private keys to decrypt messages that the producer produces. If you want to receive encrypted messages, create a public or private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/docs/security-extending.md b/site2/website-next/docs/security-extending.md index 1a1880aa28100..4bfcd651b1449 100644 --- a/site2/website-next/docs/security-extending.md +++ b/site2/website-next/docs/security-extending.md @@ -1,7 +1,7 @@ --- id: security-extending title: Extending Authentication and Authorization in Pulsar -sidebar_label: Extending +sidebar_label: "Extending" --- import Tabs from '@theme/Tabs'; @@ -12,7 +12,7 @@ Pulsar provides a way to use custom authentication and authorization mechanisms. ## Authentication -Pulsar supports mutual TLS and Athenz authentication plugins. For how to use these authentication plugins, you can refer to the description in [Security](security-overview.md). +Pulsar supports mutual TLS and Athenz authentication plugins. For how to use these authentication plugins, you can refer to the description in [Security](security-overview). You can use a custom authentication mechanism by providing the implementation in the form of two plugins. One plugin is for the Client library and the other plugin is for the Pulsar Proxy and/or Pulsar Broker to validate the credentials. @@ -54,6 +54,7 @@ In `conf/broker.conf` you can choose to specify a list of valid providers: authenticationProviders= ``` + To implement `org.apache.pulsar.broker.authentication.AuthenticationProvider` on one single interface: ```java @@ -109,10 +110,10 @@ the Authorization plugin is designed only for use on the Broker however the Prox To provide a custom provider, you need to implement the `org.apache.pulsar.broker.authorization.AuthorizationProvider` interface, put this class in the Pulsar broker classpath and configure the class in `conf/broker.conf`: ```properties - + # Authorization provider fully qualified class-name authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - + ``` ```java @@ -181,8 +182,8 @@ public interface AuthorizationProvider extends Closeable { * @param authDataJson * additional authdata in json format * @return CompletableFuture - * @completesWith
    - * IllegalArgumentException when namespace not found
    + * @completesWith
    + * IllegalArgumentException when namespace not found
    * IllegalStateException when failed to grant permission */ CompletableFuture grantPermissionAsync(NamespaceName namespace, Set actions, String role, @@ -196,8 +197,8 @@ public interface AuthorizationProvider extends Closeable { * @param authDataJson * additional authdata in json format * @return CompletableFuture - * @completesWith
    - * IllegalArgumentException when namespace not found
    + * @completesWith
    + * IllegalArgumentException when namespace not found
    * IllegalStateException when failed to grant permission */ CompletableFuture grantPermissionAsync(TopicName topicName, Set actions, String role, @@ -206,3 +207,4 @@ public interface AuthorizationProvider extends Closeable { } ``` + diff --git a/site2/website-next/docs/security-jwt.md b/site2/website-next/docs/security-jwt.md index 21df342702846..3fcca5b87aedc 100644 --- a/site2/website-next/docs/security-jwt.md +++ b/site2/website-next/docs/security-jwt.md @@ -1,7 +1,7 @@ --- id: security-jwt title: Client authentication using tokens based on JSON Web Tokens -sidebar_label: Authentication using JWT +sidebar_label: "Authentication using JWT" --- import Tabs from '@theme/Tabs'; @@ -20,6 +20,7 @@ A user typically gets a token string from the administrator (or some automated s The compact representation of a signed JWT is a string that looks like as the following: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY ``` @@ -28,11 +29,11 @@ Application specifies the token when you create the client instance. An alternat > #### Always use TLS transport encryption > Sending a token is equivalent to sending a password over the wire. You had better use TLS encryption all the time when you connect to the Pulsar service. See -> [Transport Encryption using TLS](security-tls-transport.md) for more details. +> [Transport Encryption using TLS](security-tls-transport) for more details. ### CLI Tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use the token authentication with CLI tools of Pulsar: @@ -48,6 +49,7 @@ authParams=token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPt The token string can also be read from a file, for example: ``` + authParams=file:///path/to/token/file ``` @@ -143,6 +145,7 @@ client, err := NewClient(ClientOptions{ }) ``` + Similarly, you can also pass a `Supplier`: ```go @@ -161,6 +164,7 @@ client, err := NewClient(ClientOptions{ ```c++ + #include pulsar::ClientConfiguration config; @@ -174,6 +178,7 @@ pulsar::Client client("pulsar://broker.example.com:6650/", config); ```c# + var client = PulsarClient.Builder() .AuthenticateUsingToken("eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY") .Build(); @@ -314,7 +319,7 @@ tokenSecretKey=file:///path/to/secret.key To configure proxies to authenticate clients, add the following parameters to `proxy.conf`: -The proxy uses its own token when connecting to brokers. You need to configure the role token for this key pair in the `proxyRoles` of the brokers. For more details, see the [authorization guide](security-authorization.md). +The proxy uses its own token when connecting to brokers. You need to configure the role token for this key pair in the `proxyRoles` of the brokers. For more details, see the [authorization guide](security-authorization). ```properties @@ -335,3 +340,4 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 forwardAuthorizationCredentials=true ``` + diff --git a/site2/website-next/docs/security-kerberos.md b/site2/website-next/docs/security-kerberos.md index 58422ed14f8f1..ad61e38021d71 100644 --- a/site2/website-next/docs/security-kerberos.md +++ b/site2/website-next/docs/security-kerberos.md @@ -1,7 +1,7 @@ --- id: security-kerberos title: Authentication using Kerberos -sidebar_label: Authentication using Kerberos +sidebar_label: "Authentication using Kerberos" --- import Tabs from '@theme/Tabs'; @@ -55,11 +55,13 @@ You need to enter the command below to specify the path to the `krb5.conf` file -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + Here is an example of the krb5.conf file: In the configuration file, `EXAMPLE.COM` is the default realm; `kdc = localhost:62037` is the kdc server url for realm `EXAMPLE.COM `: ``` + [libdefaults] default_realm = EXAMPLE.COM @@ -77,6 +79,7 @@ Usually machines configured with kerberos already have a system wide configurati You need JAAS configuration file for the client side and the broker side. JAAS configuration file provides the section of information that is used to connect KDC. Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -101,7 +104,7 @@ You need to set the `JAAS` configuration file path as JVM parameter for client a ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf ``` @@ -109,7 +112,7 @@ In the `pulsar_jaas.conf` file above 1. `PulsarBroker` is a section name in the JAAS file that each broker uses. This section tells the broker to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarBroker` allows the broker to use the keytab specified in this section. 2. `PulsarClient` is a section name in the JASS file that each broker uses. This section tells the client to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarClient` allows the client to use the keytab specified in this section. - The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. + The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. You can have 2 separate JAAS configuration files: * the file for a broker that has sections of both `PulsarBroker` and `PulsarClient`; @@ -134,6 +137,7 @@ You can have 2 separate JAAS configuration files: Here is an example: ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -151,9 +155,10 @@ brokerClientAuthenticationParameters={"saslJaasClientSectionName":"PulsarClient" ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + You can add this at the end of `PULSAR_EXTRA_OPTS` in the file [`pulsar_env.sh`](https://github.com/apache/pulsar/blob/master/conf/pulsar_env.sh) You must ensure that the operating system user who starts broker can reach the keytabs configured in the `pulsar_jaas.conf` file and kdc server in the `krb5.conf` file. @@ -165,6 +170,7 @@ You must ensure that the operating system user who starts broker can reach the k In client application, include `pulsar-client-auth-sasl` in your project dependency. ``` + org.apache.pulsar pulsar-client-auth-sasl @@ -182,10 +188,9 @@ You need 2 parameters: When you authenticate between client and broker with the setting in above JAAS configuration file, we need to set `saslJaasClientSectionName` to `PulsarClient` and set `serverType` to `broker`. The following is an example of creating a Java client: - ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -200,12 +205,13 @@ The following is an example of creating a Java client: .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -229,7 +235,7 @@ Step 2. Enter the command below to set JVM parameters for JAAS configuration fil ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` @@ -271,6 +277,7 @@ In comparison with the above configuration, add a new section for Pulsar Proxy i Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -305,7 +312,7 @@ Here is an example named `pulsar_jaas.conf`: Pulsar client configuration is similar with client and broker configuration, except that you need to set `serverType` to `proxy` instead of `broker`, for the reason that you need to do the Kerberos authentication between client and proxy. ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -320,12 +327,13 @@ Pulsar client configuration is similar with client and broker configuration, exc .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -358,6 +366,7 @@ The second part relates to authenticating between Pulsar Proxy and Pulsar Broker The broker side configuration file is the same with the above `broker.conf`, you do not need special configuration for Pulsar Proxy. ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -367,7 +376,7 @@ saslJaasBrokerSectionName=PulsarBroker ## Regarding authorization and role token -For Kerberos authentication, we usually use the authenticated principal as the role token for Pulsar authorization. For more information of authorization in Pulsar, see [security authorization](security-authorization.md). +For Kerberos authentication, we usually use the authenticated principal as the role token for Pulsar authorization. For more information of authorization in Pulsar, see [security authorization](security-authorization). If you enable 'authorizationEnabled', you need to set `superUserRoles` in `broker.conf` that corresponds to the name registered in kdc. @@ -384,6 +393,7 @@ superUserRoles=client/{clientIp}@EXAMPLE.COM Pulsar Broker acts as a Kerberos client when you authenticate with Zookeeper. According to [ZooKeeper document](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Client-Server+mutual+authentication), you need these settings in `conf/zookeeper.conf`: ``` + authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider requireClientAuthScheme=sasl @@ -392,6 +402,7 @@ requireClientAuthScheme=sasl Enter the following commands to add a section of `Client` configurations in the file `pulsar_jaas.conf`, which Pulsar Broker uses: ``` + Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -410,6 +421,7 @@ In this setting, the principal of Pulsar Broker and keyTab file indicates the ro Pulsar Broker acts as a Kerberos client when you authenticate with Bookie. According to [BookKeeper document](http://bookkeeper.apache.org/docs/latest/security/sasl/), you need to add `bookkeeperClientAuthenticationPlugin` parameter in `broker.conf`: ``` + bookkeeperClientAuthenticationPlugin=org.apache.bookkeeper.sasl.SASLClientProviderFactory ``` @@ -419,6 +431,7 @@ In this setting, `SASLClientProviderFactory` creates a BookKeeper SASL client in Enter the following commands to add a section of `BookKeeper` configurations in the `pulsar_jaas.conf` that Pulsar Broker uses: ``` + BookKeeper { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true diff --git a/site2/website-next/docs/security-oauth2.md b/site2/website-next/docs/security-oauth2.md index 19c37893f9972..85880647a689f 100644 --- a/site2/website-next/docs/security-oauth2.md +++ b/site2/website-next/docs/security-oauth2.md @@ -1,7 +1,7 @@ --- id: security-oauth2 title: Client authentication using OAuth 2.0 access tokens -sidebar_label: Authentication using OAuth 2.0 access tokens +sidebar_label: "Authentication using OAuth 2.0 access tokens" --- import Tabs from '@theme/Tabs'; @@ -33,6 +33,7 @@ The following table lists parameters supported for the `client credentials` auth | `issuerUrl` | URL of the authentication provider which allows the Pulsar client to obtain an access token | `https://accounts.google.com` | Required | | `privateKey` | URL to a JSON credentials file | Support the following pattern formats:
  • `file:///path/to/file`
  • `file:/path/to/file`
  • `data:application/json;base64,`
  • | Required | | `audience` | An OAuth 2.0 "resource server" identifier for the Pulsar cluster | `https://broker.example.com` | Required | +| `scope` | Scope of an access request.
    For more more information, see [access token scope](https://datatracker.ietf.org/doc/html/rfc6749#section-3.3). | api://pulsar-cluster-1/.default | Optional | The credentials file contains service account credentials used with the client authentication type. The following shows an example of a credentials file `credentials_file.json`. @@ -110,9 +111,10 @@ PulsarClient client = PulsarClient.builder() ### C++ client -The C++ client is similar to the Java client. You need to provide parameters of `issuerUrl`, `private_key` (the credentials file path), and the audience. +The C++ client is similar to the Java client. You need to provide parameters of `issuerUrl`, `private_key` (the credentials file path), and `audience`. ```c++ + #include pulsar::ClientConfiguration config; @@ -178,6 +180,7 @@ This section describes how to use Pulsar CLI tools to connect a cluster through This example shows how to use pulsar-admin to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-admin --admin-url https://streamnative.cloud:443 \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", @@ -195,6 +198,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-client to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-client \ --url SERVICE_URL \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ @@ -213,6 +217,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-perf to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-perf produce --service-url pulsar+ssl://streamnative.cloud:6651 \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", diff --git a/site2/website-next/docs/security-overview.md b/site2/website-next/docs/security-overview.md index 7fed80eef3c39..cf59a9f5c1b87 100644 --- a/site2/website-next/docs/security-overview.md +++ b/site2/website-next/docs/security-overview.md @@ -1,7 +1,7 @@ --- id: security-overview title: Pulsar security overview -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -24,15 +24,15 @@ You had better secure the service components in your Apache Pulsar deployment. In Pulsar, a *role* is a string, like `admin` or `app1`, which can represent a single client or multiple clients. You can use roles to control permission for clients to produce or consume from certain topics, administer the configuration for tenants, and so on. -Apache Pulsar uses a [Authentication Provider](#authentication-providers) to establish the identity of a client and then assign a *role token* to that client. This role token is then used for [Authorization and ACLs](security-authorization.md) to determine what the client is authorized to do. +Apache Pulsar uses a [Authentication Provider](#authentication-providers) to establish the identity of a client and then assign a *role token* to that client. This role token is then used for [Authorization and ACLs](security-authorization) to determine what the client is authorized to do. ## Authentication providers Currently Pulsar supports the following authentication providers: -- [TLS Authentication](security-tls-authentication.md) -- [Athenz](security-athenz.md) -- [Kerberos](security-kerberos.md) -- [JSON Web Token Authentication](security-jwt.md) +- [TLS Authentication](security-tls-authentication) +- [Athenz](security-athenz) +- [Kerberos](security-kerberos) +- [JSON Web Token Authentication](security-jwt) diff --git a/site2/website-next/docs/security-tls-authentication.md b/site2/website-next/docs/security-tls-authentication.md index 47cb11da6b569..1b23394a2819b 100644 --- a/site2/website-next/docs/security-tls-authentication.md +++ b/site2/website-next/docs/security-tls-authentication.md @@ -1,7 +1,7 @@ --- id: security-tls-authentication title: Authentication using TLS -sidebar_label: Authentication using TLS +sidebar_label: "Authentication using TLS" --- import Tabs from '@theme/Tabs'; @@ -10,9 +10,9 @@ import TabItem from '@theme/TabItem'; ## TLS authentication overview -TLS authentication is an extension of [TLS transport encryption](security-tls-transport.md). Not only servers have keys and certs that the client uses to verify the identity of servers, clients also have keys and certs that the server uses to verify the identity of clients. You must have TLS transport encryption configured on your cluster before you can use TLS authentication. This guide assumes you already have TLS transport encryption configured. +TLS authentication is an extension of [TLS transport encryption](security-tls-transport). Not only servers have keys and certs that the client uses to verify the identity of servers, clients also have keys and certs that the server uses to verify the identity of clients. You must have TLS transport encryption configured on your cluster before you can use TLS authentication. This guide assumes you already have TLS transport encryption configured. -`Bouncy Castle Provider` provides TLS related cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle.md). +`Bouncy Castle Provider` provides TLS related cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle). ### Create client certificates @@ -47,6 +47,7 @@ $ openssl req -config openssl.cnf \ -key admin.key.pem -new -sha256 -out admin.csr.pem ``` + :::note If openssl.cnf is not specified, read [Certificate authority](http://pulsar.apache.org/docs/en/security-tls-transport/#certificate-authority) to get the openssl.cnf. @@ -105,7 +106,7 @@ brokerClientTrustCertsFilePath=/path/my-ca/certs/ca.cert.pem To configure proxies to authenticate clients, add the following parameters to `proxy.conf`, alongside [the configuration to enable tls transport](security-tls-transport.md#proxy-configuration): -The proxy should have its own client key pair for connecting to brokers. You need to configure the role token for this key pair in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization.md) for more details. +The proxy should have its own client key pair for connecting to brokers. You need to configure the role token for this key pair in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization) for more details. ```properties @@ -125,11 +126,11 @@ When you use TLS authentication, client connects via TLS transport. You need to ### CLI tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use TLS authentication with the CLI tools of Pulsar: -```properties +``` webServiceUrl=https://broker.example.com:8443/ brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ @@ -143,7 +144,7 @@ authParams=tlsCertFile:/path/to/my-role.cert.pem,tlsKeyFile:/path/to/my-role.key ### Java client -```java +``` import org.apache.pulsar.client.api.PulsarClient; @@ -159,7 +160,7 @@ PulsarClient client = PulsarClient.builder() ### Python client -```python +``` from pulsar import Client, AuthenticationTLS @@ -173,7 +174,8 @@ client = Client("pulsar+ssl://broker.example.com:6651/", ### C++ client -```c++ +``` + #include pulsar::ClientConfiguration config; @@ -191,7 +193,7 @@ pulsar::Client client("pulsar+ssl://broker.example.com:6651/", config); ### Node.js client -```JavaScript +``` const Pulsar = require('pulsar-client'); @@ -212,10 +214,12 @@ const Pulsar = require('pulsar-client'); ### C# client -```c# +``` + var clientCertificate = new X509Certificate2("admin.pfx"); var client = PulsarClient.Builder() .AuthenticateUsingClientCertificate(clientCertificate) .Build(); ``` + diff --git a/site2/website-next/docs/security-tls-keystore.md b/site2/website-next/docs/security-tls-keystore.md index b9b240c522d93..8187608fd25b6 100644 --- a/site2/website-next/docs/security-tls-keystore.md +++ b/site2/website-next/docs/security-tls-keystore.md @@ -1,7 +1,7 @@ --- id: security-tls-keystore title: Using TLS with KeyStore configure -sidebar_label: Using TLS with KeyStore configure +sidebar_label: "Using TLS with KeyStore configure" --- import Tabs from '@theme/Tabs'; @@ -10,7 +10,7 @@ import TabItem from '@theme/TabItem'; ## Overview -Apache Pulsar supports [TLS encryption](security-tls-transport.md) and [TLS authentication](security-tls-authentication.md) between clients and Apache Pulsar service. +Apache Pulsar supports [TLS encryption](security-tls-transport.md) and [TLS authentication](security-tls-authentication) between clients and Apache Pulsar service. By default it uses PEM format file configuration. This page tries to describe use [KeyStore](https://en.wikipedia.org/wiki/Java_KeyStore) type configure for TLS. @@ -31,7 +31,7 @@ keytool -keystore broker.keystore.jks -alias localhost -validity {validity} -gen You need to specify two parameters in the above command: 1. `keystore`: the keystore file that stores the certificate. The *keystore* file contains the private key of - the certificate; hence, it needs to be kept safely. + the certificate; hence, it needs to be kept safely. 2. `validity`: the valid time of the certificate in days. > Ensure that common name (CN) matches exactly with the fully qualified domain name (FQDN) of the server. @@ -154,10 +154,12 @@ NOTE: it is important to restrict access to the store files via filesystem permi If you have configured TLS on the broker, to disable non-TLS ports, you can set the values of the following configurations to empty as below. ``` + brokerServicePort= webServicePort= ``` + In this case, you need to set the following configurations. ```conf @@ -170,50 +172,51 @@ brokerClientTlsTrustStorePassword= // Set this to your desired value Optional settings that may worth consider: 1. tlsClientAuthentication=false: Enable/Disable using TLS for authentication. This config when enabled will authenticate the other end - of the communication channel. It should be enabled on both brokers and clients for mutual TLS. + of the communication channel. It should be enabled on both brokers and clients for mutual TLS. 2. tlsCiphers=[TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256], A cipher suite is a named combination of authentication, encryption, MAC and key exchange - algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, - it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) - [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) + algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, + it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) + [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) 3. tlsProtocols=[TLSv1.3,TLSv1.2] (list out the TLS protocols that you are going to accept from clients). - By default, it is not set. + By default, it is not set. ``` + ### Configuring Clients This is similar to [TLS encryption configuing for client with PEM type](security-tls-transport.md#Client configuration). For a a minimal configuration, user need to provide the TrustStore information. e.g. -1. for [Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - - ``` +1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. + + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .build(); + + ``` 1. for java admin client @@ -230,7 +233,7 @@ e.g. ## TLS authentication with KeyStore configure -This similar to [TLS authentication with PEM type](security-tls-authentication.md) +This similar to [TLS authentication with PEM type](security-tls-authentication) ### broker authentication config @@ -278,56 +281,56 @@ webSocketServiceEnabled=false Besides the TLS encryption configuring. The main work is configuring the KeyStore, which contains a valid CN as client role, for client. e.g. -1. for [Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls - authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} - - ``` +1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. + + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls + authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` 1. for java admin client - ```java - - PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` ## Enabling TLS Logging diff --git a/site2/website-next/docs/security-tls-transport.md b/site2/website-next/docs/security-tls-transport.md index 3cccc1b31a7a2..9ee7fa0aa2db2 100644 --- a/site2/website-next/docs/security-tls-transport.md +++ b/site2/website-next/docs/security-tls-transport.md @@ -1,7 +1,7 @@ --- id: security-tls-transport title: Transport Encryption using TLS -sidebar_label: Transport Encryption using TLS +sidebar_label: "Transport Encryption using TLS" --- import Tabs from '@theme/Tabs'; @@ -12,7 +12,7 @@ import TabItem from '@theme/TabItem'; By default, Apache Pulsar clients communicate with the Apache Pulsar service in plain text. This means that all data is sent in the clear. You can use TLS to encrypt this traffic to protect the traffic from the snooping of a man-in-the-middle attacker. -You can also configure TLS for both encryption and authentication. Use this guide to configure just TLS transport encryption and refer to [here](security-tls-authentication.md) for TLS authentication configuration. Alternatively, you can use [another authentication mechanism](security-athenz.md) on top of TLS transport encryption. +You can also configure TLS for both encryption and authentication. Use this guide to configure just TLS transport encryption and refer to [here](security-tls-authentication.md) for TLS authentication configuration. Alternatively, you can use [another authentication mechanism](security-athenz) on top of TLS transport encryption. > Note that enabling TLS may impact the performance due to encryption overhead. @@ -22,7 +22,7 @@ TLS is a form of [public key cryptography](https://en.wikipedia.org/wiki/Public- To use TLS transport encryption, you need two kinds of key pairs, **server key pairs** and a **certificate authority**. -You can use a third kind of key pair, **client key pairs**, for [client authentication](security-tls-authentication.md). +You can use a third kind of key pair, **client key pairs**, for [client authentication](security-tls-authentication). You should store the **certificate authority** private key in a very secure location (a fully encrypted, disconnected, air gapped computer). As for the certificate authority public key, the **trust cert**, you can freely shared it. @@ -30,9 +30,9 @@ For both client and server key pairs, the administrator first generates a privat For TLS transport encryption, the clients can use the **trust cert** to verify that the server has a key pair that the certificate authority signed when the clients are talking to the server. A man-in-the-middle attacker does not have access to the certificate authority, so they couldn't create a server with such a key pair. -For TLS authentication, the server uses the **trust cert** to verify that the client has a key pair that the certificate authority signed. The common name of the **client cert** is then used as the client's role token (see [Overview](security-overview.md)). +For TLS authentication, the server uses the **trust cert** to verify that the client has a key pair that the certificate authority signed. The common name of the **client cert** is then used as the client's role token (see [Overview](security-overview)). -`Bouncy Castle Provider` provides cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle.md). +`Bouncy Castle Provider` provides cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle). ## Create TLS certificates @@ -131,7 +131,7 @@ At this point, you have a cert, `broker.cert.pem`, and a key, `broker.key-pk8.pe ## Configure broker -To configure a Pulsar [broker](reference-terminology.md#broker) to use TLS transport encryption, you need to make some changes to `broker.conf`, which locates in the `conf` directory of your [Pulsar installation](getting-started-standalone.md). +To configure a Pulsar [broker](reference-terminology.md#broker) to use TLS transport encryption, you need to make some changes to `broker.conf`, which locates in the `conf` directory of your [Pulsar installation](getting-started-standalone). Add these values to the configuration file (substituting the appropriate certificate paths where necessary): @@ -171,7 +171,7 @@ For JDK 11, you can obtain a list of supported values from the documentation: Proxies need to configure TLS in two directions, for clients connecting to the proxy, and for the proxy connecting to brokers. -```properties +``` # For clients connecting to the proxy tlsEnabledInProxy=true @@ -187,7 +187,7 @@ brokerClientTrustCertsFilePath=/path/to/ca.cert.pem ## Client configuration -When you enable the TLS transport encryption, you need to configure the client to use ```https://``` and port 8443 for the web service URL, and ```pulsar+ssl://``` and port 6651 for the broker service URL. +When you enable the TLS transport encryption, you need to configure the client to use ``` As the server certificate that you generated above does not belong to any of the default trust chains, you also need to either specify the path the **trust cert** (recommended), or tell the client to allow untrusted server certs. @@ -203,7 +203,7 @@ The examples below show hostname verification being disabled for the Java client ### CLI tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools.md#pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools.md#pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use TLS transport with the CLI tools of Pulsar: @@ -250,6 +250,7 @@ client = Client("pulsar+ssl://broker.example.com:6651/", #### C++ client ```c++ + #include ClientConfiguration config = ClientConfiguration(); @@ -279,6 +280,7 @@ const Pulsar = require('pulsar-client'); #### C# client ```c# + var certificate = new X509Certificate2("ca.cert.pem"); var client = PulsarClient.Builder() .TrustedCertificateAuthority(certificate) //If the CA is not trusted on the host, you can add it explicitly. @@ -287,3 +289,4 @@ var client = PulsarClient.Builder() .Build(); ``` + diff --git a/site2/website-next/docs/security-token-admin.md b/site2/website-next/docs/security-token-admin.md index 7bb67df9469f9..9c7bdb49031b1 100644 --- a/site2/website-next/docs/security-token-admin.md +++ b/site2/website-next/docs/security-token-admin.md @@ -1,7 +1,7 @@ --- id: security-token-admin title: Token authentication admin -sidebar_label: Token authentication admin +sidebar_label: "Token authentication admin" --- import Tabs from '@theme/Tabs'; @@ -20,9 +20,10 @@ A user will typically be given a token string by an administrator (or some autom The compact representation of a signed JWT is a string that looks like: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY - ``` +``` Application will specify the token when creating the client instance. An alternative is to pass a "token supplier", that is to say a function that returns the token when the client library @@ -31,7 +32,7 @@ will need one. > #### Always use TLS transport encryption > Sending a token is equivalent to sending a password over the wire. It is strongly recommended to > always use TLS encryption when talking to the Pulsar service. See -> [Transport Encryption using TLS](security-tls-transport.md) +> [Transport Encryption using TLS](security-tls-transport) ## Secret vs Public/Private keys @@ -58,6 +59,7 @@ the brokers to allow them to validate the clients. $ bin/pulsar tokens create-secret-key --output my-secret.key ``` + To generate base64 encoded private key ```shell @@ -164,7 +166,7 @@ tokenSecretKey=file:///path/to/secret.key To configure proxies to authenticate clients, put the following in `proxy.conf`: The proxy will have its own token used when talking to brokers. The role token for this -key pair should be configured in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization.md) for more details. +key pair should be configured in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization) for more details. ```properties @@ -180,4 +182,5 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 # Or, alternatively, read token from file # brokerClientAuthenticationParameters=file:///path/to/proxy-token.txt -``` \ No newline at end of file +``` + diff --git a/site2/website-next/docs/sql-deployment-configurations.md b/site2/website-next/docs/sql-deployment-configurations.md index ed181d8aa9673..0a74c33f1a454 100644 --- a/site2/website-next/docs/sql-deployment-configurations.md +++ b/site2/website-next/docs/sql-deployment-configurations.md @@ -1,7 +1,7 @@ --- id: sql-deployment-configurations title: Pulsar SQL configuration and deployment -sidebar_label: Configuration and deployment +sidebar_label: "Configuration and deployment" --- import Tabs from '@theme/Tabs'; @@ -14,6 +14,7 @@ You can configure Presto Pulsar connector and deploy a cluster with the followin You can configure Presto Pulsar Connector in the `${project.root}/conf/presto/catalog/pulsar.properties` properties file. The configuration for the connector and the default values are as follows. ```properties + # name of the connector to be displayed in the catalog connector.name=pulsar @@ -28,13 +29,16 @@ pulsar.entry-read-batch-size=100 # default number of splits to use per query pulsar.target-num-splits=4 + ``` You can connect Presto to a Pulsar cluster with multiple hosts. To configure multiple hosts for brokers, add multiple URLs to `pulsar.web-service-url`. To configure multiple hosts for ZooKeeper, add multiple URIs to `pulsar.zookeeper-uri`. The following is an example. - + ``` + pulsar.web-service-url=http://localhost:8080,localhost:8081,localhost:8082 pulsar.zookeeper-uri=localhost1,localhost2:2181 + ``` ## Query data from existing Presto clusters @@ -42,7 +46,9 @@ pulsar.zookeeper-uri=localhost1,localhost2:2181 If you already have a Presto cluster, you can copy the Presto Pulsar connector plugin to your existing cluster. Download the archived plugin package with the following command. ```bash + $ wget pulsar:binary_release_url + ``` ## Deploy a new cluster @@ -51,14 +57,14 @@ Since Pulsar SQL is powered by [Trino (formerly Presto SQL)](https://trino.io), :::note -For how to set up a standalone single node environment, refer to [Query data](sql-getting-started.md). +For how to set up a standalone single node environment, refer to [Query data](sql-getting-started). ::: - You can use the same CLI args as the Presto launcher. ```bash + $ ./bin/pulsar sql-worker --help Usage: launcher [options] command @@ -92,13 +98,17 @@ The default configuration for the cluster is located in `${project.root}/conf/pr You can set the worker to read from a different configuration directory, or set a different directory to write data. ```bash + $ ./bin/pulsar sql-worker run --etc-dir /tmp/incubator-pulsar/conf/presto --data-dir /tmp/presto-1 + ``` You can start the worker as daemon process. ```bash + $ ./bin/pulsar sql-worker start + ``` ### Deploy a cluster on multiple nodes @@ -110,6 +120,7 @@ You can deploy a Pulsar SQL cluster or Presto cluster on multiple nodes. The fol The first node runs as Presto coordinator. The minimal configuration requirement in the `${project.root}/conf/presto/config.properties` file is as follows. ```properties + coordinator=true node-scheduler.include-coordinator=true http-server.http.port=8080 @@ -117,16 +128,19 @@ query.max-memory=50GB query.max-memory-per-node=1GB discovery-server.enabled=true discovery.uri= + ``` The other two nodes serve as worker nodes, you can use the following configuration for worker nodes. ```properties + coordinator=false http-server.http.port=8080 query.max-memory=50GB query.max-memory-per-node=1GB discovery.uri= + ``` 2. Modify `pulsar.web-service-url` and `pulsar.zookeeper-uri` configuration in the `${project.root}/conf/presto/catalog/pulsar.properties` file accordingly for the three nodes. @@ -134,30 +148,38 @@ discovery.uri= 3. Start the coordinator node. ``` + $ ./bin/pulsar sql-worker run + ``` 4. Start worker nodes. ``` + $ ./bin/pulsar sql-worker run + ``` 5. Start the SQL CLI and check the status of your cluster. ```bash + $ ./bin/pulsar sql --server + ``` 6. Check the status of your nodes. ```bash + presto> SELECT * FROM system.runtime.nodes; node_id | http_uri | node_version | coordinator | state ---------+-------------------------+--------------+-------------+-------- 1 | http://192.168.2.1:8081 | testversion | true | active 3 | http://192.168.2.2:8081 | testversion | false | active - 2 | http://192.168.2.3:8081 | testversion | false | active + 2 | http://192.168.2.3:8081 | testversion | false | active + ``` For more information about deployment in Presto, refer to [Presto deployment](https://trino.io/docs/current/installation/deployment.html). diff --git a/site2/website-next/docs/sql-getting-started.md b/site2/website-next/docs/sql-getting-started.md index e47979b81ec1a..ada474a8bd158 100644 --- a/site2/website-next/docs/sql-getting-started.md +++ b/site2/website-next/docs/sql-getting-started.md @@ -1,7 +1,7 @@ --- id: sql-getting-started title: Query data with Pulsar SQL -sidebar_label: Query data +sidebar_label: "Query data" --- import Tabs from '@theme/Tabs'; @@ -20,24 +20,31 @@ To query data in Pulsar with Pulsar SQL, complete the following steps. 1. Start a Pulsar standalone cluster. ```bash + ./bin/pulsar standalone + ``` 2. Start a Pulsar SQL worker. ```bash + ./bin/pulsar sql-worker run + ``` 3. After initializing Pulsar standalone cluster and the SQL worker, run SQL CLI. ```bash + ./bin/pulsar sql + ``` 4. Test with SQL commands. ```bash + presto> show catalogs; Catalog --------- @@ -80,12 +87,15 @@ Since there is no data in Pulsar, no records is returned. 5. Start the built-in connector _DataGeneratorSource_ and ingest some mock data. ```bash + ./bin/pulsar-admin sources create --name generator --destinationTopicName generator_test --source-type data-generator + ``` And then you can query a topic in the namespace "public/default". ```bash + presto> show tables in pulsar."public/default"; Table ---------------- @@ -95,11 +105,13 @@ presto> show tables in pulsar."public/default"; Query 20180829_213202_00000_csyeu, FINISHED, 1 node Splits: 19 total, 19 done (100.00%) 0:02 [1 rows, 38B] [0 rows/s, 17B/s] + ``` You can now query the data within the topic "generator_test". ```bash + presto> select * from pulsar."public/default".generator_test; firstname | middlename | lastname | email | username | password | telephonenumber | age | companyemail | nationalidentitycardnumber | @@ -113,6 +125,7 @@ presto> select * from pulsar."public/default".generator_test; . . . + ``` You can query the mock data. @@ -121,6 +134,7 @@ You can query the mock data. If you want to query your own data, you need to ingest your own data first. You can write a simple producer and write custom defined data to Pulsar. The following is an example. ```java + public class TestProducer { public static class Foo { @@ -171,4 +185,6 @@ public class TestProducer { pulsarClient.close(); } } + ``` + diff --git a/site2/website-next/docs/sql-overview.md b/site2/website-next/docs/sql-overview.md index d99f672477f8a..9597e2c268df7 100644 --- a/site2/website-next/docs/sql-overview.md +++ b/site2/website-next/docs/sql-overview.md @@ -1,14 +1,14 @@ --- id: sql-overview title: Pulsar SQL Overview -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -Apache Pulsar is used to store streams of event data, and the event data is structured with predefined fields. With the implementation of the [Schema Registry](schema-get-started.md), you can store structured data in Pulsar and query the data by using [Trino (formerly Presto SQL)](https://trino.io/). +Apache Pulsar is used to store streams of event data, and the event data is structured with predefined fields. With the implementation of the [Schema Registry](schema-get-started), you can store structured data in Pulsar and query the data by using [Trino (formerly Presto SQL)](https://trino.io/). As the core of Pulsar SQL, Presto Pulsar connector enables Presto workers within a Presto cluster to query data from Pulsar. diff --git a/site2/website-next/docs/sql-rest-api.md b/site2/website-next/docs/sql-rest-api.md index 5ee292f78fa63..cf3cad867e2b8 100644 --- a/site2/website-next/docs/sql-rest-api.md +++ b/site2/website-next/docs/sql-rest-api.md @@ -1,7 +1,7 @@ --- id: sql-rest-api title: Pulsar SQL REST APIs -sidebar_label: REST APIs +sidebar_label: "REST APIs" --- import Tabs from '@theme/Tabs'; @@ -19,7 +19,9 @@ To request services, use explicit URL `http://presto.service:8081/v1`. You need `POST` requests require the `X-Presto-User` header. If you use authentication, you must use the same `username` that is specified in the authentication configuration. If you do not use authentication, you can specify anything for `username`. ```properties + X-Presto-User: username + ``` For more information about headers, refer to [PrestoHeaders](https://github.com/trinodb/trino). @@ -31,6 +33,7 @@ You can use statement in the HTTP body. All data is received as JSON document th The following is an example of `show catalogs`. The query continues until the received JSON document does not contain a `nextUri` link. Since no `error` is displayed in `stats`, it means that the query completes successfully. ```powershell + ➜ ~ curl --header "X-Presto-User: test-user" --request POST --data 'show catalogs' http://localhost:8081/v1/statement { "infoUri" : "http://localhost:8081/ui/query.html?20191113_033653_00006_dg6hb", @@ -180,6 +183,7 @@ The following is an example of `show catalogs`. The query continues until the re "completedSplits" : 19 } } + ``` :::note @@ -188,5 +192,4 @@ Since the response data is not in sync with the query state from the perspective ::: - For more information about Presto REST API, refer to [Presto HTTP Protocol](https://github.com/prestosql/presto/wiki/HTTP-Protocol). diff --git a/site2/website-next/docs/tiered-storage-aliyun.md b/site2/website-next/docs/tiered-storage-aliyun.md index 0095da2e9729f..8729575b3f72d 100644 --- a/site2/website-next/docs/tiered-storage-aliyun.md +++ b/site2/website-next/docs/tiered-storage-aliyun.md @@ -1,7 +1,7 @@ --- id: tiered-storage-aliyun title: Use Aliyun OSS offloader with Pulsar -sidebar_label: Aliyun OSS offloader +sidebar_label: "Aliyun OSS offloader" --- import Tabs from '@theme/Tabs'; @@ -26,27 +26,28 @@ This example uses Pulsar 2.8.0. 2. Download and untar the Pulsar offloaders package, then copy the Pulsar offloaders as `offloaders` in the Pulsar directory, see [here](https://pulsar.apache.org/docs/en/standalone/#install-tiered-storage-offloaders-optional). - **Output** - - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - - ``` - tiered-storage-file-system-2.8.0.nar - tiered-storage-jcloud-2.8.0.nar - ``` + **Output** + + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. -:::note + ``` + + tiered-storage-file-system-2.8.0.nar + tiered-storage-jcloud-2.8.0.nar + + ``` -* If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + :::note -::: + * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + + ::: ## Configuration :::note - Before offloading data from BookKeeper to Aliyun OSS, you need to configure some properties of the Aliyun OSS offload driver. ::: @@ -59,21 +60,21 @@ You can configure the Aliyun OSS offloader driver in the configuration file `bro - **Required** configurations are as below. - | Required configuration | Description | Example value | - | --- | --- |--- | - | `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | aliyun-oss | - | `offloadersDirectory` | Offloader directory | offloaders | - | `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload | - | `managedLedgerOffloadServiceEndpoint` | Endpoint | http://oss-cn-hongkong.aliyuncs.com | + | Required configuration | Description | Example value | + | --- | --- |--- | + | `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | aliyun-oss | + | `offloadersDirectory` | Offloader directory | offloaders | + | `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload | + | `managedLedgerOffloadServiceEndpoint` | Endpoint | http://oss-cn-hongkong.aliyuncs.com | - **Optional** configurations are as below. - | Optional | Description | Example value | - | --- | --- | --- | - | `managedLedgerOffloadReadBufferSizeInBytes` | Size of block read | 1 MB | - | `managedLedgerOffloadMaxBlockSizeInBytes` | Size of block write | 64 MB | - | `managedLedgerMinLedgerRolloverTimeMinutes` | Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment. | 2 | - | `managedLedgerMaxEntriesPerLedger` | Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment. | 5000 | + | Optional | Description | Example value | + | --- | --- | --- | + | `managedLedgerOffloadReadBufferSizeInBytes` | Size of block read | 1 MB | + | `managedLedgerOffloadMaxBlockSizeInBytes` | Size of block write | 64 MB | + | `managedLedgerMinLedgerRolloverTimeMinutes` | Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment. | 2 | + | `managedLedgerMaxEntriesPerLedger` | Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment. | 5000 | #### Bucket (required) @@ -84,7 +85,9 @@ A bucket is a basic container that holds your data. Everything you store in Aliy This example names the bucket as _pulsar-topic-offload_. ```conf + managedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Endpoint (required) @@ -93,17 +96,19 @@ The endpoint is the region where a bucket is located. :::tip - For more information about Aliyun OSS regions and endpoints, see [International website](https://www.alibabacloud.com/help/doc-detail/31837.htm) or [Chinese website](https://help.aliyun.com/document_detail/31837.html). ::: + ##### Example This example sets the endpoint as _oss-us-west-1-internal_. ``` + managedLedgerOffloadServiceEndpoint=http://oss-us-west-1-internal.aliyuncs.com + ``` #### Authentication (required) @@ -115,8 +120,10 @@ Set the environment variables `ALIYUN_OSS_ACCESS_KEY_ID` and `ALIYUN_OSS_ACCESS_ "export" is important so that the variables are made available in the environment of spawned processes. ```bash + export ALIYUN_OSS_ACCESS_KEY_ID=ABC123456789 export ALIYUN_OSS_ACCESS_KEY_SECRET=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + ``` #### Size of block read/write @@ -149,12 +156,13 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the Aliyun OSS offloader threshold size to 10 MB using pulsar-admin. ```bash + bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -167,69 +175,86 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Aliyun OSS until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Aliyun OSS until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the Aliyun OSS offloader to run manually using pulsar-admin. - ```bash - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -::: + ::: - This example checks the Aliyun OSS offloader status using pulsar-admin. - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` + + To wait for the Aliyun OSS offloader to complete the job, add the `-w` flag. + + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - To wait for the Aliyun OSS offloader to complete the job, add the `-w` flag. + **Output** - ```bash - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ``` + + Offload was a success + + ``` - **Output** - - ``` - Offload was a success - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + **Output** - **Output** + ``` + + Error in offload + null - ``` - Error in offload - null + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + :::tip -:::tip + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + ::: -::: diff --git a/site2/website-next/docs/tiered-storage-aws.md b/site2/website-next/docs/tiered-storage-aws.md index 577b94f7a7667..ee9eeef60fa3b 100644 --- a/site2/website-next/docs/tiered-storage-aws.md +++ b/site2/website-next/docs/tiered-storage-aws.md @@ -1,7 +1,7 @@ --- id: tiered-storage-aws title: Use AWS S3 offloader with Pulsar -sidebar_label: AWS S3 offloader +sidebar_label: "AWS S3 offloader" --- import Tabs from '@theme/Tabs'; @@ -30,46 +30,53 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget): - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: ## Configuration :::note - Before offloading data from BookKeeper to AWS S3, you need to configure some properties of the AWS S3 offload driver. ::: @@ -82,21 +89,21 @@ You can configure the AWS S3 offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 - `offloadersDirectory` | Offloader directory | offloaders - `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 + `offloadersDirectory` | Offloader directory | offloaders + `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 - `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 + `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -107,7 +114,9 @@ A bucket is a basic container that holds your data. Everything you store in AWS This example names the bucket as _pulsar-topic-offload_. ```conf + s3ManagedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Bucket region @@ -116,17 +125,19 @@ A bucket region is a region where a bucket is located. If a bucket region is not :::tip - For more information about AWS regions and endpoints, see [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). ::: + ##### Example This example sets the bucket region as _europe-west-3_. ``` + s3ManagedLedgerOffloadRegion=eu-west-3 + ``` #### Authentication (required) @@ -140,41 +151,49 @@ Once you have created a set of credentials in the AWS IAM console, you can confi * Use EC2 instance metadata credentials. - If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. + If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. * Set the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. + "export" is important so that the variables are made available in the environment of spawned processes. - ```bash - export AWS_ACCESS_KEY_ID=ABC123456789 - export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```bash + + export AWS_ACCESS_KEY_ID=ABC123456789 + export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` * Add the Java system properties `aws.accessKeyId` and `aws.secretKey` to `PULSAR_EXTRA_OPTS` in `conf/pulsar_env.sh`. - ```bash - PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" - ``` + ```bash + + PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" + + ``` * Set the access credentials in `~/.aws/credentials`. - ```conf - [default] - aws_access_key_id=ABC123456789 - aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```conf + + [default] + aws_access_key_id=ABC123456789 + aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` * Assume an IAM role. - This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. + This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. - The broker must be rebooted for credentials specified in `pulsar_env` to take effect. + The broker must be rebooted for credentials specified in `pulsar_env` to take effect. - ```conf - s3ManagedLedgerOffloadRole= - s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload - ``` + ```conf + + s3ManagedLedgerOffloadRole= + s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload + + ``` #### Size of block read/write @@ -206,12 +225,13 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the AWS S3 offloader threshold size to 10 MB using pulsar-admin. ```bash + bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -224,72 +244,88 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the AWS S3 offloader to run manually using pulsar-admin. - ```bash - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -::: + ::: - This example checks the AWS S3 offloader status using pulsar-admin. - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the AWS S3 offloader to complete the job, add the `-w` flag. + To wait for the AWS S3 offloader to complete the job, add the `-w` flag. - ```bash - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` + + Error in offload + null -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + :::tip -::: + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + + ::: ## Tutorial diff --git a/site2/website-next/docs/tiered-storage-azure.md b/site2/website-next/docs/tiered-storage-azure.md index 44003afe7871f..f7b11288f9e51 100644 --- a/site2/website-next/docs/tiered-storage-azure.md +++ b/site2/website-next/docs/tiered-storage-azure.md @@ -1,7 +1,7 @@ --- id: tiered-storage-azure title: Use Azure BlobStore offloader with Pulsar -sidebar_label: Azure BlobStore offloader +sidebar_label: "Azure BlobStore offloader" --- import Tabs from '@theme/Tabs'; @@ -30,46 +30,53 @@ This example uses Pulsar 2.6.2. * Use [wget](https://www.gnu.org/software/wget): - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders + ``` + + mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. - ``` - tiered-storage-file-system-2.6.2.nar - tiered-storage-jcloud-2.6.2.nar - ``` + ``` + + tiered-storage-file-system-2.6.2.nar + tiered-storage-jcloud-2.6.2.nar + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: ## Configuration :::note - Before offloading data from BookKeeper to Azure BlobStore, you need to configure some properties of the Azure BlobStore offload driver. ::: @@ -82,20 +89,20 @@ You can configure the Azure BlobStore offloader driver in the configuration file - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name | azureblob - `offloadersDirectory` | Offloader directory | offloaders - `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name | azureblob + `offloadersDirectory` | Offloader directory | offloaders + `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -106,7 +113,9 @@ A bucket is a basic container that holds your data. Everything you store in Azur This example names the bucket as _pulsar-topic-offload_. ```conf + managedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Authentication (required) @@ -115,12 +124,14 @@ To be able to access Azure BlobStore, you need to authenticate with Azure BlobSt * Set the environment variables `AZURE_STORAGE_ACCOUNT` and `AZURE_STORAGE_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. + "export" is important so that the variables are made available in the environment of spawned processes. - ```bash - export AZURE_STORAGE_ACCOUNT=ABC123456789 - export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```bash + + export AZURE_STORAGE_ACCOUNT=ABC123456789 + export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` #### Size of block read/write @@ -152,12 +163,13 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the Azure BlobStore offloader threshold size to 10 MB using pulsar-admin. ```bash + bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -170,69 +182,86 @@ For individual topics, you can trigger Azure BlobStore offloader manually using - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the Azure BlobStore offloader to run manually using pulsar-admin. - ```bash - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -::: + ::: - This example checks the Azure BlobStore offloader status using pulsar-admin. - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. + To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. - ```bash - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` + + Error in offload + null -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + :::tip + + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + + ::: -::: diff --git a/site2/website-next/docs/tiered-storage-filesystem.md b/site2/website-next/docs/tiered-storage-filesystem.md index cea5c1858046b..d59d89a8a0d95 100644 --- a/site2/website-next/docs/tiered-storage-filesystem.md +++ b/site2/website-next/docs/tiered-storage-filesystem.md @@ -1,7 +1,7 @@ --- id: tiered-storage-filesystem title: Use filesystem offloader with Pulsar -sidebar_label: Filesystem offloader +sidebar_label: "Filesystem offloader" --- import Tabs from '@theme/Tabs'; @@ -32,39 +32,47 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` :::note @@ -77,7 +85,6 @@ This example uses Pulsar 2.5.1. :::note - Before offloading data from BookKeeper to filesystem, you need to configure some properties of the filesystem offloader driver. ::: @@ -90,18 +97,18 @@ You can configure filesystem offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem - `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 - `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem + `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 + `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml - **Optional** configurations are as below. - Optional configuration| Description | Example value - |---|---|--- - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional configuration| Description | Example value + |---|---|--- + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Offloader driver (required) @@ -110,7 +117,9 @@ Offloader driver name, which is case-insensitive. This example sets the offloader driver name as _filesystem_. ```conf + managedLedgerOffloadDriver=filesystem + ``` #### Connection address (required) @@ -122,7 +131,9 @@ Connection address is the URI to access the default Hadoop distributed file syst This example sets the connection address as _hdfs://127.0.0.1:9000_. ```conf + fileSystemURI=hdfs://127.0.0.1:9000 + ``` #### Hadoop profile path (required) @@ -134,12 +145,15 @@ The configuration file is stored in the Hadoop profile path. It contains various This example sets the Hadoop profile path as _../conf/filesystem_offload_core_site.xml_. ```conf + fileSystemProfilePath=../conf/filesystem_offload_core_site.xml + ``` You can set the following configurations in the _filesystem_offload_core_site.xml_ file. ``` + fs.defaultFS @@ -169,11 +183,11 @@ You can set the following configurations in the _filesystem_offload_core_site.xm io.map.index.interval 128 + ``` :::tip - For more information about the Hadoop HDFS, see [here](https://hadoop.apache.org/docs/current/). ::: @@ -197,12 +211,13 @@ You can configure the threshold size using CLI tools, such as pulsar-admin. This example sets the filesystem offloader threshold size to 10 MB using pulsar-admin. ```bash + pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#set-offload-threshold). ::: @@ -221,66 +236,82 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh - This example triggers the filesystem offloader to run manually using pulsar-admin. - ```bash - pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). -::: + ::: - This example checks filesystem offloader status using pulsar-admin. - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the filesystem to complete the job, add the `-w` flag. + To wait for the filesystem to complete the job, add the `-w` flag. - ```bash - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. + + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ``` - Error in offload - null + ``` + + Error in offload + null - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). -::: + ::: ## Tutorial diff --git a/site2/website-next/docs/tiered-storage-gcs.md b/site2/website-next/docs/tiered-storage-gcs.md index 3f20a1cd4b81a..17dd758b3ad74 100644 --- a/site2/website-next/docs/tiered-storage-gcs.md +++ b/site2/website-next/docs/tiered-storage-gcs.md @@ -1,7 +1,7 @@ --- id: tiered-storage-gcs title: Use GCS offloader with Pulsar -sidebar_label: GCS offloader +sidebar_label: "GCS offloader" --- import Tabs from '@theme/Tabs'; @@ -30,47 +30,54 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. + As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` ## Configuration :::note - Before offloading data from BookKeeper to GCS, you need to configure some properties of the GCS offloader driver. ::: @@ -83,22 +90,22 @@ You can configure GCS offloader driver in the configuration file `broker.conf` o - **Required** configurations are as below. - **Required** configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage - `offloadersDirectory`|Offloader directory|offloaders - `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload - `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 - `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json + **Required** configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage + `offloadersDirectory`|Offloader directory|offloaders + `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload + `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 + `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json - **Optional** configurations are as below. - Optional configuration|Description|Example value - |---|---|--- - `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 - `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 + Optional configuration|Description|Example value + |---|---|--- + `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 + `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 #### Bucket (required) @@ -109,7 +116,9 @@ A bucket is a basic container that holds your data. Everything you store in GCS This example names the bucket as _pulsar-topic-offload_. ```conf + gcsManagedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Bucket region (required) @@ -118,7 +127,6 @@ Bucket region is the region where a bucket is located. If a bucket region is not :::tip - For more information about bucket location, see [here](https://cloud.google.com/storage/docs/bucket-locations). ::: @@ -128,7 +136,9 @@ For more information about bucket location, see [here](https://cloud.google.com/ This example sets the bucket region as _europe-west3_. ``` + gcsManagedLedgerOffloadRegion=europe-west3 + ``` #### Authentication (required) @@ -150,7 +160,7 @@ To generate service account credentials or view the public credentials that you' 4. In the **Create service account** window, type a name for the service account and select **Furnish a new private key**. - If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. + If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. 5. Click **Create**. @@ -161,10 +171,12 @@ To generate service account credentials or view the public credentials that you' ::: 6. You can get the following information and set this in `broker.conf`. + + ```conf - ```conf - gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" - ``` + gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" + + ``` :::tip @@ -203,12 +215,13 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the GCS offloader threshold size to 10 MB using pulsar-admin. ```bash + pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#set-offload-threshold). ::: @@ -221,72 +234,88 @@ For individual topics, you can trigger GCS offloader manually using one of the f - Use CLI tools (such as pulsar-admin). - To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. + To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the GCS offloader to run manually using pulsar-admin with the command `pulsar-admin topics offload (topic-name) (threshold)`. - ```bash - pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M - ``` + ```bash + + pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). -::: + ::: - This example checks the GCS offloader status using pulsar-admin with the command `pulsar-admin topics offload-status options`. - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for GCS to complete the job, add the `-w` flag. + To wait for GCS to complete the job, add the `-w` flag. - ```bash - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ``` - Offload was a success - ``` + ``` + + Offload was a success + + ``` - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - ```bash + ```bash + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + + ``` - **Output** + **Output** - ``` - Error in offload - null + ``` + + Error in offload + null - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). -::: + ::: ## Tutorial diff --git a/site2/website-next/docs/tiered-storage-overview.md b/site2/website-next/docs/tiered-storage-overview.md index 7799d7f31ccc3..e39d8a72a0a96 100644 --- a/site2/website-next/docs/tiered-storage-overview.md +++ b/site2/website-next/docs/tiered-storage-overview.md @@ -1,7 +1,7 @@ --- id: tiered-storage-overview title: Overview of tiered storage -sidebar_label: Overview +sidebar_label: "Overview" --- import Tabs from '@theme/Tabs'; @@ -14,22 +14,22 @@ Pulsar's **Tiered Storage** feature allows older backlog data to be moved from B With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -:::tip + :::tip -For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws.md). -For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs.md). + For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws). + For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs). -::: + ::: * Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystems for long term storage. - With Hadoop, it is easy to add support for more filesystems in the future. + With Hadoop, it is easy to add support for more filesystems in the future. -:::tip + :::tip -For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem.md). + For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem). -::: + ::: ## When to use tiered storage? diff --git a/site2/website-next/docs/transaction-api.md b/site2/website-next/docs/transaction-api.md new file mode 100644 index 0000000000000..f76a928395f20 --- /dev/null +++ b/site2/website-next/docs/transaction-api.md @@ -0,0 +1,175 @@ +--- +id: transactions-api +title: Transactions API +sidebar_label: "Transactions API" +--- + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +All messages in a transaction are available only to consumers after the transaction has been committed. If a transaction has been aborted, all the writes and acknowledgments in this transaction roll back. + +## Prerequisites +1. To enable transactions in Pulsar, you need to configure the parameter in the `broker.conf` file. + +``` + +transactionCoordinatorEnabled=true + +``` + +2. Initialize transaction coordinator metadata, so the transaction coordinators can leverage advantages of the partitioned topic, such as load balance. + +``` + +bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone + +``` + +After initializing transaction coordinator metadata, you can use the transactions API. The following APIs are available. + +## Initialize Pulsar client + +You can enable transaction for transaction client and initialize transaction coordinator client. + +``` + +PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl("pulsar://localhost:6650") + .enableTransaction(true) + .build(); + +``` + +## Start transactions +You can start transaction in the following way. + +``` + +Transaction txn = pulsarClient + .newTransaction() + .withTransactionTimeout(5, TimeUnit.MINUTES) + .build() + .get(); + +``` + +## Produce transaction messages + +A transaction parameter is required when producing new transaction messages. The semantic of the transaction messages in Pulsar is `read-committed`, so the consumer cannot receive the ongoing transaction messages before the transaction is committed. + +``` + +producer.newMessage(txn).value("Hello Pulsar Transaction".getBytes()).sendAsync(); + +``` + +## Acknowledge the messages with the transaction + +The transaction acknowledgement requires a transaction parameter. The transaction acknowledgement marks the messages state to pending-ack state. When the transaction is committed, the pending-ack state becomes ack state. If the transaction is aborted, the pending-ack state becomes unack state. + +``` + +Message message = consumer.receive(); +consumer.acknowledgeAsync(message.getMessageId(), txn); + +``` + +## Commit transactions + +When the transaction is committed, consumers receive the transaction messages and the pending-ack state becomes ack state. + +``` + +txn.commit().get(); + +``` + +## Abort transaction + +When the transaction is aborted, the transaction acknowledgement is canceled and the pending-ack messages are redelivered. + +``` + +txn.abort().get(); + +``` + +### Example +The following example shows how messages are processed in transaction. + +``` + +PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(getPulsarServiceList().get(0).getBrokerServiceUrl()) + .statsInterval(0, TimeUnit.SECONDS) + .enableTransaction(true) + .build(); + +String sourceTopic = "public/default/source-topic"; +String sinkTopic = "public/default/sink-topic"; + +Producer sourceProducer = pulsarClient + .newProducer(Schema.STRING) + .topic(sourceTopic) + .create(); +sourceProducer.newMessage().value("hello pulsar transaction").sendAsync(); + +Consumer sourceConsumer = pulsarClient + .newConsumer(Schema.STRING) + .topic(sourceTopic) + .subscriptionName("test") + .subscriptionType(SubscriptionType.Shared) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + +Producer sinkProducer = pulsarClient + .newProducer(Schema.STRING) + .topic(sinkTopic) + .sendTimeout(0, TimeUnit.MILLISECONDS) + .create(); + +Transaction txn = pulsarClient + .newTransaction() + .withTransactionTimeout(5, TimeUnit.MINUTES) + .build() + .get(); + +// source message acknowledgement and sink message produce belong to one transaction, +// they are combined into an atomic operation. +Message message = sourceConsumer.receive(); +sourceConsumer.acknowledgeAsync(message.getMessageId(), txn); +sinkProducer.newMessage(txn).value("sink data").sendAsync(); + +txn.commit().get(); + +``` + +## Enable batch messages in transactions + +To enable batch messages in transactions, you need to enable the batch index acknowledgement feature. The transaction acks check whether the batch index acknowledgement conflicts. + +To enable batch index acknowledgement, you need to set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. + +``` + +acknowledgmentAtBatchIndexLevelEnabled=true + +``` + +And then you need to call the `enableBatchIndexAcknowledgment(true)` method in the consumer builder. + +``` + +Consumer sinkConsumer = pulsarClient + .newConsumer() + .topic(transferTopic) + .subscriptionName("sink-topic") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Shared) + .enableBatchIndexAcknowledgment(true) // enable batch index acknowledgement + .subscribe(); + +``` + diff --git a/site2/website-next/docs/transaction-guarantee.md b/site2/website-next/docs/transaction-guarantee.md new file mode 100644 index 0000000000000..e18818ae0ae8c --- /dev/null +++ b/site2/website-next/docs/transaction-guarantee.md @@ -0,0 +1,20 @@ +--- +id: transactions-guarantee +title: Transactions Guarantee +sidebar_label: "Transactions Guarantee" +--- + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar transactions support the following guarantee. + +## Atomic multi-partition writes and multi-subscription acknowledges +Transactions enable atomic writes to multiple topics and partitions. A batch of messages in a transaction can be received from, produced to, and acknowledged by many partitions. All the operations involved in a transaction succeed or fail as a single unit. + +## Read transactional message +All the messages in a transaction are available only for consumers until the transaction is committed. + +## Acknowledge transactional message +A message is acknowledged successfully only once by a consumer under the subscription when acknowledging the message with the transaction ID. \ No newline at end of file diff --git a/site2/website-next/docs/txn-how.md b/site2/website-next/docs/txn-how.md index 81f9df18a99bd..f38e40de541f3 100644 --- a/site2/website-next/docs/txn-how.md +++ b/site2/website-next/docs/txn-how.md @@ -1,7 +1,7 @@ --- id: txn-how title: How transactions work? -sidebar_label: How transactions work? +sidebar_label: "How transactions work?" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/txn-monitor.md b/site2/website-next/docs/txn-monitor.md index d090e440f3c15..e596e1484766f 100644 --- a/site2/website-next/docs/txn-monitor.md +++ b/site2/website-next/docs/txn-monitor.md @@ -1,7 +1,7 @@ --- id: txn-monitor title: How to monitor transactions? -sidebar_label: How to monitor transactions? +sidebar_label: "How to monitor transactions?" --- import Tabs from '@theme/Tabs'; diff --git a/site2/website-next/docs/txn-use.md b/site2/website-next/docs/txn-use.md index 20cb511a75c1f..36616232f0fac 100644 --- a/site2/website-next/docs/txn-use.md +++ b/site2/website-next/docs/txn-use.md @@ -1,7 +1,7 @@ --- id: txn-use title: How to use transactions? -sidebar_label: How to use transactions? +sidebar_label: "How to use transactions?" --- import Tabs from '@theme/Tabs'; @@ -24,47 +24,57 @@ This section provides an example of how to use the transaction API to send and r 2. Enable transaction. - Change the configuration in the `broker.conf` file. + Change the configuration in the `broker.conf` file. - ``` - transactionCoordinatorEnabled=true - ``` + ``` + + transactionCoordinatorEnabled=true + + ``` - If you want to enable batch messages in transactions, follow the steps below. + If you want to enable batch messages in transactions, follow the steps below. - Set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. + Set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. - ``` - acknowledgmentAtBatchIndexLevelEnabled=true - ``` + ``` + + acknowledgmentAtBatchIndexLevelEnabled=true + + ``` 3. Initialize transaction coordinator metadata. - The transaction coordinator can leverage the advantages of partitioned topics (such as load balance). + The transaction coordinator can leverage the advantages of partitioned topics (such as load balance). - **Input** + **Input** - ``` - bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone - ``` + ``` + + bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone + + ``` - **Output** + **Output** - ``` - Transaction coordinator metadata setup success - ``` + ``` + + Transaction coordinator metadata setup success + + ``` 4. Initialize a Pulsar client. - ``` - PulsarClient client = PulsarClient.builder() + ``` + + PulsarClient client = PulsarClient.builder() - .serviceUrl(“pulsar://localhost:6650”) + .serviceUrl(“pulsar://localhost:6650”) - .enableTransaction(true) + .enableTransaction(true) - .build(); - ``` + .build(); + + ``` Now you can start using the transaction API to send and receive messages. Below is an example of a `consume-process-produce` application written in Java. @@ -83,6 +93,7 @@ Let’s walk through this example step by step. [1] Example of enabling batch messages ack in transactions in the consumer builder. ``` + Consumer sinkConsumer = pulsarClient .newConsumer() .topic(transferTopic) @@ -92,5 +103,6 @@ Consumer sinkConsumer = pulsarClient .subscriptionType(SubscriptionType.Shared) .enableBatchIndexAcknowledgment(true) // enable batch index acknowledgement .subscribe(); + ``` diff --git a/site2/website-next/docs/txn-what.md b/site2/website-next/docs/txn-what.md index 7c7e730020822..db21379465ce6 100644 --- a/site2/website-next/docs/txn-what.md +++ b/site2/website-next/docs/txn-what.md @@ -1,7 +1,7 @@ --- id: txn-what title: What are transactions? -sidebar_label: What are transactions? +sidebar_label: "What are transactions?" --- import Tabs from '@theme/Tabs'; @@ -31,9 +31,9 @@ Pulsar transactions have the following semantics: * A group of messages in a transaction can be received from, produced to, and acknowledged by multiple partitions. * Consumers are only allowed to read committed (acked) messages. In other words, the broker does not deliver transactional messages which are part of an open transaction or messages which are part of an aborted transaction. - + * Message writes across multiple partitions are atomic. - + * Message acks across multiple subscriptions are atomic. A message is acked successfully only once by a consumer under the subscription when acknowledging the message with the transaction ID. ## Transactions and stream processing @@ -56,8 +56,8 @@ Prior to Pulsar 2.8.0, there was no easy way to build stream processing applicat * [Pulsar Flink connector](https://flink.apache.org/2021/01/07/pulsar-flink-connector-270.html) - Prior to Pulsar 2.8.0, if you want to build stream applications using Pulsar and Flink, the Pulsar Flink connector only supported exactly-once source connector and at-least-once sink connector, which means the highest processing guarantee for end-to-end was at-least-once, there was possibility that the resulting messages from streaming applications produce duplicated messages to the resulting topics in Pulsar. + Prior to Pulsar 2.8.0, if you want to build stream applications using Pulsar and Flink, the Pulsar Flink connector only supported exactly-once source connector and at-least-once sink connector, which means the highest processing guarantee for end-to-end was at-least-once, there was possibility that the resulting messages from streaming applications produce duplicated messages to the resulting topics in Pulsar. - With the transaction introduced in Pulsar 2.8.0, the Pulsar Flink sink connector can support exactly-once semantics by implementing the designated `TwoPhaseCommitSinkFunction` and hooking up the Flink sink message lifecycle with Pulsar transaction API. + With the transaction introduced in Pulsar 2.8.0, the Pulsar Flink sink connector can support exactly-once semantics by implementing the designated `TwoPhaseCommitSinkFunction` and hooking up the Flink sink message lifecycle with Pulsar transaction API. * Support for Pulsar Functions and other connectors will be added in the future releases. diff --git a/site2/website-next/docs/txn-why.md b/site2/website-next/docs/txn-why.md index 9e950dba56a77..c76740705345c 100644 --- a/site2/website-next/docs/txn-why.md +++ b/site2/website-next/docs/txn-why.md @@ -1,7 +1,7 @@ --- id: txn-why title: Why transactions? -sidebar_label: Why transactions? +sidebar_label: "Why transactions?" --- import Tabs from '@theme/Tabs'; @@ -39,9 +39,9 @@ In Pulsar, the highest level of message delivery guarantee is using an [idempote - Similarly, for Pulsar Function, it only guarantees exactly once semantics for an idempotent function on a single event rather than processing multiple events or producing multiple results that can happen exactly. - For example, if a function accepts multiple events and produces one result (for example, window function), the function may fail between producing the result and acknowledging the incoming messages, or even between acknowledging individual events, which causes all (or some) incoming messages to be re-delivered and reprocessed, and a new result is generated. + For example, if a function accepts multiple events and produces one result (for example, window function), the function may fail between producing the result and acknowledging the incoming messages, or even between acknowledging individual events, which causes all (or some) incoming messages to be re-delivered and reprocessed, and a new result is generated. - However, many scenarios need atomic guarantees across multiple partitions and sessions. + However, many scenarios need atomic guarantees across multiple partitions and sessions. - Consumers need to rely on more mechanisms to acknowledge (ack) messages once. diff --git a/site2/website-next/docs/window-functions-context.md b/site2/website-next/docs/window-functions-context.md index 3c8334f754937..fe635c7562e7e 100644 --- a/site2/website-next/docs/window-functions-context.md +++ b/site2/website-next/docs/window-functions-context.md @@ -49,8 +49,8 @@ The `getInputTopics` method gets the **name list** of all input topics. This example demonstrates how to get the name list of all input topics in a Java window function. - ```java + public class GetInputTopicsWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -61,6 +61,7 @@ public class GetInputTopicsWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -80,6 +81,7 @@ public class GetOutputTopicWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -100,6 +102,7 @@ public class GetTenantWindowFunction implements WindowFunction { } } + ``` ### Get namespace @@ -108,8 +111,8 @@ The `getNamespace` method gets the namespace associated with the window function This example demonstrates how to get the namespace in a Java window function. - ```java + public class GetNamespaceWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -120,6 +123,7 @@ public class GetNamespaceWindowFunction implements WindowFunction } } + ``` ### Get function name @@ -128,8 +132,8 @@ The `getFunctionName` method gets the window function name. This example demonstrates how to get the function name in a Java window function. - ```java + public class GetNameOfWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -140,6 +144,7 @@ public class GetNameOfWindowFunction implements WindowFunction { } } + ``` ### Get function ID @@ -148,8 +153,8 @@ The `getFunctionId` method gets the window function ID. This example demonstrates how to get the function ID in a Java window function. - ```java + public class GetFunctionIDWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -160,6 +165,7 @@ public class GetFunctionIDWindowFunction implements WindowFunction } } + ``` ### Get function version @@ -168,8 +174,8 @@ The `getFunctionVersion` method gets the window function version. This example demonstrates how to get the function version of a Java window function. - ```java + public class GetVersionOfWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -180,6 +186,7 @@ public class GetVersionOfWindowFunction implements WindowFunction } } + ``` ### Get instance ID @@ -188,8 +195,8 @@ The `getInstanceId` method gets the instance ID of a window function. This example demonstrates how to get the instance ID in a Java window function. - ```java + public class GetInstanceIDWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -200,6 +207,7 @@ public class GetInstanceIDWindowFunction implements WindowFunction } } + ``` ### Get num instances @@ -208,8 +216,8 @@ The `getNumInstances` method gets the number of instances that invoke the window This example demonstrates how to get the number of instances in a Java window function. - ```java + public class GetNumInstancesWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -220,6 +228,7 @@ public class GetNumInstancesWindowFunction implements WindowFunction { @Override @@ -240,6 +249,7 @@ public class GetOutputSchemaTypeWindowFunction implements WindowFunction { } } + ``` If you need your function to produce logs, specify a log topic when creating or running the function. - ```bash + bin/pulsar-admin functions create \ --jar my-functions.jar \ --classname my.package.LoggingFunction \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` You can access all logs produced by `LoggingFunction` via the `persistent://public/default/logging-function-logs` topic. @@ -286,16 +298,18 @@ You can access all logs produced by `LoggingFunction` via the `persistent://publ Pulsar window functions can publish arbitrary metrics to the metrics interface which can be queried. -> **Note** -> -> If a Pulsar window function uses the language-native interface for Java, that function is not able to publish metrics and stats to Pulsar. +:::note + +If a Pulsar window function uses the language-native interface for Java, that function is not able to publish metrics and stats to Pulsar. + +::: You can record metrics using the context object on a per-key basis. This example sets a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message in a Java function. - ```java + import java.util.Collection; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.WindowContext; @@ -319,6 +333,7 @@ public class UserMetricWindowFunction implements WindowFunction { return null; } } + ``` ## User config @@ -327,12 +342,13 @@ When you run or update Pulsar Functions that are created using SDK, you can pass This example passes a user configured key/value to a function. - ```bash + bin/pulsar-admin functions create \ --name word-filter \ --user-config '{"forbidden-word":"rosebud"}' \ # Other function configs + ``` ### API @@ -341,24 +357,23 @@ You can use the following APIs to get user-defined information for window functi `getUserConfigMap` API gets a map of all user-defined key/value configurations for the window function. - - ```java + /** * Get a map of all user-defined key/value configs for the function. * * @return The full map of user-defined config values */ Map getUserConfigMap(); -``` +``` #### getUserConfigValue The `getUserConfigValue` API gets a user-defined key/value. - ```java + /** * Get any user-defined key/value. * @@ -366,14 +381,15 @@ The `getUserConfigValue` API gets a user-defined key/value. * @return The Optional value specified by the user for that key. */ Optional getUserConfigValue(String key); + ``` #### getUserConfigValueOrDefault The `getUserConfigValueOrDefault` API gets a user-defined key/value or a default value if none is present. - ```java + /** * Get any user-defined key/value or a default value if none is present. * @@ -382,23 +398,27 @@ The `getUserConfigValueOrDefault` API gets a user-defined key/value or a default * @return Either the user config value associated with a given key or a supplied default value */ Object getUserConfigValueOrDefault(String key, Object defaultValue); + ``` This example demonstrates how to access key/value pairs provided to Pulsar window functions. Java SDK context object enables you to access key/value pairs provided to Pulsar window functions via the command line (as JSON). ->**Tip** -> -> For all key/value pairs passed to Java window functions, both the `key` and the `value` are `String`. To set the value to be a different type, you need to deserialize it from the `String` type. +:::tip -This example passes a key/value pair in a Java window function. +For all key/value pairs passed to Java window functions, both the `key` and the `value` are `String`. To set the value to be a different type, you need to deserialize it from the `String` type. +::: + +This example passes a key/value pair in a Java window function. ```bash + bin/pulsar-admin functions create \ --user-config '{"word-of-the-day":"verdure"}' \ # Other function configs + ``` This example accesses values in a Java window function. @@ -406,8 +426,8 @@ This example accesses values in a Java window function. The `UserConfigFunction` function logs the string `"The word of the day is verdure"` every time the function is invoked (which means every time a message arrives). The user config of `word-of-the-day` is changed **only** when the function is updated with a new config value via multiple ways, such as the command line tool or REST API. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -426,17 +446,19 @@ public class UserConfigWindowFunction implements WindowFunction } } + ``` If no value is provided, you can access the entire user config map or set a default value. - ```java + // Get the whole config map Map allConfigs = context.getUserConfigMap(); // Get value or resort to default String wotd = context.getUserConfigValueOrDefault("word-of-the-day", "perspicacious"); + ``` ## Routing @@ -445,8 +467,8 @@ You can use the `context.publish()` interface to publish as many results as you This example shows that the `PublishFunction` class uses the built-in function in the context to publish messages to the `publishTopic` in a Java function. - ```java + public class PublishWindowFunction implements WindowFunction { @Override public Void process(Collection> input, WindowContext context) throws Exception { @@ -458,6 +480,7 @@ public class PublishWindowFunction implements WindowFunction { } } + ``` ## State storage @@ -484,14 +507,15 @@ The `incrCounter` API increases a built-in distributed counter referred by key. Applications use the `incrCounter` API to change the counter of a given `key` by the given `amount`. If the `key` does not exist, a new key is created. - ```java + /** * Increment the builtin distributed counter referred by key * @param key The name of the key * @param amount The amount to be incremented */ void incrCounter(String key, long amount); + ``` #### getCounter @@ -500,8 +524,8 @@ The `getCounter` API gets the counter value for the key. Applications uses the `getCounter` API to retrieve the counter of a given `key` changed by the `incrCounter` API. - ```java + /** * Retrieve the counter value for the key. * @@ -509,6 +533,7 @@ Applications uses the `getCounter` API to retrieve the counter of a given `key` * @return the amount of the counter value for this key */ long getCounter(String key); + ``` Except the `getCounter` API, Pulsar also exposes a general key/value API (`putState`) for functions to store general key/value state. @@ -517,8 +542,8 @@ Except the `getCounter` API, Pulsar also exposes a general key/value API (`putSt The `putState` API updates the state value for the key. - ```java + /** * Update the state value for the key. * @@ -526,6 +551,7 @@ The `putState` API updates the state value for the key. * @param value state value of the key */ void putState(String key, ByteBuffer value); + ``` This example demonstrates how applications store states in Pulsar window functions. @@ -536,8 +562,8 @@ The logic of the `WordCountWindowFunction` is simple and straightforward. 2. For each `word`, the function increments the corresponding `counter` by 1 via `incrCounter(key, amount)`. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -553,5 +579,6 @@ public class WordCountWindowFunction implements WindowFunction { } } + ``` diff --git a/site2/website-next/package.json b/site2/website-next/package.json index dfe0459bda87e..785bcdf8c9139 100644 --- a/site2/website-next/package.json +++ b/site2/website-next/package.json @@ -14,9 +14,9 @@ "write-heading-ids": "docusaurus write-heading-ids" }, "dependencies": { - "@docusaurus/core": "2.0.0-beta.6", - "@docusaurus/plugin-client-redirects": "^2.0.0-beta.6", - "@docusaurus/preset-classic": "2.0.0-beta.6", + "@docusaurus/core": "2.0.0-beta.7", + "@docusaurus/plugin-client-redirects": "^2.0.0-beta.7", + "@docusaurus/preset-classic": "2.0.0-beta.7", "@emotion/react": "^11.5.0", "@emotion/styled": "^11.3.0", "@mdx-js/react": "^1.6.22", @@ -30,9 +30,9 @@ "react": "^17.0.2", "react-dom": "^17.0.2", "react-markdown": "^7.0.1", - "react-svg": "^14.0.15", + "react-svg": "^14.0.16", "remark-linkify-regex": "^1.0.0", - "replace-in-file": "^6.2.0", + "replace-in-file": "^6.3.1", "url-loader": "^4.1.1" }, "browserslist": { diff --git a/site2/website-next/versioned_docs/version-2.7.2/schema-evolution-compatibility.md b/site2/website-next/versioned_docs/version-2.7.2/schema-evolution-compatibility.md index a2bb99922254f..6e08e164a05ae 100644 --- a/site2/website-next/versioned_docs/version-2.7.2/schema-evolution-compatibility.md +++ b/site2/website-next/versioned_docs/version-2.7.2/schema-evolution-compatibility.md @@ -37,9 +37,9 @@ For more information, see [Schema compatibility check strategy](#schema-compatib 1. When a producer/consumer/reader connects to a broker, the broker deploys the schema compatibility checker configured by `schemaRegistryCompatibilityCheckers` to enforce schema compatibility check. - The schema compatibility checker is one instance per schema type. - - Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. + The schema compatibility checker is one instance per schema type. + + Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. 2. The producer/consumer/reader sends its client `SchemaInfo` to the broker. @@ -77,25 +77,25 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t * Example 1 - In some situations, an application needs to store events of several different types in the same Pulsar topic. + In some situations, an application needs to store events of several different types in the same Pulsar topic. - In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. + In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. - For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. + For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. - Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. + Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. * Example 2 - Sometimes we also make incompatible changes. + Sometimes we also make incompatible changes. - For example, you are modifying a field type from `string` to `int`. + For example, you are modifying a field type from `string` to `int`. - In this case, you need to: + In this case, you need to: - * Upgrade all producers and consumers to the new schema versions at the same time. + * Upgrade all producers and consumers to the new schema versions at the same time. - * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. + * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. ### BACKWARD and BACKWARD_TRANSITIVE @@ -110,15 +110,15 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t * Example 1 - Remove a field. - - A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. + Remove a field. + + A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. * Example 2 - You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. + You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. - Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. + Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. ### FORWARD and FORWARD_TRANSITIVE @@ -168,10 +168,10 @@ When a producer tries to connect to a topic (suppose ignore the schema auto crea * Check if the schema carried by the producer exists in the schema registry or not. - * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. - - * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. - + * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. + + * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. + ### Consumer When a consumer tries to connect to a topic, a broker checks if a carried schema is compatible with a registered schema based on the configured schema compatibility check strategy. diff --git a/site2/website-next/versioned_docs/version-2.7.2/schema-get-started.md b/site2/website-next/versioned_docs/version-2.7.2/schema-get-started.md index c8193ab572c69..3cc345855ea4f 100644 --- a/site2/website-next/versioned_docs/version-2.7.2/schema-get-started.md +++ b/site2/website-next/versioned_docs/version-2.7.2/schema-get-started.md @@ -72,6 +72,10 @@ If you construct a producer without specifying a schema, then the producer can o **Example** ``` +<<<<<<< HEAD +======= + +>>>>>>> up/master Producer producer = client.newProducer() .topic(topic) .create(); @@ -80,6 +84,10 @@ byte[] message = … // serialize the `user` by yourself; producer.send(message); ``` +<<<<<<< HEAD +======= + +>>>>>>> up/master ### With schema If you construct a producer with specifying a schema, then you can send a class to a topic directly without worrying about how to serialize POJOs into bytes. diff --git a/site2/website-next/versioned_docs/version-2.7.2/schema-manage.md b/site2/website-next/versioned_docs/version-2.7.2/schema-manage.md index e95369f0da08a..b8377c0e6f4fb 100644 --- a/site2/website-next/versioned_docs/version-2.7.2/schema-manage.md +++ b/site2/website-next/versioned_docs/version-2.7.2/schema-manage.md @@ -35,25 +35,25 @@ For a producer, the `AutoUpdate` happens in the following cases: * If a **producer doesn’t carry a schema**: - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. - - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. + + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. * If a **producer carries a schema**: - A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. - - * If the schema is registered, a producer is connected to a broker. - - * If the schema is not registered: - - * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. - - * If `isAllowAutoUpdateSchema` sets to **true**: - - * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. - - * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. + A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. + + * If the schema is registered, a producer is connected to a broker. + + * If the schema is not registered: + + * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. + + * If `isAllowAutoUpdateSchema` sets to **true**: + + * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. + + * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. ![AutoUpdate Producer](/assets/schema-producer.png) @@ -65,18 +65,18 @@ For a consumer, the `AutoUpdate` happens in the following cases: * If a **consumer connects to a topic with a schema**. - * If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - - * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. - + * If a topic does not have all of them (a schema/data/a local consumer and a local producer): + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + + * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + ![AutoUpdate Consumer](/assets/schema-consumer.png) @@ -293,9 +293,10 @@ PostSchemaPayload payload = new PostSchemaPayload(); payload.setType("INT8"); payload.setSchema(""); -admin.createSchema("my-tenant/my-ns/my-topic", payload); +admin.createSchema("my-tenant/my-ns/my-topic", payload); ``` + @@ -395,7 +396,7 @@ Here is an example of `SchemaInfo`: PulsarAdmin admin = …; -SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); +SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); ``` @@ -430,7 +431,7 @@ Use the `get` subcommand. ```bash -$ pulsar-admin schemas get --version= +$ pulsar-admin schemas get --version= ``` @@ -595,7 +596,7 @@ Here is an example of deleting a schema. PulsarAdmin admin = …; -admin.deleteSchema("my-tenant/my-ns/my-topic"); +admin.deleteSchema("my-tenant/my-ns/my-topic"); ``` diff --git a/site2/website-next/versioned_docs/version-2.7.2/schema-understand.md b/site2/website-next/versioned_docs/version-2.7.2/schema-understand.md index 87822c55810b4..25fd65e9453b2 100644 --- a/site2/website-next/versioned_docs/version-2.7.2/schema-understand.md +++ b/site2/website-next/versioned_docs/version-2.7.2/schema-understand.md @@ -101,21 +101,21 @@ This example demonstrates how to use a string schema. 1. Create a producer with a string schema and send messages. - ```java - - Producer producer = client.newProducer(Schema.STRING).create(); - producer.newMessage().value("Hello Pulsar!").send(); - - ``` + ```java + + Producer producer = client.newProducer(Schema.STRING).create(); + producer.newMessage().value("Hello Pulsar!").send(); + + ``` 2. Create a consumer with a string schema and receive messages. - ```java - - Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); - consumer.receive(); - - ``` + ```java + + Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); + consumer.receive(); + + ``` ### Complex type @@ -154,72 +154,72 @@ This example shows how to construct a key/value schema and then use it to produc 1. Construct a key/value schema with `INLINE` encoding type. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.INLINE - ); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.INLINE + ); + + ``` 2. Optionally, construct a key/value schema with `SEPARATED` encoding type. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + ``` 3. Produce messages using a key/value schema. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Producer> producer = client.newProducer(kvSchema) - .topic(TOPIC) - .create(); - - final int key = 100; - final String value = "value-100"; - - // send the key/value message - producer.newMessage() - .value(new KeyValue(key, value)) - .send(); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Producer> producer = client.newProducer(kvSchema) + .topic(TOPIC) + .create(); + + final int key = 100; + final String value = "value-100"; + + // send the key/value message + producer.newMessage() + .value(new KeyValue(key, value)) + .send(); + + ``` 4. Consume messages using a key/value schema. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Consumer> consumer = client.newConsumer(kvSchema) - ... - .topic(TOPIC) - .subscriptionName(SubscriptionName).subscribe(); - - // receive key/value pair - Message> msg = consumer.receive(); - KeyValue kv = msg.getValue(); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Consumer> consumer = client.newConsumer(kvSchema) + ... + .topic(TOPIC) + .subscriptionName(SubscriptionName).subscribe(); + + // receive key/value pair + Message> msg = consumer.receive(); + KeyValue kv = msg.getValue(); + + ``` #### struct @@ -247,32 +247,32 @@ Pulsar gets the schema definition from the predefined `struct` using an Avro lib 1. Create the _User_ class to define the messages sent to Pulsar topics. - ```java - - public class User { - String name; - int age; - } - - ``` + ```java + + public class User { + String name; + int age; + } + + ``` 2. Create a producer with a `struct` schema and send messages. - ```java - - Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); - producer.newMessage().value(User.builder().userName("pulsar-user").userId(1L).build()).send(); - - ``` + ```java + + Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); + producer.newMessage().value(User.builder().userName("pulsar-user").userId(1L).build()).send(); + + ``` 3. Create a consumer with a `struct` schema and receive messages - ```java - - Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); - User user = consumer.receive(); - - ``` + ```java + + Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); + User user = consumer.receive(); + + ``` ##### generic @@ -284,25 +284,25 @@ You can define the `struct` schema using the `GenericSchemaBuilder`, generate a 1. Use `RecordSchemaBuilder` to build a schema. - ```java - - RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); - recordSchemaBuilder.field("intField").type(SchemaType.INT32); - SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - - Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + ```java + + RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); + recordSchemaBuilder.field("intField").type(SchemaType.INT32); + SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - ``` + Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + + ``` 2. Use `RecordBuilder` to build the struct records. - ```java - - producer.newMessage().value(schema.newRecordBuilder() - .set("intField", 32) - .build()).send(); - - ``` + ```java + + producer.newMessage().value(schema.newRecordBuilder() + .set("intField", 32) + .build()).send(); + + ``` ### Auto Schema @@ -420,9 +420,9 @@ This diagram illustrates how does schema work on the Producer side. 1. The application uses a schema instance to construct a producer instance. - The schema instance defines the schema for the data being produced using the producer instance. + The schema instance defines the schema for the data being produced using the producer instance. - Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. + Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. 2. The producer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. @@ -446,7 +446,7 @@ For how to set `isAllowAutoUpdateSchema` via Pulsar admin API, see [Manage AutoU * If the schema is compatible, the broker stores it and returns the schema version to the producer. - All the messages produced by this producer are tagged with the schema version. + All the messages produced by this producer are tagged with the schema version. * If the schema is incompatible, the broker rejects it. @@ -458,24 +458,24 @@ This diagram illustrates how does Schema work on the consumer side. 1. The application uses a schema instance to construct a consumer instance. - The schema instance defines the schema that the consumer uses for decoding messages received from a broker. + The schema instance defines the schema that the consumer uses for decoding messages received from a broker. 2. The consumer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. 3. The broker determines whether the topic has one of them (a schema/data/a local consumer and a local producer). 4. If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + 5. If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. 6. The consumer receives messages from the broker. - If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. + If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. diff --git a/site2/website-next/versioned_docs/version-2.7.3/adaptors-kafka.md b/site2/website-next/versioned_docs/version-2.7.3/adaptors-kafka.md index 7fe8c3ca2a370..24bf975aa558d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/adaptors-kafka.md +++ b/site2/website-next/versioned_docs/version-2.7.3/adaptors-kafka.md @@ -1,7 +1,7 @@ --- id: adaptors-kafka title: Pulsar adaptor for Apache Kafka -sidebar_label: Kafka client wrapper +sidebar_label: "Kafka client wrapper" original_id: adaptors-kafka --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/adaptors-spark.md b/site2/website-next/versioned_docs/version-2.7.3/adaptors-spark.md index 0ece08291dc91..940d401ed0086 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/adaptors-spark.md +++ b/site2/website-next/versioned_docs/version-2.7.3/adaptors-spark.md @@ -1,7 +1,7 @@ --- id: adaptors-spark title: Pulsar adaptor for Apache Spark -sidebar_label: Apache Spark +sidebar_label: "Apache Spark" original_id: adaptors-spark --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/adaptors-storm.md b/site2/website-next/versioned_docs/version-2.7.3/adaptors-storm.md index f5962626c72f5..e4b07db19090c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/adaptors-storm.md +++ b/site2/website-next/versioned_docs/version-2.7.3/adaptors-storm.md @@ -1,7 +1,7 @@ --- id: adaptors-storm title: Pulsar adaptor for Apache Storm -sidebar_label: Apache Storm +sidebar_label: "Apache Storm" original_id: adaptors-storm --- @@ -97,3 +97,4 @@ boltConf.setTupleToMessageMapper(tupleToMessageMapper); PulsarBolt bolt = new PulsarBolt(boltConf); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-brokers.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-brokers.md index 257cea6db2e0a..28adfdf784689 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-brokers.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-brokers.md @@ -1,7 +1,7 @@ --- id: admin-api-brokers title: Managing Brokers -sidebar_label: Brokers +sidebar_label: "Brokers" original_id: admin-api-brokers --- @@ -55,6 +55,7 @@ $ pulsar-admin brokers list use ``` ``` + broker1.use.org.com:8080 ``` @@ -117,6 +118,7 @@ $ pulsar-admin brokers namespaces use \ } ``` + @@ -130,6 +132,7 @@ $ pulsar-admin brokers namespaces use \ admin.brokers().getOwnedNamespaces(cluster,brokerUrl); ``` + @@ -184,6 +187,7 @@ $ pulsar-admin brokers update-dynamic-config --config brokerShutdownTimeoutMs -- admin.brokers().updateDynamicConfiguration(configName, configValue); ``` + @@ -229,6 +233,7 @@ brokerShutdownTimeoutMs admin.brokers().getDynamicConfigurationNames(); ``` + @@ -275,6 +280,7 @@ brokerShutdownTimeoutMs:100 admin.brokers().getAllDynamicConfigurations(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-clusters.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-clusters.md index 67af03b8f6e0a..eb3d5b0b56cd2 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-clusters.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-clusters.md @@ -1,7 +1,7 @@ --- id: admin-api-clusters title: Managing Clusters -sidebar_label: Clusters +sidebar_label: "Clusters" original_id: admin-api-clusters --- @@ -73,6 +73,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().createCluster(clusterName, clusterData); ``` + @@ -163,6 +164,7 @@ $ pulsar-admin clusters get cluster-1 admin.clusters().getCluster(clusterName); ``` + @@ -218,6 +220,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().updateCluster(clusterName, clusterData); ``` + @@ -247,6 +250,7 @@ Clusters can be deleted from a Pulsar [instance](reference-terminology.md#instan Use the [`delete`](reference-pulsar-admin.md#clusters-delete) subcommand and specify the name of the cluster. ``` + $ pulsar-admin clusters delete cluster-1 ``` @@ -264,6 +268,7 @@ $ pulsar-admin clusters delete cluster-1 admin.clusters().deleteCluster(clusterName); ``` + @@ -313,6 +318,7 @@ cluster-2 admin.clusters().getClusters(); ``` + @@ -342,6 +348,7 @@ Peer clusters can be configured for a given cluster in a Pulsar [instance](refer Use the [`update-peer-clusters`](reference-pulsar-admin.md#clusters-update-peer-clusters) subcommand and specify the list of peer-cluster names. ``` + $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 ``` @@ -359,6 +366,7 @@ $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 admin.clusters().updatePeerClusterNames(clusterName, peerClusterList); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-functions.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-functions.md index fd992cb09cbc3..759deb23dcbab 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-functions.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-functions.md @@ -1,7 +1,7 @@ --- id: admin-api-functions title: Manage Functions -sidebar_label: Functions +sidebar_label: "Functions" original_id: admin-api-functions --- @@ -91,6 +91,7 @@ functionConfig.setOutput(sinkTopic); admin.functions().createFunction(functionConfig, fileName); ``` + @@ -154,6 +155,7 @@ updateOptions.setUpdateAuthData(updateAuthData); admin.functions().updateFunction(functionConfig, userCodeFile, updateOptions); ``` + @@ -205,6 +207,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -257,6 +260,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName); ``` + @@ -310,6 +314,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -362,6 +367,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName); ``` + @@ -415,6 +421,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -467,6 +474,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName); ``` + @@ -518,6 +526,7 @@ $ pulsar-admin functions list \ admin.functions().getFunctions(tenant, namespace); ``` + @@ -553,7 +562,7 @@ Use the [`delete`](reference-pulsar-admin.md#functions-delete) subcommand. $ pulsar-admin functions delete \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -570,6 +579,7 @@ $ pulsar-admin functions delete \ admin.functions().deleteFunction(tenant, namespace, functionName); ``` + @@ -605,7 +615,7 @@ Use the [`get`](reference-pulsar-admin.md#functions-get) subcommand. $ pulsar-admin functions get \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -622,6 +632,7 @@ $ pulsar-admin functions get \ admin.functions().getFunction(tenant, namespace, functionName); ``` + @@ -674,6 +685,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -709,7 +721,7 @@ Use the [`status`](reference-pulsar-admin.md#functions-status) subcommand. $ pulsar-admin functions status \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -726,6 +738,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName); ``` + @@ -778,6 +791,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -813,7 +827,7 @@ Use the [`stats`](reference-pulsar-admin.md#functions-stats) subcommand. $ pulsar-admin functions stats \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -830,6 +844,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName); ``` + @@ -871,6 +886,7 @@ $ pulsar-admin functions trigger \ # or --trigger-file (the path of trigger file) ``` + @@ -884,6 +900,7 @@ $ pulsar-admin functions trigger \ admin.functions().triggerFunction(tenant, namespace, functionName, topic, triggerValue, triggerFile); ``` + @@ -920,7 +937,7 @@ $ pulsar-admin functions putstate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" + --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" ``` @@ -939,6 +956,7 @@ FunctionState stateRepr = ObjectMapperFactory.getThreadLocal().readValue(state, admin.functions().putFunctionState(tenant, namespace, functionName, stateRepr); ``` + @@ -975,7 +993,7 @@ $ pulsar-admin functions querystate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --key (the key of state) + --key (the key of state) ``` @@ -992,6 +1010,7 @@ $ pulsar-admin functions querystate \ admin.functions().getFunctionState(tenant, namespace, functionName, key); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-namespaces.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-namespaces.md index 606d1011b8055..cd89034a3a7bb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-namespaces.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-namespaces.md @@ -1,7 +1,7 @@ --- id: admin-api-namespaces title: Managing Namespaces -sidebar_label: Namespaces +sidebar_label: "Namespaces" original_id: admin-api-namespaces --- @@ -62,6 +62,7 @@ $ pulsar-admin namespaces create test-tenant/test-namespace admin.namespaces().createNamespace(namespace); ``` + @@ -130,6 +131,7 @@ $ pulsar-admin namespaces policies test-tenant/test-namespace admin.namespaces().getPolicies(namespace); ``` + @@ -179,6 +181,7 @@ test-tenant/ns2 admin.namespaces().getNamespaces(tenant); ``` + @@ -226,6 +229,7 @@ $ pulsar-admin namespaces delete test-tenant/ns1 admin.namespaces().deleteNamespace(namespace); ``` + @@ -255,6 +259,7 @@ It sets replication clusters for a namespace, so Pulsar can internally replicate ``` + $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ --clusters cl1 @@ -264,6 +269,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replication|operation/setNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -276,6 +282,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ admin.namespaces().setNamespaceReplicationClusters(namespace, clusters); ``` + @@ -303,11 +310,13 @@ It gives a list of replication clusters for a given namespace. ``` + $ pulsar-admin namespaces get-clusters test-tenant/cl1/ns1 ``` ``` + cl2 ``` @@ -316,6 +325,7 @@ cl2 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replication|operation/getNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -328,6 +338,7 @@ cl2 admin.namespaces().getNamespaceReplicationClusters(namespace) ``` + @@ -365,11 +376,13 @@ Backlog quota helps the broker to restrict bandwidth/storage of a namespace once ``` + $ pulsar-admin namespaces set-backlog-quota --limit 10 --policy producer_request_hold test-tenant/ns1 ``` ``` + N/A ``` @@ -378,6 +391,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/setBacklogQuota?version=@pulsar:version_number@} ``` @@ -390,6 +404,7 @@ N/A admin.namespaces().setBacklogQuota(namespace, new BacklogQuota(limit, policy)) ``` + @@ -417,6 +432,7 @@ It shows a configured backlog quota for a given namespace. ``` + $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` @@ -436,6 +452,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/backlogQuotaMap|operation/getBacklogQuotaMap?version=@pulsar:version_number@} ``` @@ -448,6 +465,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 admin.namespaces().getBacklogQuotaMap(namespace); ``` + @@ -475,11 +493,13 @@ It removes backlog quota policies for a given namespace ``` + $ pulsar-admin namespaces remove-backlog-quota test-tenant/ns1 ``` ``` + N/A ``` @@ -488,6 +508,7 @@ N/A ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/removeBacklogQuota?version=@pulsar:version_number@} ``` @@ -500,6 +521,7 @@ N/A admin.namespaces().removeBacklogQuota(namespace, backlogQuotaType) ``` + @@ -537,11 +559,13 @@ Persistence policies allow to configure persistency-level for all topic messages ``` + $ pulsar-admin namespaces set-persistence --bookkeeper-ack-quorum 2 --bookkeeper-ensemble 3 --bookkeeper-write-quorum 2 --ml-mark-delete-max-rate 0 test-tenant/ns1 ``` ``` + N/A ``` @@ -550,6 +574,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/setPersistence?version=@pulsar:version_number@} ``` @@ -562,6 +587,7 @@ N/A admin.namespaces().setPersistence(namespace,new PersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum,bookkeeperAckQuorum,managedLedgerMaxMarkDeleteRate)) ``` + @@ -589,6 +615,7 @@ It shows the configured persistence policies of a given namespace. ``` + $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` @@ -608,6 +635,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/getPersistence?version=@pulsar:version_number@} ``` @@ -620,6 +648,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 admin.namespaces().getPersistence(namespace) ``` + @@ -649,11 +678,13 @@ The namespace bundle is a virtual group of topics which belong to the same names ``` + $ pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` ``` + N/A ``` @@ -662,6 +693,7 @@ N/A ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/unload|operation/unloadNamespaceBundle?version=@pulsar:version_number@} ``` @@ -674,6 +706,7 @@ N/A admin.namespaces().unloadNamespaceBundle(namespace, bundle) ``` + @@ -702,11 +735,13 @@ If a single bundle is creating an excessive load on a broker, an admin splits th ``` + $ pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` ``` + N/A ``` @@ -715,6 +750,7 @@ N/A ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/split|operation/splitNamespaceBundle?version=@pulsar:version_number@} ``` @@ -727,6 +763,7 @@ N/A admin.namespaces().splitNamespaceBundle(namespace, bundle) ``` + @@ -756,11 +793,13 @@ It configures message’s time to live (in seconds) duration. ``` + $ pulsar-admin namespaces set-message-ttl --messageTTL 100 test-tenant/ns1 ``` ``` + N/A ``` @@ -769,6 +808,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/setNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -781,6 +821,7 @@ N/A admin.namespaces().setNamespaceMessageTTL(namespace, messageTTL) ``` + @@ -808,11 +849,13 @@ It gives a message ttl of configured namespace. ``` + $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` ``` + 100 ``` @@ -821,6 +864,7 @@ $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/getNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -833,6 +877,7 @@ $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 admin.namespaces().getNamespaceMessageTTL(namespace) ``` + @@ -860,11 +905,13 @@ Remove a message TTL of the configured namespace. ``` + $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` ``` + 100 ``` @@ -873,6 +920,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/removeNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -885,6 +933,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + @@ -915,11 +964,13 @@ It clears all message backlog for all the topics that belong to a specific names ``` + $ pulsar-admin namespaces clear-backlog --sub my-subscription test-tenant/ns1 ``` ``` + N/A ``` @@ -928,6 +979,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/clearBacklog|operation/clearNamespaceBacklogForSubscription?version=@pulsar:version_number@} ``` @@ -940,6 +992,7 @@ N/A admin.namespaces().clearNamespaceBacklogForSubscription(namespace, subscription) ``` + @@ -967,11 +1020,13 @@ It clears all message backlog for all the topics that belong to a specific Names ``` + $ pulsar-admin namespaces clear-backlog --bundle 0x00000000_0xffffffff --sub my-subscription test-tenant/ns1 ``` ``` + N/A ``` @@ -980,6 +1035,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/{bundle}/clearBacklog|operation?version=@pulsar:version_number@/clearNamespaceBundleBacklogForSubscription} ``` @@ -992,6 +1048,7 @@ N/A admin.namespaces().clearNamespaceBundleBacklogForSubscription(namespace, bundle, subscription) ``` + @@ -1021,11 +1078,13 @@ Each namespace contains multiple topics and the retention size (storage size) of ``` + $ pulsar-admin set-retention --size 10 --time 100 test-tenant/ns1 ``` ``` + N/A ``` @@ -1034,6 +1093,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/retention|operation/setRetention?version=@pulsar:version_number@} ``` @@ -1046,6 +1106,7 @@ N/A admin.namespaces().setRetention(namespace, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)) ``` + @@ -1073,6 +1134,7 @@ It shows retention information of a given namespace. ``` + $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` @@ -1090,6 +1152,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/retention|operation/getRetention?version=@pulsar:version_number@} ``` @@ -1102,6 +1165,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 admin.namespaces().getRetention(namespace) ``` + @@ -1144,6 +1208,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1155,6 +1220,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1167,6 +1233,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ admin.namespaces().setDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1194,6 +1261,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` @@ -1212,6 +1280,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1224,6 +1293,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 admin.namespaces().getDispatchRate(namespace) ``` + @@ -1256,6 +1326,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1267,6 +1338,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1279,6 +1351,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ admin.namespaces().setSubscriptionDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1306,6 +1379,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` @@ -1324,6 +1398,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1336,6 +1411,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 admin.namespaces().getSubscriptionDispatchRate(namespace) ``` + @@ -1368,6 +1444,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1379,6 +1456,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1391,6 +1469,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ admin.namespaces().setReplicatorDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1418,6 +1497,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` @@ -1436,6 +1516,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1448,6 +1529,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 admin.namespaces().getReplicatorDispatchRate(namespace) ``` + @@ -1477,6 +1559,7 @@ It shows configured `deduplicationSnapshotInterval` for a namespace (Each topic ``` + $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1485,6 +1568,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1497,6 +1581,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().getDeduplicationSnapshotInterval(namespace) ``` + @@ -1525,7 +1610,8 @@ Set configured `deduplicationSnapshotInterval` for a namespace. Each topic under ``` -$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 + +$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 ``` @@ -1533,6 +1619,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1553,6 +1640,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- admin.namespaces().setDeduplicationSnapshotInterval(namespace, 1000) ``` + @@ -1580,6 +1668,7 @@ Remove configured `deduplicationSnapshotInterval` of a namespace (Each topic und ``` + $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1588,6 +1677,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1600,6 +1690,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().removeDeduplicationSnapshotInterval(namespace) ``` + @@ -1644,6 +1735,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/unload|operation/unloadNamespace?version=@pulsar:version_number@} ``` @@ -1656,6 +1748,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns admin.namespaces().unload(namespace) ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-partitioned-topics.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-partitioned-topics.md index e654e6dc8d229..51ab7b6e2a4fd 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-partitioned-topics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-partitioned-topics title: Managing non-partitioned topics -sidebar_label: Non-partitioned topics +sidebar_label: "Non-partitioned topics" original_id: admin-api-non-partitioned-topics --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-persistent-topics.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-persistent-topics.md index 1599b2971162d..06f938212dbbb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-persistent-topics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-non-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-persistent-topics title: Managing non-persistent topics -sidebar_label: Non-Persistent topics +sidebar_label: "Non-Persistent topics" original_id: admin-api-non-persistent-topics --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-overview.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-overview.md index 4d5db3eb762fc..3f04c4fa5b27d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-overview.md @@ -1,7 +1,7 @@ --- id: admin-api-overview title: The Pulsar admin interface -sidebar_label: Overview +sidebar_label: "Overview" original_id: admin-api-overview --- @@ -19,9 +19,9 @@ You can currently interact with the admin interface via: - The `pulsar-admin` CLI tool, which is available in the `bin` folder of your [Pulsar installation](getting-started-standalone): ```shell - + $ bin/pulsar-admin - + ``` For the complete commands and descriptions of `pulsar-admin`, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.7.0-SNAPSHOT/). @@ -116,6 +116,7 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + @@ -137,12 +138,11 @@ Kubernetes requires a name that can be used as a DNS subdomain name as defined i - Dots (.) - Replace beginning and ending non-alphanumeric characters with 0 - + :::tip - - If you get an error in translating Pulsar object names into Kubernetes resource labels (for example, you may have a naming collision if your Pulsar object name is too long) or want to customize the translating rules, see [customize Kubernetes runtime](https://pulsar.apache.org/docs/en/next/functions-runtime/#customize-kubernetes-runtime). - - For how to configure Kubernetes runtime, see [here](https://pulsar.apache.org/docs/en/next/functions-runtime/#configure-kubernetes-runtime). ::: + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-partitioned-topics.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-partitioned-topics.md index e0166cd99016e..f42ef93b6a9c8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-partitioned-topics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-partitioned-topics title: Managing partitioned topics -sidebar_label: Partitioned topics +sidebar_label: "Partitioned topics" original_id: admin-api-partitioned-topics --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-permissions.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-permissions.md index 4641732366b5e..e4106f08df568 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-permissions.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-permissions.md @@ -1,7 +1,7 @@ --- id: admin-api-permissions title: Managing permissions -sidebar_label: Permissions +sidebar_label: "Permissions" original_id: admin-api-permissions --- @@ -96,6 +96,7 @@ Roles `my.1.role`, `my.2.role`, `my.foo.role`, `my.bar.role`, etc. **cannot** pr admin.namespaces().grantPermissionOnNamespace(namespace, role, getAuthActions(actions)); ``` + @@ -132,7 +133,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 "produce", "consume" ] -} +} ``` @@ -149,6 +150,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 admin.namespaces().getPermissions(namespace); ``` + @@ -197,6 +199,7 @@ $ pulsar-admin namespaces revoke-permission test-tenant/ns1 \ admin.namespaces().revokePermissionsOnNamespace(namespace, role); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-persistent-topics.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-persistent-topics.md index 58502e3f9d5e3..42a43527fb780 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-persistent-topics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-persistent-topics title: Managing persistent topics -sidebar_label: Persistent topics +sidebar_label: "Persistent topics" original_id: admin-api-persistent-topics --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-schemas.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-schemas.md index 9bf86b9aff322..cd56ead2f32ba 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-schemas.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-schemas.md @@ -1,7 +1,7 @@ --- id: admin-api-schemas title: Managing Schemas -sidebar_label: Schemas +sidebar_label: "Schemas" original_id: admin-api-schemas --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-tenants.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-tenants.md index 23c5070d46b74..82880d1305c2a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-tenants.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-tenants.md @@ -1,7 +1,7 @@ --- id: admin-api-tenants title: Managing Tenants -sidebar_label: Tenants +sidebar_label: "Tenants" original_id: admin-api-tenants --- @@ -61,6 +61,7 @@ my-tenant-2 admin.tenants().getTenants(); ``` + @@ -106,6 +107,7 @@ $ pulsar-admin tenants create my-tenant \ -r role1 ``` + @@ -163,6 +165,7 @@ $ pulsar-admin tenants get my-tenant } ``` + @@ -224,6 +227,7 @@ $ pulsar-admin tenants delete my-tenant admin.Tenants().deleteTenant(tenantName); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/admin-api-topics.md b/site2/website-next/versioned_docs/version-2.7.3/admin-api-topics.md index 5a0634cbd51b5..f264bc8c8c647 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/admin-api-topics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/admin-api-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-topics title: Manage topics -sidebar_label: Topics +sidebar_label: "Topics" original_id: admin-api-topics --- @@ -24,6 +24,7 @@ Non-persistent topics are used in applications that only consume real-time publi non-persistent://tenant/namespace/topic ``` + ## Manage topic resources Whether it is persistent or non-persistent topic, you can obtain the topic resources through `pulsar-admin` tool, REST API and Java. @@ -77,6 +78,7 @@ String namespace = "my-tenant/my-namespace"; admin.topics().getList(namespace); ``` + @@ -472,6 +474,7 @@ The following is an example of a topic status. } ``` + To get the status of a topic, you can use the following ways. + When you create non-partitioned topics with the [`create`](reference-pulsar-admin.md#create-3) command, you need to specify the topic name as an argument. ```shell @@ -1218,6 +1223,7 @@ $ bin/pulsar-admin topics create \ persistent://my-tenant/my-namespace/my-topic ``` + :::note When you create a non-partitioned topic with the suffix '-partition-' followed by numeric value like 'xyz-topic-partition-x' for the topic name, if a partitioned topic with same suffix 'xyz-topic-partition-y' exists, then the numeric value(x) for the non-partitioned topic must be larger than the number of partitions(y) of the partitioned topic. Otherwise, you cannot create such a non-partitioned topic. @@ -1373,6 +1379,7 @@ You can check the current statistics of a given topic. The following is an examp } ``` + You can check the current statistics of a given topic and its connected producers and consumers in the following ways. @@ -1446,6 +1454,7 @@ You can create partitioned topics in the following ways. } ]}> + When you create partitioned topics with the [`create-partitioned-topic`](reference-pulsar-admin.md#create-partitioned-topic) command, you need to specify the topic name as an argument and the number of partitions using the `-p` or `--partitions` flag. @@ -1504,6 +1513,7 @@ When topic auto-creation is disabled, and you have a partitioned topic without a } ]}> + You can create missed partitions with the [`create-missed-partitions`](reference-pulsar-admin.md#create-missed-partitions) command and specify the topic name as an argument. ```shell @@ -1557,6 +1567,7 @@ Field | Description } ]}> + You can check the number of partitions in a partitioned topic with the [`get-partitioned-topic-metadata`](reference-pulsar-admin.md#get-partitioned-topic-metadata) subcommand. ```shell @@ -1611,6 +1622,7 @@ Producers and consumers can find the newly created partitions automatically. } ]}> + You can update partitioned topics with the [`update-partitioned-topic`](reference-pulsar-admin.md#update-partitioned-topic) command. ```shell @@ -2014,4 +2026,5 @@ If a message has a key, it supersedes the round robin routing policy. The follow return signSafeMod(PARTITION_INDEX_UPDATER.getAndIncrement(this), topicMetadata.numPartitions()); } -``` +``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-dashboard.md b/site2/website-next/versioned_docs/version-2.7.3/administration-dashboard.md index a4f332d5c7818..64f903045add5 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-dashboard.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-dashboard.md @@ -1,17 +1,16 @@ --- id: administration-dashboard title: Pulsar dashboard -sidebar_label: Dashboard +sidebar_label: "Dashboard" original_id: administration-dashboard --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; - :::note -Pulsar dashboard is deprecated. If you want to manage and monitor the stats of your topics, use [Pulsar Manager](administration-pulsar-manager.md). +Pulsar dashboard is deprecated. If you want to manage and monitor the stats of your topics, use [Pulsar Manager](administration-pulsar-manager). ::: @@ -26,28 +25,36 @@ You can use the [Django](https://www.djangoproject.com) web app to render the co The easiest way to use the dashboard is to run it inside a [Docker](https://www.docker.com/products/docker) container. ```shell + $ SERVICE_URL=http://broker.example.com:8080/ $ docker run -p 80:80 \ -e SERVICE_URL=$SERVICE_URL \ apachepulsar/pulsar-dashboard:@pulsar:version@ + ``` You can find the {@inject: github:Dockerfile:/dashboard/Dockerfile} in the `dashboard` directory and build an image from scratch as well: ```shell + $ docker build -t apachepulsar/pulsar-dashboard dashboard + ``` If token authentication is enabled: > Provided token should have super-user access. + ```shell + $ SERVICE_URL=http://broker.example.com:8080/ $ JWT_TOKEN=eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiIxMjM0NTY3ODkwIiwibmFtZSI6IkpvaG4gRG9lIiwiaWF0IjoxNTE2MjM5MDIyfQ.SflKxwRJSMeKKF2QT4fwpMeJf36POk6yJV_adQssw5c $ docker run -p 80:80 \ -e SERVICE_URL=$SERVICE_URL \ -e JWT_TOKEN=$JWT_TOKEN \ apachepulsar/pulsar-dashboard + ``` + You need to specify only one service URL for a Pulsar cluster. Internally, the collector figures out all the existing clusters and the brokers from where it needs to pull the metrics. If you connect the dashboard to Pulsar running in standalone mode, the URL is `http://:8080` by default. `` is the ip address or hostname of the machine running Pulsar standalone. The ip address or hostname should be accessible from the docker instance running dashboard. @@ -62,7 +69,9 @@ Similarly, given the Pulsar standalone advertises itself with localhost by defau explicitly set the advertise address to the host IP. For example: ```shell + $ bin/pulsar standalone --advertised-address 1.2.3.4 + ``` ### Known issues diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-geo.md b/site2/website-next/versioned_docs/version-2.7.3/administration-geo.md index 06078b13631a3..37b52e27d1532 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-geo.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-geo.md @@ -1,7 +1,7 @@ --- id: administration-geo title: Pulsar geo-replication -sidebar_label: Geo-replication +sidebar_label: "Geo-replication" original_id: administration-geo --- @@ -65,16 +65,18 @@ Suppose that you have 3 replication clusters: `us-west`, `us-cent`, and `us-east Run the following command on `us-west`. ```shell + $ bin/pulsar-admin clusters create \ --broker-url pulsar://: \ --url http://: \ us-east + ``` :::tip - -If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). +- If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). +- Different clusters may have different authentications. You can use the authentication flag `--auth-plugin` and `--auth-parameters` together to set cluster authentication, which overrides `brokerClientAuthenticationPlugin` and `brokerClientAuthenticationParameters` if `authenticationEnabled` sets to `true` in `broker.conf` and `standalone.conf`. For more information, see [authentication and authorization](concepts-authentication). ::: @@ -83,10 +85,12 @@ If you want to use a secure connection for a cluster, you can use the flags `--b Run the following command on `us-west`. ```shell + $ bin/pulsar-admin clusters create \ --broker-url pulsar://: \ --url http://: \ us-cent + ``` 3. Run similar commands on `us-east` and `us-cent` to create connections among clusters. @@ -98,9 +102,11 @@ To replicate to a cluster, the tenant needs permission to use that cluster. You Specify all the intended clusters when you create a tenant: ```shell + $ bin/pulsar-admin tenants create my-tenant \ --admin-roles my-admin-role \ --allowed-clusters us-west,us-east,us-cent + ``` To update permissions of an existing tenant, use `update` instead of `create`. @@ -110,14 +116,18 @@ To update permissions of an existing tenant, use `update` instead of `create`. You can create a namespace with the following command sample. ```shell + $ bin/pulsar-admin namespaces create my-tenant/my-namespace + ``` Initially, the namespace is not assigned to any cluster. You can assign the namespace to clusters using the `set-clusters` subcommand: ```shell + $ bin/pulsar-admin namespaces set-clusters my-tenant/my-namespace \ --clusters us-west,us-east,us-cent + ``` You can change the replication clusters for a namespace at any time, without disruption to ongoing traffic. Replication channels are immediately set up or stopped in all clusters as soon as the configuration changes. @@ -130,9 +140,10 @@ Once you create a geo-replication namespace, any topics that producers or consum By default, messages are replicated to all clusters configured for the namespace. You can restrict replication selectively by specifying a replication list for a message, and then that message is replicated only to the subset in the replication list. -The following is an example for the [Java API](client-libraries-java.md). Note the use of the `setReplicationClusters` method when you construct the {@inject: javadoc:Message:/client/org/apache/pulsar/client/api/Message} object: +The following is an example for the [Java API](client-libraries-java). Note the use of the `setReplicationClusters` method when you construct the {@inject: javadoc:Message:/client/org/apache/pulsar/client/api/Message} object: ```java + List restrictReplicationTo = Arrays.asList( "us-west", "us-east" @@ -146,14 +157,17 @@ producer.newMessage() .value("my-payload".getBytes()) .setReplicationClusters(restrictReplicationTo) .send(); + ``` #### Topic stats -Topic-specific statistics for geo-replication topics are available via the [`pulsar-admin`](reference-pulsar-admin.md) tool and {@inject: rest:REST:/} API: +Topic-specific statistics for geo-replication topics are available via the [`pulsar-admin`](reference-pulsar-admin) tool and {@inject: rest:REST:/} API: ```shell + $ bin/pulsar-admin persistent stats persistent://my-tenant/my-namespace/my-topic + ``` Each cluster reports its own local stats, including the incoming and outgoing replication rates and backlogs. @@ -183,11 +197,13 @@ In case of failover, a consumer can restart consuming from the failure point in Replicated subscription is disabled by default. You can enable replicated subscription when creating a consumer. ```java + Consumer consumer = client.newConsumer(Schema.STRING) .topic("my-topic") .subscriptionName("my-subscription") .replicateSubscriptionState(true) .subscribe(); + ``` ### Advantages diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-isolation.md b/site2/website-next/versioned_docs/version-2.7.3/administration-isolation.md index 2f1cfbb58f085..f57844b1ee22a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-isolation.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-isolation.md @@ -1,7 +1,7 @@ --- id: administration-isolation title: Pulsar isolation -sidebar_label: Pulsar isolation +sidebar_label: "Pulsar isolation" original_id: administration-isolation --- @@ -39,7 +39,9 @@ You can set a namespace isolation policy for a cluster using one of the followin ``` + pulsar-admin ns-isolation-policy set options + ``` For more information about the command `pulsar-admin ns-isolation-policy set options`, see [here](https://pulsar.apache.org/tools/pulsar-admin/). @@ -47,11 +49,13 @@ For more information about the command `pulsar-admin ns-isolation-policy set opt **Example** ```shell + bin/pulsar-admin ns-isolation-policy set \ --auto-failover-policy-type min_available \ --auto-failover-policy-params min_limit=1,usage_threshold=80 \ --namespaces my-tenant/my-namespace \ --primary 10.193.216.* my-cluster policy-name + ``` @@ -94,7 +98,9 @@ You can set a bookie affinity group using one of the following methods. ``` + pulsar-admin namespaces set-bookie-affinity-group options + ``` For more information about the command `pulsar-admin namespaces set-bookie-affinity-group options`, see [here](https://pulsar.apache.org/tools/pulsar-admin/). @@ -102,8 +108,10 @@ For more information about the command `pulsar-admin namespaces set-bookie-affin **Example** ```shell + bin/pulsar-admin namespaces set-bookie-affinity-group public/default \ --primary-group group-bookie1 + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-load-balance.md b/site2/website-next/versioned_docs/version-2.7.3/administration-load-balance.md index fc5d6d1295e56..a14142bcac7de 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-load-balance.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-load-balance.md @@ -1,7 +1,7 @@ --- id: administration-load-balance title: Pulsar load balance -sidebar_label: Load balance +sidebar_label: "Load balance" original_id: administration-load-balance --- @@ -53,15 +53,19 @@ Each bundle is independent of the others and thus is independently assigned to d When you create a new namespace, the new namespace sets to use the default number of bundles. You can set this in `conf/broker.conf`: ```properties + # When a namespace is created without specifying the number of bundle, this # value will be used as the default defaultNumberOfNamespaceBundles=4 + ``` You can either change the system default, or override it when you create a new namespace: ```shell + $ bin/pulsar-admin namespaces create my-tenant/my-namespace --clusters us-west --bundles 16 + ``` With this command, you create a namespace with 16 initial bundles. Therefore the topics for this namespaces can immediately be spread across up to 16 brokers. @@ -82,13 +86,17 @@ Unloading is the mechanism that the load-manager uses to perform the load sheddi Unloading a topic has no effect on the assignment, but just closes and reopens the particular topic: ```shell + pulsar-admin topics unload persistent://tenant/namespace/topic + ``` To unload all topics for a namespace and trigger reassignments: ```shell + pulsar-admin namespaces unload tenant/namespace + ``` ### Split namespace bundles @@ -98,6 +106,7 @@ Since the load for the topics in a bundle might change over time, or predicting The splitting happens based on some tunable thresholds. Any existing bundle that exceeds any of the threshold is a candidate to be split. By default the newly split bundles are also immediately offloaded to other brokers, to facilitate the traffic distribution. ```properties + # enable/disable namespace bundle auto split loadBalancerAutoBundleSplitEnabled=true @@ -118,6 +127,7 @@ loadBalancerNamespaceBundleMaxBandwidthMbytes=100 # maximum number of bundles in a namespace (for auto-split) loadBalancerNamespaceMaximumBundles=128 + ``` ### Shed load automatically @@ -135,19 +145,23 @@ and memory), broker unloads bundles for at least 15% of traffic. The automatic load shedding is enabled by default and you can disable the automatic load shedding with this setting: ```properties + # Enable/disable automatic bundle unloading for load-shedding loadBalancerSheddingEnabled=true + ``` Additional settings that apply to shedding: ```properties + # Load shedding interval. Broker periodically checks whether some traffic should be offload from # some over-loaded broker to other under-loaded brokers loadBalancerSheddingIntervalMinutes=1 # Prevent the same topics to be shed and moved to other brokers more that once within this timeframe loadBalancerSheddingGracePeriodMinutes=30 + ``` #### Broker overload thresholds @@ -157,8 +171,10 @@ The determinations of when a broker is overloaded is based on threshold of CPU, By default, overload threshold is set at 85%: ```properties + # Usage threshold to determine a broker as over-loaded loadBalancerBrokerOverloadedThresholdPercentage=85 + ``` Pulsar gathers the usage stats from the system metrics. @@ -172,6 +188,7 @@ Because of the incorrect max speed, the Pulsar load manager might think the brok You can use the following setting to correct the max NIC speed: ```properties + # Override the auto-detection of the network interfaces max speed. # This option is useful in some environments (eg: EC2 VMs) where the max speed # reported by Linux is not reflecting the real bandwidth available to the broker. @@ -180,6 +197,7 @@ You can use the following setting to correct the max NIC speed: # with the right value here. The configured value can be a double (eg: 0.8) and that # can be used to trigger load-shedding even before hitting on NIC limits. loadBalancerOverrideBrokerNicSpeedGbps= + ``` When the value is empty, Pulsar uses the value that the OS reports. diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-proxy.md b/site2/website-next/versioned_docs/version-2.7.3/administration-proxy.md index 31cc6e2d1c23f..821aa4d905a02 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-proxy.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-proxy.md @@ -1,7 +1,7 @@ --- id: administration-proxy title: Pulsar proxy -sidebar_label: Pulsar proxy +sidebar_label: "Pulsar proxy" original_id: administration-proxy --- @@ -18,9 +18,12 @@ Before using the proxy, you need to configure it with the brokers addresses in t ### Use service discovery Pulsar uses [ZooKeeper](https://zookeeper.apache.org) for service discovery. To connect the proxy to ZooKeeper, specify the following in `conf/proxy.conf`. + ```properties + zookeeperServers=zk-0,zk-1,zk-2 configurationStoreServers=zk-0:2184,zk-remote:2184 + ``` > To use service discovery, you need to open the network ACLs, so the proxy can connects to the ZooKeeper nodes through the ZooKeeper client port (port `2181`) and the configuration store client port (port `2184`). @@ -36,16 +39,21 @@ Proxy authorization requires access to ZooKeeper, so if you use these broker URL You can configure the broker URLs in `conf/proxy.conf` as follows. ```properties + brokerServiceURL=pulsar://brokers.example.com:6650 brokerWebServiceURL=http://brokers.example.com:8080 functionWorkerWebServiceURL=http://function-workers.example.com:8080 + ``` If you use TLS, configure the broker URLs in the following way: + ```properties + brokerServiceURLTLS=pulsar+ssl://brokers.example.com:6651 brokerWebServiceURLTLS=https://brokers.example.com:8443 functionWorkerWebServiceURL=https://function-workers.example.com:8443 + ``` The hostname in the URLs provided should be a DNS entry which points to multiple brokers or a virtual IP address, which is backed by multiple broker IP addresses, so that the proxy does not lose connectivity to Pulsar cluster if a single broker becomes unavailable. @@ -59,8 +67,10 @@ Note that if you do not use functions, you do not need to configure `functionWor To start the proxy: ```bash + $ cd /path/to/pulsar/directory $ bin/pulsar proxy + ``` > You can run multiple instances of the Pulsar proxy in a cluster. diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-pulsar-manager.md b/site2/website-next/versioned_docs/version-2.7.3/administration-pulsar-manager.md index facdeb8237274..6c8945e9e9783 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-pulsar-manager.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-pulsar-manager.md @@ -1,7 +1,7 @@ --- id: administration-pulsar-manager title: Pulsar Manager -sidebar_label: Pulsar Manager +sidebar_label: "Pulsar Manager" original_id: administration-pulsar-manager --- @@ -13,7 +13,7 @@ Pulsar Manager is a web-based GUI management and monitoring tool that helps admi :::note -If you monitor your current stats with [Pulsar dashboard](administration-dashboard.md), you can try to use Pulsar Manager instead. Pulsar dashboard is deprecated. +If you monitor your current stats with [Pulsar dashboard](administration-dashboard), you can try to use Pulsar Manager instead. Pulsar dashboard is deprecated. ::: @@ -22,11 +22,13 @@ If you monitor your current stats with [Pulsar dashboard](administration-dashboa The easiest way to use the Pulsar Manager is to run it inside a [Docker](https://www.docker.com/products/docker) container. ```shell + docker pull apachepulsar/pulsar-manager:v0.2.0 docker run -it \ -p 9527:9527 -p 7750:7750 \ -e SPRING_CONFIGURATION_FILE=/pulsar-manager/pulsar-manager/application.properties \ apachepulsar/pulsar-manager:v0.2.0 + ``` * `SPRING_CONFIGURATION_FILE`: Default configuration file for spring. @@ -34,18 +36,21 @@ docker run -it \ ### Set administrator account and password ```shell -CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) -curl \ - -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ - -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ - -H "Content-Type: application/json" \ - -X PUT http://localhost:7750/pulsar-manager/users/superuser \ - -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' -``` + + CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) + curl \ + -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ + -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ + -H "Content-Type: application/json" \ + -X PUT http://localhost:7750/pulsar-manager/users/superuser \ + -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' + + ``` You can find the docker image in the [Docker Hub](https://github.com/apache/pulsar-manager/tree/master/docker) directory and build an image from the source code as well: ``` + git clone https://github.com/apache/pulsar-manager cd pulsar-manager/front-end npm install --save @@ -54,6 +59,7 @@ cd .. ./gradlew build -x test cd .. docker build -f docker/Dockerfile --build-arg BUILD_DATE=`date -u +"%Y-%m-%dT%H:%M:%SZ"` --build-arg VCS_REF=`latest` --build-arg VERSION=`latest` -t apachepulsar/pulsar-manager . + ``` ### Use custom databases @@ -65,16 +71,20 @@ If you have a large amount of data, you can use a custom database. The following 2. Modify the [configuration file](https://github.com/apache/pulsar-manager/blob/master/src/main/resources/application.properties) and add PostgreSQL configuration. ``` + spring.datasource.driver-class-name=org.postgresql.Driver spring.datasource.url=jdbc:postgresql://127.0.0.1:5432/pulsar_manager spring.datasource.username=postgres spring.datasource.password=postgres + ``` 3. Compile to generate a new executable jar package. ``` + ./gradlew build -x test + ``` ### Enable JWT authentication @@ -96,14 +106,17 @@ If you want to enable JWT authentication, use one of the following methods. * Method 1: use command-line tool ``` + wget https://dist.apache.org/repos/dist/release/pulsar/pulsar-manager/apache-pulsar-manager-0.2.0/apache-pulsar-manager-0.2.0-bin.tar.gz tar -zxvf apache-pulsar-manager-0.2.0-bin.tar.gz cd pulsar-manager tar -zxvf pulsar-manager.tar cd pulsar-manager cp -r ../dist ui -./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key +./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key + ``` + Firstly, [set the administrator account and password](#set-administrator-account-and-password) Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. @@ -111,6 +124,7 @@ Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. * Method 2: configure the application.properties file ``` + backend.jwt.token=token jwt.broker.token.mode=PRIVATE @@ -120,13 +134,16 @@ jwt.broker.private.key=file:///path/broker-private.key or jwt.broker.token.mode=SECRET jwt.broker.secret.key=file:///path/broker-secret.key + ``` * Method 3: use Docker and enable token authentication. ``` + export JWT_TOKEN="your-token" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -v $PWD:/data apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --secret-key` or `bin/pulsar tokens create --private-key` command. @@ -141,10 +158,12 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 4: use Docker and turn on **token authentication** and **token management** by private key and public key. ``` + export JWT_TOKEN="your-token" export PRIVATE_KEY="file:///pulsar-manager/secret/my-private.key" export PUBLIC_KEY="file:///pulsar-manager/secret/my-public.key" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -e PRIVATE_KEY=$PRIVATE_KEY -e PUBLIC_KEY=$PUBLIC_KEY -v $PWD:/data -v $PWD/secret:/pulsar-manager/secret apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --private-key` command. @@ -162,9 +181,11 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 5: use Docker and turn on **token authentication** and **token management** by secret key. ``` + export JWT_TOKEN="your-token" export SECRET_KEY="file:///pulsar-manager/secret/my-secret.key" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -e SECRET_KEY=$SECRET_KEY -v $PWD:/data -v $PWD/secret:/pulsar-manager/secret apachepulsar/pulsar-manager:v0.2.0 /bin/sh + ``` * `JWT_TOKEN`: the token of superuser configured for the broker. It is generated by the `bin/pulsar tokens create --secret-key` command. @@ -178,7 +199,7 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * `PASSWORD`: the password of PostgreSQL. * `LOG_LEVEL`: the level of log. -* For more information about backend configurations, see [here](https://github.com/apache/pulsar-manager/blob/master/src/README.md). +* For more information about backend configurations, see [here](https://github.com/apache/pulsar-manager/blob/master/src/README). * For more information about frontend configurations, see [here](https://github.com/apache/pulsar-manager/tree/master/front-end). ## Log in diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-stats.md b/site2/website-next/versioned_docs/version-2.7.3/administration-stats.md index fade1697d83cc..029ebf2d1d8f8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-stats.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-stats.md @@ -1,7 +1,7 @@ --- id: administration-stats title: Pulsar stats -sidebar_label: Pulsar statistics +sidebar_label: "Pulsar statistics" original_id: administration-stats --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-upgrade.md b/site2/website-next/versioned_docs/version-2.7.3/administration-upgrade.md index d76dafd62f02e..aefdd8125ddcc 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-upgrade.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-upgrade.md @@ -1,7 +1,7 @@ --- id: administration-upgrade title: Upgrade Guide -sidebar_label: Upgrade +sidebar_label: "Upgrade" original_id: administration-upgrade --- @@ -35,15 +35,24 @@ To upgrade an Apache Pulsar cluster, follow the upgrade sequence. 2. Upgrade bookies - Canary test: test an upgraded version in one or a small set of bookies. - Rolling upgrade: - - a. Disable `autorecovery` with the following command. - ```shell - bin/bookkeeper shell autorecovery -disable - ``` - - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. - - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. - ```shell - bin/bookkeeper shell autorecovery -enable - ``` + - a. Disable `autorecovery` with the following command. + + ```shell + + bin/bookkeeper shell autorecovery -disable + + ``` + + + - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. + - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. + + ```shell + + bin/bookkeeper shell autorecovery -enable + + ``` + 3. Upgrade brokers - Canary test: test an upgraded version in one or a small set of brokers. - Rolling upgrade: rollout the upgraded version to all brokers in the cluster after you determine that a version is safe after canary. @@ -90,13 +99,21 @@ To upgrade bookie to a new version, complete the following steps: 1. Stop a bookie. 2. Upgrade the binary and configuration files. 3. Start the bookie in `ReadOnly` mode to verify if the bookie of this new version runs well for read workload. + ```shell + bin/pulsar bookie --readOnly + ``` + 4. When the bookie runs successfully in `ReadOnly` mode, stop the bookie and restart it in `Write/Read` mode. + ```shell + bin/pulsar bookie + ``` + 5. Observe and make sure the cluster serves both write and read traffic. #### Canary rollback diff --git a/site2/website-next/versioned_docs/version-2.7.3/administration-zk-bk.md b/site2/website-next/versioned_docs/version-2.7.3/administration-zk-bk.md index b33e3094e0509..a4da422aaa960 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/administration-zk-bk.md +++ b/site2/website-next/versioned_docs/version-2.7.3/administration-zk-bk.md @@ -1,7 +1,7 @@ --- id: administration-zk-bk title: ZooKeeper and BookKeeper administration -sidebar_label: ZooKeeper and BookKeeper +sidebar_label: "ZooKeeper and BookKeeper" original_id: administration-zk-bk --- @@ -35,9 +35,11 @@ To deploy a Pulsar instance, you need to stand up one local ZooKeeper cluster *p To begin, add all ZooKeeper servers to the quorum configuration specified in the [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) file. Add a `server.N` line for each node in the cluster to the configuration, where `N` is the number of the ZooKeeper node. The following is an example for a three-node cluster: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` On each host, you need to specify the node ID in `myid` file of each node, which is in `data/zookeeper` folder of each server by default (you can change the file location via the [`dataDir`](reference-configuration.md#zookeeper-dataDir) parameter). @@ -48,8 +50,10 @@ On each host, you need to specify the node ID in `myid` file of each node, which On a ZooKeeper server at `zk1.us-west.example.com`, for example, you can set the `myid` value like this: ```shell + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com` the command is `echo 2 > data/zookeeper/myid` and so on. @@ -57,7 +61,9 @@ On `zk2.us-west.example.com` the command is `echo 2 > data/zookeeper/myid` and s Once you add each server to the `zookeeper.conf` configuration and each server has the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start zookeeper + ``` ### Deploy configuration store @@ -70,13 +76,15 @@ If you deploy a [single-cluster](#single-cluster-pulsar-instance) instance, you If your Pulsar instance consists of just one cluster, then you can deploy a configuration store on the same machines as the local ZooKeeper quorum but run on different TCP ports. -To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorom uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: +To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorum uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 server.3=zk3.us-west.example.com:2185:2186 + ``` As before, create the `myid` files for each server on `data/global-zookeeper/myid`. @@ -92,7 +100,9 @@ Again, given the very low expected load on the configuration store servers, you For example, you can assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also you can assume, each cluster has its own local ZK servers named such as ``` + zk[1-3].${CLUSTER}.example.com + ``` In this scenario you want to pick the quorum participants from few clusters and let all the others be ZK observers. For example, to form a 7 servers quorum, you can pick 3 servers from `us-west`, 2 from `us-central` and 2 from `us-east`. @@ -102,6 +112,7 @@ This guarantees that writes to configuration store is possible even if one of th The ZK configuration in all the servers looks like: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 @@ -118,12 +129,15 @@ server.12=zk3.eu-central.example.com:2185:2186:observer server.13=zk1.ap-south.example.com:2185:2186:observer server.14=zk2.ap-south.example.com:2185:2186:observer server.15=zk3.ap-south.example.com:2185:2186:observer + ``` Additionally, ZK observers need to have: ```properties + peerType=observer + ``` ##### Start the service @@ -131,7 +145,9 @@ peerType=observer Once your configuration store configuration is in place, you can start up the service using [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) ```shell + $ bin/pulsar-daemon start configuration-store + ``` ### ZooKeeper configuration @@ -163,7 +179,7 @@ The [`conf/global-zookeeper.conf`](reference-configuration.md#configuration-stor BookKeeper stores all durable message in Pulsar. BookKeeper is a distributed [write-ahead log](https://en.wikipedia.org/wiki/Write-ahead_logging) WAL system that guarantees read consistency of independent message logs calls ledgers. Individual BookKeeper servers are also called *bookies*. -> To manage message persistence, retention, and expiry in Pulsar, refer to [cookbook](cookbooks-retention-expiry.md). +> To manage message persistence, retention, and expiry in Pulsar, refer to [cookbook](cookbooks-retention-expiry). ### Hardware requirements @@ -184,6 +200,7 @@ You can configure BookKeeper bookies using the [`conf/bookkeeper.conf`](referenc The minimum configuration changes required in `conf/bookkeeper.conf` are as follows: ```properties + # Change to point to journal disk mount point journalDirectory=data/bookkeeper/journal @@ -192,6 +209,7 @@ ledgerDirectories=data/bookkeeper/ledgers # Point to local ZK quorum zkServers=zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181 + ``` To change the ZooKeeper root path that BookKeeper uses, use `zkLedgersRootPath=/MY-PREFIX/ledgers` instead of `zkServers=localhost:2181/MY-PREFIX`. @@ -209,19 +227,25 @@ You can start a bookie in the foreground or as a background daemon. To start a bookie in the foreground, use the [`bookkeeper`](reference-cli-tools.md#bookkeeper) CLI tool: ```bash + $ bin/bookkeeper bookie + ``` To start a bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` You can verify whether the bookie works properly with the `bookiesanity` command for the [BookKeeper shell](reference-cli-tools.md#bookkeeper-shell): ```shell + $ bin/bookkeeper shell bookiesanity + ``` When you use this command, you create a new ledger on the local bookie, write a few entries, read them back and finally delete the ledger. @@ -246,9 +270,9 @@ And then you can decommission bookies safely. To decommission bookies, complete 3. Run the decommission command. - If you have logged in to the node to be decommissioned, you do not need to provide `-bookieid`. - If you are running the decommission command for the target bookie node from another bookie node, you should mention the target bookie ID in the arguments for `-bookieid` - `$ bin/bookkeeper shell decommissionbookie` - or - `$ bin/bookkeeper shell decommissionbookie -bookieid ` + `$ bin/bookkeeper shell decommissionbookie` + or + `$ bin/bookkeeper shell decommissionbookie -bookieid ` 4. Validate that no ledgers are on the decommissioned bookie. `$ bin/bookkeeper shell listledgers -bookieid ` @@ -256,8 +280,10 @@ And then you can decommission bookies safely. To decommission bookies, complete You can run the following command to check if the bookie you have decommissioned is listed in the bookies list: ```bash + ./bookkeeper shell listbookies -rw -h ./bookkeeper shell listbookies -ro -h + ``` ## BookKeeper persistence policies @@ -279,7 +305,7 @@ Use the [`set-persistence`](reference-pulsar-admin.md#namespaces-set-persistence Flag | Description | Default :----|:------------|:------- -`-a`, `--bookkeeper-ack-quorom` | The number of acks (guaranteed copies) to wait on for each entry | 0 +`-a`, `--bookkeeper-ack-quorum` | The number of acks (guaranteed copies) to wait on for each entry | 0 `-e`, `--bookkeeper-ensemble` | The number of [bookies](reference-terminology.md#bookie) to use for topics in the namespace | 0 `-w`, `--bookkeeper-write-quorum` | The number of writes to make for each entry | 0 `-r`, `--ml-mark-delete-max-rate` | Throttling rate for mark-delete operations (0 means no throttle) | 0 @@ -287,9 +313,11 @@ Flag | Description | Default The following is an example: ```shell + $ pulsar-admin namespaces set-persistence my-tenant/my-ns \ - --bookkeeper-ack-quorom 3 \ + --bookkeeper-ack-quorum 3 \ --bookeeper-ensemble 2 + ``` #### REST API @@ -299,6 +327,7 @@ $ pulsar-admin namespaces set-persistence my-tenant/my-ns \ #### Java ```java + int bkEnsemble = 2; int bkQuorum = 3; int bkAckQuorum = 2; @@ -306,6 +335,7 @@ double markDeleteRate = 0.7; PersistencePolicies policies = new PersistencePolicies(ensemble, quorum, ackQuorum, markDeleteRate); admin.namespaces().setPersistence(namespace, policies); + ``` ### List persistence policies @@ -319,6 +349,7 @@ Use the [`get-persistence`](reference-pulsar-admin.md#namespaces-get-persistence The following is an example: ```shell + $ pulsar-admin namespaces get-persistence my-tenant/my-ns { "bookkeeperEnsemble": 1, @@ -326,6 +357,7 @@ $ pulsar-admin namespaces get-persistence my-tenant/my-ns "bookkeeperAckQuorum", 1, "managedLedgerMaxMarkDeleteRate": 0 } + ``` #### REST API @@ -335,7 +367,9 @@ $ pulsar-admin namespaces get-persistence my-tenant/my-ns #### Java ```java + PersistencePolicies policies = admin.namespaces().getPersistence(namespace); + ``` ## How Pulsar uses ZooKeeper and BookKeeper diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cgo.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cgo.md index 422cb8fe4334d..b7e216f8f9985 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cgo.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cgo.md @@ -1,7 +1,7 @@ --- id: client-libraries-cgo title: Pulsar CGo client -sidebar_label: CGo(deprecated) +sidebar_label: "CGo(deprecated)" original_id: client-libraries-cgo --- @@ -580,3 +580,4 @@ fmt.Println(s.Name) // output: pulsar defer consumer.Close() ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cpp.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cpp.md index ee0aae2d9ca47..6e7a091ae829d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cpp.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-cpp.md @@ -1,7 +1,7 @@ --- id: client-libraries-cpp title: Pulsar C++ client -sidebar_label: C++ +sidebar_label: "C++" original_id: client-libraries-cpp --- @@ -63,6 +63,7 @@ The command the when use this pulsar library is like this: g++ --std=c++11 PulsarTest.cpp -o test /usr/lib/libpulsarwithdeps.a -lssl -lcrypto -ldl -lpthread -I/usr/local/ssl/include -L/usr/local/ssl/lib ``` + `libpulsarwithdeps.a` does not include library openssl related libraries: `libssl` and `libcrypto`, because these 2 library is related to security, by using user local system provided version is more reasonable, and more easy for user to handling security issue and library upgrade. @@ -196,6 +197,7 @@ pulsar+ssl://pulsar.us-west.example.com:6651 To connect to Pulsar as a consumer, you need to create a consumer on the C++ client. The following is an example. ```c++ + Client client("pulsar://localhost:6650"); Consumer consumer; @@ -223,6 +225,7 @@ client.close(); To connect to Pulsar as a producer, you need to create a producer on the C++ client. The following is an example. ```c++ + Client client("pulsar://localhost:6650"); Producer producer; @@ -295,3 +298,4 @@ consumerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); client.subscribe("topic-avro", "sub-2", consumerConf, consumer) ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-dotnet.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-dotnet.md index 8fe67bd908361..68c58105d1f18 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-dotnet.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-dotnet.md @@ -1,7 +1,7 @@ --- id: client-libraries-dotnet title: Pulsar C# client -sidebar_label: C# +sidebar_label: "C#" original_id: client-libraries-dotnet --- @@ -31,10 +31,11 @@ To install the Pulsar C# client library, following these steps: 3. Create the project using the following command. - ``` - dotnet new console - - ``` + ``` + + dotnet new console + + ``` 4. Use `dotnet run` to test that the app has been created properly. @@ -42,20 +43,21 @@ To install the Pulsar C# client library, following these steps: 1. Use the following command to install the `DotPulsar` package. - ``` - dotnet add package DotPulsar - - ``` + ``` + + dotnet add package DotPulsar + + ``` 2. After the command completes, open the `.csproj` file to see the added reference. - ```xml - - - - - - ``` + ```xml + + + + + + ``` ## Client @@ -66,6 +68,7 @@ This section describes some configuration examples for the Pulsar C# client. This example shows how to create a Pulsar C# client connected to localhost. ```c# + var client = PulsarClient.Builder().Build(); ``` @@ -83,20 +86,22 @@ This section describes how to create a producer. - Create a producer by using the builder. - ```c# - var producer = client.NewProducer() - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var producer = client.NewProducer() + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a producer without using the builder. - ```c# - var options = new ProducerOptions("persistent://public/default/mytopic"); - var producer = client.CreateProducer(options); - - ``` + ```c# + + var options = new ProducerOptions("persistent://public/default/mytopic"); + var producer = client.CreateProducer(options); + + ``` ### Create consumer @@ -104,21 +109,23 @@ This section describes how to create a consumer. - Create a consumer by using the builder. - ```c# - var consumer = client.NewConsumer() - .SubscriptionName("MySubscription") - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var consumer = client.NewConsumer() + .SubscriptionName("MySubscription") + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a consumer without using the builder. - ```c# - var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); - var consumer = client.CreateConsumer(options); - - ``` + ```c# + + var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); + var consumer = client.CreateConsumer(options); + + ``` ### Create reader @@ -126,21 +133,23 @@ This section describes how to create a reader. - Create a reader by using the builder. - ```c# - var reader = client.NewReader() - .StartMessageId(MessageId.Earliest) - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var reader = client.NewReader() + .StartMessageId(MessageId.Earliest) + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a reader without using the builder. - ```c# - var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); - var reader = client.CreateReader(options); - - ``` + ```c# + + var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); + var reader = client.CreateReader(options); + + ``` ### Configure encryption policies @@ -154,6 +163,7 @@ The Pulsar C# client supports four kinds of encryption policies: This example shows how to set the `EnforceUnencrypted` encryption policy. ```c# + var client = PulsarClient.Builder() .ConnectionSecurity(EncryptionPolicy.EnforceEncrypted) .Build(); @@ -168,20 +178,22 @@ If you have followed [Authentication using TLS](security-tls-authentication), yo 1. Create an unencrypted and password-less pfx file. - ```c# - openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: - - ``` + ```c# + + openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: + + ``` 2. Use the admin.pfx file to create an X509Certificate2 and pass it to the Pulsar C# client. - ```c# - var clientCertificate = new X509Certificate2("admin.pfx"); - var client = PulsarClient.Builder() - .AuthenticateUsingClientCertificate(clientCertificate) - .Build(); - - ``` + ```c# + + var clientCertificate = new X509Certificate2("admin.pfx"); + var client = PulsarClient.Builder() + .AuthenticateUsingClientCertificate(clientCertificate) + .Build(); + + ``` ## Producer @@ -192,6 +204,7 @@ A producer is a process that attaches to a topic and publishes messages to a Pul This example shows how to send data. ```c# + var data = Encoding.UTF8.GetBytes("Hello World"); await producer.Send(data); @@ -201,23 +214,25 @@ await producer.Send(data); - Send messages with customized metadata by using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var messageId = await producer.NewMessage() - .Property("SomeKey", "SomeValue") - .Send(data); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var messageId = await producer.NewMessage() + .Property("SomeKey", "SomeValue") + .Send(data); + + ``` - Send messages with customized metadata without using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var metadata = new MessageMetadata(); - metadata["SomeKey"] = "SomeValue"; - var messageId = await producer.Send(metadata, data)); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var metadata = new MessageMetadata(); + metadata["SomeKey"] = "SomeValue"; + var messageId = await producer.Send(metadata, data)); + + ``` ## Consumer @@ -228,6 +243,7 @@ A consumer is a process that attaches to a topic through a subscription and then This example shows how a consumer receives messages from a topic. ```c# + await foreach (var message in consumer.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -241,26 +257,29 @@ Messages can be acknowledged individually or cumulatively. For details about mes - Acknowledge messages individually. - ```c# - await foreach (var message in consumer.Messages()) - { - Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); - } - - ``` + ```c# + + await foreach (var message in consumer.Messages()) + { + Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); + } + + ``` - Acknowledge messages cumulatively. - ```c# - await consumer.AcknowledgeCumulative(message); - - ``` + ```c# + + await consumer.AcknowledgeCumulative(message); + + ``` ### Unsubscribe from topics This example shows how a consumer unsubscribes from a topic. ```c# + await consumer.Unsubscribe(); ``` @@ -276,6 +295,7 @@ A reader is actually just a consumer without a cursor. This means that Pulsar do This example shows how a reader receives messages. ```c# + await foreach (var message in reader.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -301,6 +321,7 @@ The following table lists states available for the producer. This example shows how to monitor the producer state. ```c# + private static async ValueTask Monitor(IProducer producer, CancellationToken cancellationToken) { var state = ProducerState.Disconnected; @@ -343,6 +364,7 @@ The following table lists states available for the consumer. This example shows how to monitor the consumer state. ```c# + private static async ValueTask Monitor(IConsumer consumer, CancellationToken cancellationToken) { var state = ConsumerState.Disconnected; @@ -386,6 +408,7 @@ The following table lists states available for the reader. This example shows how to monitor the reader state. ```c# + private static async ValueTask Monitor(IReader reader, CancellationToken cancellationToken) { var state = ReaderState.Disconnected; @@ -412,3 +435,4 @@ private static async ValueTask Monitor(IReader reader, CancellationToken cancell } ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-go.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-go.md index a3ab1bfe7a2ce..e6d3c5679df90 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-go.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-go.md @@ -1,7 +1,7 @@ --- id: client-libraries-go title: Pulsar Go client -sidebar_label: Go +sidebar_label: "Go" original_id: client-libraries-go --- @@ -721,3 +721,4 @@ client, err := pulsar.NewClient(pulsar.ClientOptions{ }) ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-java.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-java.md index 78189aa5dacc1..f1c4171fef326 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-java.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-java.md @@ -1,7 +1,7 @@ --- id: client-libraries-java title: Pulsar Java client -sidebar_label: Java +sidebar_label: "Java" original_id: client-libraries-java --- @@ -176,22 +176,29 @@ stringProducer.send("My message"); ``` > Make sure that you close your producers, consumers, and clients when you do not need them. -> ```java +> ```java +> > producer.close(); > consumer.close(); > client.close(); +> +> > ``` + > > Close operations can also be asynchronous: -> ```java +> ```java +> > producer.closeAsync() > .thenRun(() -> System.out.println("Producer closed")) > .exceptionally((ex) -> { > System.err.println("Failed to close producer: " + ex); > return null; > }); +> +> > ``` ### Configure producer @@ -403,9 +410,7 @@ consumer.acknowledge(messages) :::note - Batch receive policy limits the number and bytes of messages in a single batch. You can specify a timeout to wait for enough messages. - The batch receive is completed if any of the following condition is met: enough number of messages, bytes of messages, wait timeout. ```java @@ -421,6 +426,7 @@ Consumer consumer = client.newConsumer() .subscribe(); ``` + The default batch receive policy is: ```java @@ -482,7 +488,6 @@ pulsarClient.newConsumer() :::note - By default, the `subscriptionTopicsMode` of the consumer is `PersistentOnly`. Available options of `subscriptionTopicsMode` are `PersistentOnly`, `NonPersistentOnly`, and `AllTopics`. ::: @@ -579,7 +584,6 @@ Only the first consumer is allowed to the subscription, other consumers receive :::note - If topic is a partitioned topic, the first consumer subscribes to all partitioned topics, other consumers are not assigned with partitions and receive an error. ::: @@ -603,8 +607,6 @@ Consumer consumer2 = client.newConsumer() //conumser1 is the active consumer, consumer2 is the standby consumer. //consumer1 receives 5 messages and then crashes, consumer2 takes over as an active consumer. - - ``` Multiple consumers can attach to the same subscription, yet only the first consumer is active, and others are standby. When the active consumer is disconnected, messages will be dispatched to one of standby consumers, and the standby consumer then becomes active consumer. @@ -612,6 +614,7 @@ Multiple consumers can attach to the same subscription, yet only the first consu If the first active consumer is disconnected after receiving 5 messages, the standby consumer becomes active consumer. Consumer1 will receive: ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -623,6 +626,7 @@ If the first active consumer is disconnected after receiving 5 messages, the sta consumer2 will receive: ``` + ("key-2", "message-2-3") ("key-3", "message-3-1") ("key-3", "message-3-2") @@ -633,7 +637,6 @@ consumer2 will receive: :::note - If a topic is a partitioned topic, each partition has only one active consumer, messages of one partition are distributed to only one consumer, and messages of multiple partitions are distributed to multiple consumers. ::: @@ -664,6 +667,7 @@ In shared subscription mode, multiple consumers can attach to the same subscript If a broker dispatches only one message at a time, consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-3") ("key-2", "message-2-2") @@ -675,6 +679,7 @@ If a broker dispatches only one message at a time, consumer1 receives the follow consumer2 receives the following information. ``` + ("key-1", "message-1-2") ("key-2", "message-2-1") ("key-2", "message-2-3") @@ -711,6 +716,7 @@ Consumer consumer2 = client.newConsumer() consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -722,6 +728,7 @@ consumer1 receives the following information. consumer2 receives the following information. ``` + ("key-2", "message-2-1") ("key-2", "message-2-2") ("key-2", "message-2-3") @@ -740,6 +747,7 @@ Producer producer = client.newProducer() .create(); ``` + Or the producer can disable batching. ```java @@ -750,8 +758,8 @@ Producer producer = client.newProducer() .create(); ``` -:::note +:::note If the message key is not specified, messages without key are dispatched to one consumer in order by default. @@ -872,62 +880,61 @@ The following schema formats are currently available for Java: * No schema or the byte array schema (which can be applied using `Schema.BYTES`): ```java - + Producer bytesProducer = client.newProducer(Schema.BYTES) - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` Or, equivalently: ```java - + Producer bytesProducer = client.newProducer() - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` * `String` for normal UTF-8-encoded string data. Apply the schema using `Schema.STRING`: ```java - + Producer stringProducer = client.newProducer(Schema.STRING) - .topic("some-string-topic") - .create(); - + .topic("some-string-topic") + .create(); + ``` * Create JSON schemas for POJOs using `Schema.JSON`. The following is an example. ```java - + Producer pojoProducer = client.newProducer(Schema.JSON(MyPojo.class)) - .topic("some-pojo-topic") - .create(); - + .topic("some-pojo-topic") + .create(); + ``` * Generate Protobuf schemas using `Schema.PROTOBUF`. The following example shows how to create the Protobuf schema and use it to instantiate a new producer: ```java - + Producer protobufProducer = client.newProducer(Schema.PROTOBUF(MyProtobuf.class)) - .topic("some-protobuf-topic") - .create(); - + .topic("some-protobuf-topic") + .create(); + ``` * Define Avro schemas with `Schema.AVRO`. The following code snippet demonstrates how to create and use Avro schema. - ```java - + Producer avroProducer = client.newProducer(Schema.AVRO(MyAvro.class)) - .topic("some-avro-topic") - .create(); - + .topic("some-avro-topic") + .create(); + ``` ## Authentication @@ -1024,3 +1031,4 @@ PulsarClient client = PulsarClient.builder() .build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-node.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-node.md index 9b8ecd41d1307..74671a8d78678 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-node.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-node.md @@ -1,7 +1,7 @@ --- id: client-libraries-node title: The Pulsar Node.js client -sidebar_label: Node.js +sidebar_label: "Node.js" original_id: client-libraries-node --- @@ -45,7 +45,6 @@ $ npm install pulsar-client :::note - Also, this library works only in Node.js 10.x or later because it uses the [`node-addon-api`](https://github.com/nodejs/node-addon-api) module to wrap the C++ library. ::: @@ -124,7 +123,7 @@ Here is an example: ```JavaScript const producer = await client.createProducer({ - topic: 'my-topic', // or 'my-tenant/my-namespace/my-topic' to specify topic's tenant and namespace + topic: 'my-topic', }); await producer.send({ @@ -146,7 +145,7 @@ Pulsar Node.js producers have the following methods available: | Method | Description | Return type | | :----- | :---------- | :---------- | | `send(Object)` | Publishes a [message](#messages) to the producer's topic. When the message is successfully acknowledged by the Pulsar broker, or an error will be thrown, the Promise object run executor function. | `Promise` | -| `flush()` | Sends message from send queue to Pulser broker. When the message is successfully acknowledged by the Pulsar broker, or an error will be thrown, the Promise object run executor function. | `Promise` | +| `flush()` | Sends message from send queue to Pulsar broker. When the message is successfully acknowledged by the Pulsar broker, or an error will be thrown, the Promise object run executor function. | `Promise` | | `close()` | Closes the producer and releases all resources allocated to it. If `close()` is called then no more messages will be accepted from the publisher. This method will return Promise object, and when all pending publish requests have been persisted by Pulsar then run executor function. If an error is thrown, no pending writes will be retried. | `Promise` | ### Producer configuration @@ -471,6 +470,7 @@ The following static methods are available for the message id object: If you want to use the end-to-end encryption feature in the Node.js client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -488,148 +488,151 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** + **Input** - ```shell - - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem - - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://localhost:6650', - operationTimeoutSeconds: 30, - }); - - // Create a producer - const producer = await client.createProducer({ - topic: 'persistent://public/default/my-topic', - sendTimeoutMs: 30000, - batchingEnabled: true, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem", - encryptionKey: "encryption-key" - }); - - console.log(producer.ProducerConfig) - // Send messages - for (let i = 0; i < 10; i += 1) { - const msg = `my-message-${i}`; - producer.send({ - data: Buffer.from(msg), - }); - console.log(`Sent message: ${msg}`); - } - await producer.flush(); - - await producer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://localhost:6650', + operationTimeoutSeconds: 30, + }); + + // Create a producer + const producer = await client.createProducer({ + topic: 'persistent://public/default/my-topic', + sendTimeoutMs: 30000, + batchingEnabled: true, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem", + encryptionKey: "encryption-key" + }); + + console.log(producer.ProducerConfig) + // Send messages + for (let i = 0; i < 10; i += 1) { + const msg = `my-message-${i}`; + producer.send({ + data: Buffer.from(msg), + }); + console.log(`Sent message: ${msg}`); + } + await producer.flush(); + + await producer.close(); + await client.close(); + })(); + + ``` 3. Create a consumer to receive encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://172.25.0.3:6650', - operationTimeoutSeconds: 30 - }); - - // Create a consumer - const consumer = await client.subscribe({ - topic: 'persistent://public/default/my-topic', - subscription: 'sub1', - subscriptionType: 'Shared', - ackTimeoutMs: 10000, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem" - }); - - console.log(consumer) - // Receive messages - for (let i = 0; i < 10; i += 1) { - const msg = await consumer.receive(); - console.log(msg.getData().toString()); - consumer.acknowledge(msg); - } - - await consumer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://172.25.0.3:6650', + operationTimeoutSeconds: 30 + }); + + // Create a consumer + const consumer = await client.subscribe({ + topic: 'persistent://public/default/my-topic', + subscription: 'sub1', + subscriptionType: 'Shared', + ackTimeoutMs: 10000, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem" + }); + + console.log(consumer) + // Receive messages + for (let i = 0; i < 10; i += 1) { + const msg = await consumer.receive(); + console.log(msg.getData().toString()); + consumer.acknowledge(msg); + } + + await consumer.close(); + await client.close(); + })(); + + ``` 4. Run the consumer to receive encrypted messages. - **Input** + **Input** - ```shell - - node consumer.js - - ``` + ```shell + + node consumer.js + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - node producer.js - - ``` - - Now you can see the producer sends messages and the consumer receives messages successfully. - - **Output** - - This is from the producer side. - - ``` - Sent message: my-message-0 - Sent message: my-message-1 - Sent message: my-message-2 - Sent message: my-message-3 - Sent message: my-message-4 - Sent message: my-message-5 - Sent message: my-message-6 - Sent message: my-message-7 - Sent message: my-message-8 - Sent message: my-message-9 - - ``` - - This is from the consumer side. - - ``` - my-message-0 - my-message-1 - my-message-2 - my-message-3 - my-message-4 - my-message-5 - my-message-6 - my-message-7 - my-message-8 - my-message-9 + **Input** + + ```shell + + node producer.js + + ``` + + Now you can see the producer sends messages and the consumer receives messages successfully. + + **Output** + + This is from the producer side. + + ``` + + Sent message: my-message-0 + Sent message: my-message-1 + Sent message: my-message-2 + Sent message: my-message-3 + Sent message: my-message-4 + Sent message: my-message-5 + Sent message: my-message-6 + Sent message: my-message-7 + Sent message: my-message-8 + Sent message: my-message-9 + + ``` + + This is from the consumer side. + + ``` + + my-message-0 + my-message-1 + my-message-2 + my-message-3 + my-message-4 + my-message-5 + my-message-6 + my-message-7 + my-message-8 + my-message-9 + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-python.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-python.md index 804c2e6efe720..c1886d35d32e6 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-python.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-python.md @@ -1,7 +1,7 @@ --- id: client-libraries-python title: Pulsar Python client -sidebar_label: Python +sidebar_label: "Python" original_id: client-libraries-python --- @@ -149,6 +149,7 @@ while True: # No acknowledgment ``` + ### Multi-topic subscriptions In addition to subscribing a consumer to a single Pulsar topic, you can also subscribe to multiple topics simultaneously. To use multi-topic subscriptions, you can supply a regular expression (regex) or a `List` of topics. If you select topics via regex, all topics must be within the same Pulsar namespace. @@ -333,6 +334,7 @@ class Example(Record): If you want to use the end-to-end encryption feature in the Python client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -350,89 +352,92 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** - - ```shell - - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem + **Input** - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** + **Input** - ```python + ```python + + import pulsar - import pulsar - - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) - producer.send('encryption message'.encode('utf8')) - print('sent message') - producer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) + producer.send('encryption message'.encode('utf8')) + print('sent message') + producer.close() + client.close() + + ``` 3. Create a consumer to receive encrypted messages. - **Input** - - ```python + **Input** - import pulsar + ```python + + import pulsar - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) - msg = consumer.receive() - print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) - consumer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) + msg = consumer.receive() + print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) + consumer.close() + client.close() + + ``` 4. Run the consumer to receive encrypted messages. - **Input** - - ```shell - - python consumer.py + **Input** - ``` + ```shell + + python consumer.py + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - python producer.py - - ``` + **Input** - Now you can see the producer sends messages and the consumer receives messages successfully. + ```shell + + python producer.py + + ``` - **Output** + Now you can see the producer sends messages and the consumer receives messages successfully. - This is from the producer side. + **Output** - ``` - sent message + This is from the producer side. - ``` + ``` + + sent message + + ``` - This is from the consumer side. + This is from the consumer side. - ``` - Received msg 'b'encryption message'' id = '(0,0,-1,-1)' + ``` + + Received msg 'b'encryption message'' id = '(0,0,-1,-1)' + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-websocket.md b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-websocket.md index 460ba52f4d664..9ed10057581ec 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/client-libraries-websocket.md +++ b/site2/website-next/versioned_docs/version-2.7.3/client-libraries-websocket.md @@ -1,7 +1,7 @@ --- id: client-libraries-websocket title: Pulsar WebSocket API -sidebar_label: WebSocket +sidebar_label: "WebSocket" original_id: client-libraries-websocket --- @@ -73,7 +73,7 @@ The producer endpoint requires you to specify a tenant, namespace, and topic in ```http -ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic +ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic ``` @@ -125,6 +125,7 @@ Key | Type | Required? | Explanation } ``` + ##### Example failure response ```json @@ -495,3 +496,4 @@ ws.on('message', function(message) { }); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-architecture-overview.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-architecture-overview.md index 57159c14ffeed..2719db4dfd654 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-architecture-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-architecture-overview.md @@ -1,11 +1,15 @@ --- id: concepts-architecture-overview title: Architecture Overview -sidebar_label: Architecture +sidebar_label: "Architecture" original_id: concepts-architecture-overview --- -At the highest level, a Pulsar instance is composed of one or more Pulsar clusters. Clusters within an instance can [replicate](concepts-replication.md) data amongst themselves. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +At the highest level, a Pulsar instance is composed of one or more Pulsar clusters. Clusters within an instance can [replicate](concepts-replication) data amongst themselves. In a Pulsar cluster: @@ -17,20 +21,20 @@ The diagram below provides an illustration of a Pulsar cluster: ![Pulsar architecture diagram](/assets/pulsar-system-architecture.png) -At the broader instance level, an instance-wide ZooKeeper cluster called the configuration store handles coordination tasks involving multiple clusters, for example [geo-replication](concepts-replication.md). +At the broader instance level, an instance-wide ZooKeeper cluster called the configuration store handles coordination tasks involving multiple clusters, for example [geo-replication](concepts-replication). ## Brokers The Pulsar message broker is a stateless component that's primarily responsible for running two other components: * An HTTP server that exposes a {@inject: rest:REST:/} API for both administrative tasks and [topic lookup](concepts-clients.md#client-setup-phase) for producers and consumers -* A dispatcher, which is an asynchronous TCP server over a custom [binary protocol](developing-binary-protocol.md) used for all data transfers +* A dispatcher, which is an asynchronous TCP server over a custom [binary protocol](developing-binary-protocol) used for all data transfers Messages are typically dispatched out of a [managed ledger](#managed-ledgers) cache for the sake of performance, *unless* the backlog exceeds the cache size. If the backlog grows too large for the cache, the broker will start reading entries from BookKeeper. -Finally, to support geo-replication on global topics, the broker manages replicators that tail the entries published in the local region and republish them to the remote region using the Pulsar [Java client library](client-libraries-java.md). +Finally, to support geo-replication on global topics, the broker manages replicators that tail the entries published in the local region and republish them to the remote region using the Pulsar [Java client library](client-libraries-java). -> For a guide to managing Pulsar brokers, see the [brokers](admin-api-brokers.md) guide. +> For a guide to managing Pulsar brokers, see the [brokers](admin-api-brokers) guide. ## Clusters @@ -40,9 +44,9 @@ A Pulsar instance consists of one or more Pulsar *clusters*. Clusters, in turn, * A ZooKeeper quorum used for cluster-level configuration and coordination * An ensemble of bookies used for [persistent storage](#persistent-storage) of messages -Clusters can replicate amongst themselves using [geo-replication](concepts-replication.md). +Clusters can replicate amongst themselves using [geo-replication](concepts-replication). -> For a guide to managing Pulsar clusters, see the [clusters](admin-api-clusters.md) guide. +> For a guide to managing Pulsar clusters, see the [clusters](admin-api-clusters) guide. ## Metadata store @@ -73,7 +77,9 @@ In addition to message data, *cursors* are also persistently stored in BookKeepe At the moment, Pulsar supports persistent message storage. This accounts for the `persistent` in all topic names. Here's an example: ```http + persistent://my-tenant/my-namespace/my-topic + ``` > Pulsar also supports ephemeral ([non-persistent](concepts-messaging.md#non-persistent-topics)) message storage. @@ -120,23 +126,25 @@ The **Pulsar proxy** provides a solution to this problem by acting as a single g Architecturally, the Pulsar proxy gets all the information it requires from ZooKeeper. When starting the proxy on a machine, you only need to provide ZooKeeper connection strings for the cluster-specific and instance-wide configuration store clusters. Here's an example: ```bash + $ bin/pulsar proxy \ --zookeeper-servers zk-0,zk-1,zk-2 \ --configuration-store-servers zk-0,zk-1,zk-2 + ``` > #### Pulsar proxy docs -> For documentation on using the Pulsar proxy, see the [Pulsar proxy admin documentation](administration-proxy.md). +> For documentation on using the Pulsar proxy, see the [Pulsar proxy admin documentation](administration-proxy). Some important things to know about the Pulsar proxy: * Connecting clients don't need to provide *any* specific configuration to use the Pulsar proxy. You won't need to update the client configuration for existing applications beyond updating the IP used for the service URL (for example if you're running a load balancer over the Pulsar proxy). -* [TLS encryption](security-tls-transport.md) and [authentication](security-tls-authentication.md) is supported by the Pulsar proxy +* [TLS encryption](security-tls-transport.md) and [authentication](security-tls-authentication) is supported by the Pulsar proxy ## Service discovery -[Clients](getting-started-clients.md) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. Pulsar provides a built-in service discovery mechanism that you can set up using the instructions in the [Deploying a Pulsar instance](deploy-bare-metal.md#service-discovery-setup) guide. +[Clients](getting-started-clients) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. Pulsar provides a built-in service discovery mechanism that you can set up using the instructions in the [Deploying a Pulsar instance](deploy-bare-metal.md#service-discovery-setup) guide. You can use your own service discovery system if you'd like. If you use your own system, there is just one requirement: when a client performs an HTTP request to an endpoint, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to *some* active broker in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. @@ -144,13 +152,19 @@ The diagram below illustrates Pulsar service discovery: ![alt-text](/assets/pulsar-service-discovery.png) -In this diagram, the Pulsar cluster is addressable via a single DNS name: `pulsar-cluster.acme.com`. A [Python client](client-libraries-python.md), for example, could access this Pulsar cluster like this: +In this diagram, the Pulsar cluster is addressable via a single DNS name: `pulsar-cluster.acme.com`. A [Python client](client-libraries-python), for example, could access this Pulsar cluster like this: ```python + from pulsar import Client client = Client('pulsar://pulsar-cluster.acme.com:6650') + ``` -> **Note** -> In Pulsar, each topic is handled by only one broker. Initial requests from a client to read, update or delete a topic are sent to a broker that may not be the topic owner. If the broker cannot handle the request for this topic, it redirects the request to the appropriate broker. +:::note + +In Pulsar, each topic is handled by only one broker. Initial requests from a client to read, update or delete a topic are sent to a broker that may not be the topic owner. If the broker cannot handle the request for this topic, it redirects the request to the appropriate broker. + +::: + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-authentication.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-authentication.md index 781d0689e1a3a..5580a0b019fb4 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-authentication.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-authentication.md @@ -1,9 +1,13 @@ --- id: concepts-authentication title: Authentication and Authorization -sidebar_label: Authentication and Authorization +sidebar_label: "Authentication and Authorization" original_id: concepts-authentication --- -Pulsar supports a pluggable [authentication](security-overview.md) mechanism which can be configured at the proxy and/or the broker. Pulsar also supports a pluggable [authorization](security-authorization.md) mechanism. These mechanisms work together to identify the client and its access rights on topics, namespaces and tenants. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar supports a pluggable [authentication](security-overview.md) mechanism which can be configured at the proxy and/or the broker. Pulsar also supports a pluggable [authorization](security-authorization) mechanism. These mechanisms work together to identify the client and its access rights on topics, namespaces and tenants. diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-clients.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-clients.md index 1ce286532e91a..36bef54264652 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-clients.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-clients.md @@ -1,16 +1,20 @@ --- id: concepts-clients title: Pulsar Clients -sidebar_label: Clients +sidebar_label: "Clients" original_id: concepts-clients --- -Pulsar exposes a client API with language bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md), [C++](client-libraries-cpp.md) and [C#](client-libraries-dotnet.md). The client API optimizes and encapsulates Pulsar's client-broker communication protocol and exposes a simple and intuitive API for use by applications. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar exposes a client API with language bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md), [C++](client-libraries-cpp.md) and [C#](client-libraries-dotnet). The client API optimizes and encapsulates Pulsar's client-broker communication protocol and exposes a simple and intuitive API for use by applications. Under the hood, the current official Pulsar client libraries support transparent reconnection and/or connection failover to brokers, queuing of messages until acknowledged by the broker, and heuristics such as connection retries with backoff. > **Custom client libraries** -> If you'd like to create your own client library, we recommend consulting the documentation on Pulsar's custom [binary protocol](developing-binary-protocol.md). +> If you'd like to create your own client library, we recommend consulting the documentation on Pulsar's custom [binary protocol](developing-binary-protocol). ## Client setup phase @@ -38,7 +42,7 @@ Internally, the reader interface is implemented as a consumer using an exclusive [ **IMPORTANT** ] -Unlike subscription/consumer, readers are non-durable in nature and does not prevent data in a topic from being deleted, thus it is ***strongly*** advised that [data retention](cookbooks-retention-expiry.md) be configured. If data retention for a topic is not configured for an adequate amount of time, messages that the reader has not yet read might be deleted . This causes the readers to essentially skip messages. Configuring the data retention for a topic guarantees the reader with a certain duration to read a message. +Unlike subscription/consumer, readers are non-durable in nature and does not prevent data in a topic from being deleted, thus it is ***strongly*** advised that [data retention](cookbooks-retention-expiry) be configured. If data retention for a topic is not configured for an adequate amount of time, messages that the reader has not yet read might be deleted . This causes the readers to essentially skip messages. Configuring the data retention for a topic guarantees the reader with a certain duration to read a message. Please also note that a reader can have a "backlog", but the metric is only used for users to know how behind the reader is. The metric is not considered for any backlog quota calculations. @@ -47,6 +51,7 @@ Please also note that a reader can have a "backlog", but the metric is only used Here's a Java example that begins reading from the earliest available message on a topic: ```java + import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Reader; @@ -62,24 +67,30 @@ while (true) { // Process the message } + ``` To create a reader that reads from the latest available message: ```java + Reader reader = pulsarClient.newReader() .topic(topic) .startMessageId(MessageId.latest) .create(); + ``` To create a reader that reads from some message between the earliest and the latest: ```java + byte[] msgIdBytes = // Some byte array MessageId id = MessageId.fromByteArray(msgIdBytes); Reader reader = pulsarClient.newReader() .topic(topic) .startMessageId(id) .create(); + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-messaging.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-messaging.md index 02c7e1e8611ca..ca26b305fbeee 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-messaging.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-messaging.md @@ -1,10 +1,14 @@ --- id: concepts-messaging title: Messaging -sidebar_label: Messaging +sidebar_label: "Messaging" original_id: concepts-messaging --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar is built on the [publish-subscribe](https://en.wikipedia.org/wiki/Publish%E2%80%93subscribe_pattern) pattern (often abbreviated to pub-sub). In this pattern, [producers](#producers) publish messages to [topics](#topics). [Consumers](#consumers) [subscribe](#subscription-modes) to those topics, process incoming messages, and send an acknowledgement when processing is complete. When a subscription is created, Pulsar [retains](concepts-architecture-overview.md#persistent-storage) all messages, even if the consumer is disconnected. Retained messages are discarded only when a consumer acknowledges that those messages are processed successfully. @@ -15,8 +19,8 @@ Messages are the basic "unit" of Pulsar. The following table lists the component Component | Description :---------|:------- -Value / data payload | The data carried by the message. All Pulsar messages contain raw bytes, although message data can also conform to data [schemas](schema-get-started.md). -Key | Messages are optionally tagged with keys, which is useful for things like [topic compaction](concepts-topic-compaction.md). +Value / data payload | The data carried by the message. All Pulsar messages contain raw bytes, although message data can also conform to data [schemas](schema-get-started). +Key | Messages are optionally tagged with keys, which is useful for things like [topic compaction](concepts-topic-compaction). Properties | An optional key/value map of user-defined properties. Producer name | The name of the producer who produces the message. If you do not specify a producer name, the default name is used. Sequence ID | Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID of the message is its order in that sequence. @@ -24,7 +28,7 @@ Publish time | The timestamp of when the message is published. The timestamp is Event time | An optional timestamp attached to a message by applications. For example, applications attach a timestamp on when the message is processed. If nothing is set to event time, the value is `0`. TypedMessageBuilder | It is used to construct a message. You can set message properties such as the message key, message value with `TypedMessageBuilder`.
    When you set `TypedMessageBuilder`, set the key as a string. If you set the key as other types, for example, an AVRO object, the key is sent as bytes, and it is difficult to get the AVRO object back on the consumer. -> For more information on Pulsar message contents, see Pulsar [binary protocol](developing-binary-protocol.md). +> For more information on Pulsar message contents, see Pulsar [binary protocol](developing-binary-protocol). ## Producers @@ -101,7 +105,7 @@ Messages are received from [brokers](reference-terminology.md#broker) either syn ### Listeners -Client libraries provide listener implementation for consumers. For example, the [Java client](client-libraries-java.md) provides a {@inject: javadoc:MesssageListener:/client/org/apache/pulsar/client/api/MessageListener} interface. In this interface, the `received` method is called whenever a new message is received. +Client libraries provide listener implementation for consumers. For example, the [Java client](client-libraries-java) provides a {@inject: javadoc:MesssageListener:/client/org/apache/pulsar/client/api/MessageListener} interface. In this interface, the `received` method is called whenever a new message is received. ### Acknowledgement @@ -114,8 +118,11 @@ Messages can be acknowledged in the following two ways: - Messages are acknowledged individually. With individual acknowledgement, the consumer needs to acknowledge each message and sends an acknowledgement request to the broker. - Messages are acknowledged cumulatively. With cumulative acknowledgement, the consumer only needs to acknowledge the last message it received. All messages in the stream up to (and including) the provided message are not re-delivered to that consumer. -> **Note** -> Cumulative acknowledgement cannot be used in the [shared subscription mode](#subscription-modes), because the shared subscription mode involves multiple consumers who have access to the same subscription. In the shared subscription mode, messages are acknowledged individually. +:::note + +Cumulative acknowledgement cannot be used in the [shared subscription mode](#subscription-modes), because the shared subscription mode involves multiple consumers who have access to the same subscription. In the shared subscription mode, messages are acknowledged individually. + +::: ### Negative acknowledgement @@ -129,18 +136,27 @@ In the shared and Key_Shared subscription modes, you can negatively acknowledge Be aware that negative acknowledgment on ordered subscription types, such as Exclusive, Failover and Key_Shared, can cause failed messages to arrive consumers out of the original order. -> **Note** -> If batching is enabled, other messages and the negatively acknowledged messages in the same batch are redelivered to the consumer. +:::note + +If batching is enabled, other messages and the negatively acknowledged messages in the same batch are redelivered to the consumer. + +::: ### Acknowledgement timeout If a message is not consumed successfully, and you want to trigger the broker to redeliver the message automatically, you can adopt the unacknowledged message automatic re-delivery mechanism. Client tracks the unacknowledged messages within the entire `acktimeout` time range, and sends a `redeliver unacknowledged messages` request to the broker automatically when the acknowledgement timeout is specified. -> **Note** -> If batching is enabled, other messages and the unacknowledged messages in the same batch are redelivered to the consumer. +:::note -> **Note** -> Prefer negative acknowledgements over acknowledgement timeout. Negative acknowledgement controls the re-delivery of individual messages with more precision, and avoids invalid redeliveries when the message processing time exceeds the acknowledgement timeout. +If batching is enabled, other messages and the unacknowledged messages in the same batch are redelivered to the consumer. + +::: + +:::note + +Prefer negative acknowledgements over acknowledgement timeout. Negative acknowledgement controls the re-delivery of individual messages with more precision, and avoids invalid redeliveries when the message processing time exceeds the acknowledgement timeout. + +::: ### Dead letter topic @@ -149,6 +165,7 @@ Dead letter topic enables you to consume new messages when some messages cannot The following example shows how to enable dead letter topic in a Java client using the default dead letter topic: ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -157,16 +174,22 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .maxRedeliverCount(maxRedeliveryCount) .build()) .subscribe(); - + ``` + The default dead letter topic uses this format: + ``` + --DLQ + ``` + If you want to specify the name of the dead letter topic, use this Java client example: ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -176,13 +199,16 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .deadLetterTopic("your-topic-name") .build()) .subscribe(); - + ``` - + Dead letter topic depends on message re-delivery. Messages are redelivered either due to [acknowledgement timeout](#acknowledgement-timeout) or [negative acknowledgement](#negative-acknowledgement). If you are going to use negative acknowledgement on a message, make sure it is negatively acknowledged before the acknowledgement timeout. -> **Note** -> Currently, dead letter topic is enabled only in the shared subscription mode. +:::note + +Currently, dead letter topic is enabled only in the shared subscription mode. + +::: ### Retry letter topic @@ -193,6 +219,7 @@ By default, automatic retry is disabled. You can set `enableRetry` to `true` to This example shows how to consume messages from a retry letter topic. ```java + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .topic(topic) .subscriptionName("my-subscription") @@ -205,6 +232,7 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .build()) .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); + ``` ## Topics @@ -212,7 +240,9 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) As in other pub-sub systems, topics in Pulsar are named channels for transmitting messages from producers to consumers. Topic names are URLs that have a well-defined structure: ```http + {persistent|non-persistent}://tenant/namespace/topic + ``` Topic name component | Description @@ -311,6 +341,7 @@ When subscribing to multiple topics, the Pulsar client automatically makes a cal The following are multi-topic subscription examples for Java. ```java + import java.util.regex.Pattern; import org.apache.pulsar.client.api.Consumer; @@ -331,6 +362,7 @@ Consumer someTopicsConsumer = pulsarClient.newConsumer() .topicsPattern(someTopicsInNamespace) .subscriptionName("subscription-1") .subscribe(); + ``` For code examples, see [Java](client-libraries-java.md#multi-topic-subscriptions). @@ -353,7 +385,7 @@ Decisions about routing and subscription modes can be made separately in most ca There is no difference between partitioned topics and normal topics in terms of how subscription modes work, as partitioning only determines what happens between when a message is published by a producer and processed and acknowledged by a consumer. -Partitioned topics need to be explicitly created via the [admin API](admin-api-overview.md). The number of partitions can be specified when creating the topic. +Partitioned topics need to be explicitly created via the [admin API](admin-api-overview). The number of partitions can be specified when creating the topic. ### Routing modes @@ -365,7 +397,7 @@ Mode | Description :--------|:------------ `RoundRobinPartition` | If no key is provided, the producer will publish messages across all partitions in round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it's set to the same boundary of batching delay, to ensure batching is effective. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. This is the default mode. `SinglePartition` | If no key is provided, the producer will randomly pick one single partition and publish all the messages into that partition. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. -`CustomPartition` | Use custom message router implementation that will be called to determine the partition for a particular message. User can create a custom routing mode by using the [Java client](client-libraries-java.md) and implementing the {@inject: javadoc:MessageRouter:/client/org/apache/pulsar/client/api/MessageRouter} interface. +`CustomPartition` | Use custom message router implementation that will be called to determine the partition for a particular message. User can create a custom routing mode by using the [Java client](client-libraries-java) and implementing the {@inject: javadoc:MessageRouter:/client/org/apache/pulsar/client/api/MessageRouter} interface. ### Ordering guarantee @@ -398,10 +430,12 @@ Pulsar also, however, supports **non-persistent topics**, which are topics on wh Non-persistent topics have names of this form (note the `non-persistent` in the name): ```http + non-persistent://tenant/namespace/topic + ``` -> For more info on using non-persistent topics, see the [Non-persistent messaging cookbook](cookbooks-non-persistent.md). +> For more info on using non-persistent topics, see the [Non-persistent messaging cookbook](cookbooks-non-persistent). In non-persistent topics, brokers immediately deliver messages to all connected subscribers *without persisting them* in [BookKeeper](concepts-architecture-overview.md#persistent-storage). If a subscriber is disconnected, the broker will not be able to deliver those in-transit messages, and subscribers will never be able to receive those messages again. Eliminating the persistent storage step makes messaging on non-persistent topics slightly faster than on persistent topics in some cases, but with the caveat that some of the core benefits of Pulsar are lost. @@ -420,6 +454,7 @@ Producers and consumers can connect to non-persistent topics in the same way as Here's an example [Java consumer](client-libraries-java.md#consumers) for a non-persistent topic: ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .build(); @@ -430,14 +465,17 @@ Consumer consumer = client.newConsumer() .topic(npTopic) .subscriptionName(subscriptionName) .subscribe(); + ``` Here's an example [Java producer](client-libraries-java.md#producer) for the same non-persistent topic: ```java + Producer producer = client.newProducer() .topic(npTopic) .create(); + ``` ## Message retention and expiry @@ -452,7 +490,7 @@ Pulsar has two features, however, that enable you to override this default behav * Message **retention** enables you to store messages that have been acknowledged by a consumer * Message **expiry** enables you to set a time to live (TTL) for messages that have not yet been acknowledged -> All message retention and expiry is managed at the [namespace](#namespaces) level. For a how-to, see the [Message retention and expiry](cookbooks-retention-expiry.md) cookbook. +> All message retention and expiry is managed at the [namespace](#namespaces) level. For a how-to, see the [Message retention and expiry](cookbooks-retention-expiry) cookbook. The diagram below illustrates both concepts: @@ -475,12 +513,12 @@ Message deduplication is disabled in the scenario shown at the top. Here, a prod In the second scenario at the bottom, the producer publishes message 1, which is received by the broker and persisted, as in the first scenario. When the producer attempts to publish the message again, however, the broker knows that it has already seen message 1 and thus does not persist the message. -> Message deduplication is handled at the namespace level or the topic level. For more instructions, see the [message deduplication cookbook](cookbooks-deduplication.md). +> Message deduplication is handled at the namespace level or the topic level. For more instructions, see the [message deduplication cookbook](cookbooks-deduplication). ### Producer idempotency -The other available approach to message deduplication is to ensure that each message is *only produced once*. This approach is typically called **producer idempotency**. The drawback of this approach is that it defers the work of message deduplication to the application. In Pulsar, this is handled at the [broker](reference-terminology.md#broker) level, so you do not need to modify your Pulsar client code. Instead, you only need to make administrative changes. For details, see [Managing message deduplication](cookbooks-deduplication.md). +The other available approach to message deduplication is to ensure that each message is *only produced once*. This approach is typically called **producer idempotency**. The drawback of this approach is that it defers the work of message deduplication to the application. In Pulsar, this is handled at the [broker](reference-terminology.md#broker) level, so you do not need to modify your Pulsar client code. Instead, you only need to make administrative changes. For details, see [Managing message deduplication](cookbooks-deduplication). ### Deduplication and effectively-once semantics @@ -503,6 +541,7 @@ A broker saves a message without any check. When a consumer consumes a message, Delayed message delivery is enabled by default. You can change it in the broker configuration file as below: ``` + # Whether to enable the delayed delivery for messages. # If disabled, messages are immediately delivered and there is no tracking overhead. delayedDeliveryEnabled=true @@ -511,11 +550,16 @@ delayedDeliveryEnabled=true # affecting the accuracy of the delivery time compared to the scheduled time. # Default is 1 second. delayedDeliveryTickTimeMillis=1000 + ``` ### Producer The following is an example of delayed message delivery for a producer in Java: + ```java + // message to be delivered at the configured delay interval producer.newMessage().deliverAfter(3L, TimeUnit.Minute).value("Hello Pulsar!").send(); + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-multi-tenancy.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-multi-tenancy.md index 15c802f12002b..2c556c9dce554 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-multi-tenancy.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-multi-tenancy.md @@ -1,16 +1,22 @@ --- id: concepts-multi-tenancy title: Multi Tenancy -sidebar_label: Multi Tenancy +sidebar_label: "Multi Tenancy" original_id: concepts-multi-tenancy --- -Pulsar was created from the ground up as a multi-tenant system. To support multi-tenancy, Pulsar has a concept of tenants. Tenants can be spread across clusters and can each have their own [authentication and authorization](security-overview.md) scheme applied to them. They are also the administrative unit at which storage quotas, [message TTL](cookbooks-retention-expiry.md#time-to-live-ttl), and isolation policies can be managed. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar was created from the ground up as a multi-tenant system. To support multi-tenancy, Pulsar has a concept of tenants. Tenants can be spread across clusters and can each have their own [authentication and authorization](security-overview) scheme applied to them. They are also the administrative unit at which storage quotas, [message TTL](cookbooks-retention-expiry.md#time-to-live-ttl), and isolation policies can be managed. The multi-tenant nature of Pulsar is reflected mostly visibly in topic URLs, which have this structure: ```http + persistent://tenant/namespace/topic + ``` As you can see, the tenant is the most basic unit of categorization for topics (more fundamental than the namespace and topic name). @@ -19,7 +25,7 @@ As you can see, the tenant is the most basic unit of categorization for topics ( To each tenant in a Pulsar instance you can assign: -* An [authorization](security-authorization.md) scheme +* An [authorization](security-authorization) scheme * The set of [clusters](reference-terminology.md#cluster) to which the tenant's configuration applies ## Namespaces @@ -27,16 +33,18 @@ To each tenant in a Pulsar instance you can assign: Tenants and namespaces are two key concepts of Pulsar to support multi-tenancy. * Pulsar is provisioned for specified tenants with appropriate capacity allocated to the tenant. -* A namespace is the administrative unit nomenclature within a tenant. The configuration policies set on a namespace apply to all the topics created in that namespace. A tenant may create multiple namespaces via self-administration using the REST API and the [`pulsar-admin`](reference-pulsar-admin.md) CLI tool. For instance, a tenant with different applications can create a separate namespace for each application. +* A namespace is the administrative unit nomenclature within a tenant. The configuration policies set on a namespace apply to all the topics created in that namespace. A tenant may create multiple namespaces via self-administration using the REST API and the [`pulsar-admin`](reference-pulsar-admin) CLI tool. For instance, a tenant with different applications can create a separate namespace for each application. Names for topics in the same namespace will look like this: ```http + persistent://tenant/app1/topic-1 persistent://tenant/app1/topic-2 persistent://tenant/app1/topic-3 + ``` ### Namespace change events and topic-level policies diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-multiple-advertised-listeners.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-multiple-advertised-listeners.md index 327a4d644c14e..a5e9663c17caa 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-multiple-advertised-listeners.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-multiple-advertised-listeners.md @@ -1,10 +1,14 @@ --- id: concepts-multiple-advertised-listeners title: Multiple advertised listeners -sidebar_label: Multiple advertised listeners +sidebar_label: "Multiple advertised listeners" original_id: concepts-multiple-advertised-listeners --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + When a Pulsar cluster is deployed in the production environment, it may require to expose multiple advertised addresses for the broker. For example, when you deploy a Pulsar cluster in Kubernetes and want other clients, which are not in the same Kubernetes cluster, to connect to the Pulsar cluster, you need to assign a broker URL to external clients. But clients in the same Kubernetes cluster can still connect to the Pulsar cluster through the internal network of Kubernetes. ## Advertised listeners @@ -25,15 +29,20 @@ This example shows how a Pulsar client uses multiple advertised listeners. 1. Configure multiple advertised listeners in the broker configuration file. ```shell + advertisedListeners={listenerName}:pulsar://xxxx:6650, {listenerName}:pulsar+ssl://xxxx:6651 + ``` 2. Specify the listener name for the client. ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://xxxx:6650") .listenerName("external") .build(); + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-overview.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-overview.md index a37b11fc96523..d16f57bf3c2af 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-overview.md @@ -1,31 +1,35 @@ --- id: concepts-overview title: Pulsar Overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: concepts-overview --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar is a multi-tenant, high-performance solution for server-to-server messaging. Pulsar was originally developed by Yahoo, it is under the stewardship of the [Apache Software Foundation](https://www.apache.org/). Key features of Pulsar are listed below: -* Native support for multiple clusters in a Pulsar instance, with seamless [geo-replication](administration-geo.md) of messages across clusters. +* Native support for multiple clusters in a Pulsar instance, with seamless [geo-replication](administration-geo) of messages across clusters. * Very low publish and end-to-end latency. * Seamless scalability to over a million topics. -* A simple [client API](concepts-clients.md) with bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) and [C++](client-libraries-cpp.md). +* A simple [client API](concepts-clients.md) with bindings for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) and [C++](client-libraries-cpp). * Multiple [subscription modes](concepts-messaging.md#subscription-modes) ([exclusive](concepts-messaging.md#exclusive), [shared](concepts-messaging.md#shared), and [failover](concepts-messaging.md#failover)) for topics. * Guaranteed message delivery with [persistent message storage](concepts-architecture-overview.md#persistent-storage) provided by [Apache BookKeeper](http://bookkeeper.apache.org/). -* A serverless light-weight computing framework [Pulsar Functions](functions-overview.md) offers the capability for stream-native data processing. -* A serverless connector framework [Pulsar IO](io-overview.md), which is built on Pulsar Functions, makes it easier to move data in and out of Apache Pulsar. -* [Tiered Storage](concepts-tiered-storage.md) offloads data from hot/warm storage to cold/longterm storage (such as S3 and GCS) when the data is aging out. +* A serverless light-weight computing framework [Pulsar Functions](functions-overview) offers the capability for stream-native data processing. +* A serverless connector framework [Pulsar IO](io-overview), which is built on Pulsar Functions, makes it easier to move data in and out of Apache Pulsar. +* [Tiered Storage](concepts-tiered-storage) offloads data from hot/warm storage to cold/longterm storage (such as S3 and GCS) when the data is aging out. ## Contents -- [Messaging Concepts](concepts-messaging.md) -- [Architecture Overview](concepts-architecture-overview.md) -- [Pulsar Clients](concepts-clients.md) -- [Geo Replication](concepts-replication.md) -- [Multi Tenancy](concepts-multi-tenancy.md) -- [Authentication and Authorization](concepts-authentication.md) -- [Topic Compaction](concepts-topic-compaction.md) -- [Tiered Storage](concepts-tiered-storage.md) +- [Messaging Concepts](concepts-messaging) +- [Architecture Overview](concepts-architecture-overview) +- [Pulsar Clients](concepts-clients) +- [Geo Replication](concepts-replication) +- [Multi Tenancy](concepts-multi-tenancy) +- [Authentication and Authorization](concepts-authentication) +- [Topic Compaction](concepts-topic-compaction) +- [Tiered Storage](concepts-tiered-storage) diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-proxy-sni-routing.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-proxy-sni-routing.md index ae761f97ae753..6501a661c06ba 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-proxy-sni-routing.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-proxy-sni-routing.md @@ -1,10 +1,14 @@ --- id: concepts-proxy-sni-routing title: Proxy support with SNI routing -sidebar_label: Proxy support with SNI routing +sidebar_label: "Proxy support with SNI routing" original_id: concepts-proxy-sni-routing --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + ## Pulsar Proxy with SNI routing A proxy server is an intermediary server that forwards requests from multiple clients to different servers across the Internet. The proxy server acts as a "traffic cop" in both forward and reverse proxy scenarios, and benefits your system such as load balancing, performance, security, auto-scaling, and so on. @@ -33,6 +37,7 @@ To configure the `records.config` files, complete the following steps. The following is an example. ``` + # PROXY TLS PORT CONFIG proxy.config.http.server_ports STRING 4443:ssl 4080 # PROXY CERTS FILE PATH @@ -43,6 +48,7 @@ CONFIG proxy.config.ssl.client.cert.filename STRING /proxy-key.pem # The range of origin server ports that can be used for tunneling via CONNECT. # Traffic Server allows tunnels only to the specified ports. Supports both wildcards (*) and ranges (e.g. 0-1023). CONFIG proxy.config.http.connect_ports STRING 4443 6651 + ``` The [ssl_server_name](https://docs.trafficserver.apache.org/en/8.0.x/admin-guide/files/ssl_server_name.yaml.en.html) file is used to configure TLS connection handling for inbound and outbound connections. The configuration is determined by the SNI values provided by the inbound connection. The file consists of a set of configuration items, and each is identified by an SNI value (`fqdn`). When an inbound TLS connection is made, the SNI value from the TLS negotiation is matched with the items specified in this file. If the values match, the values specified in that item override the default values. @@ -50,6 +56,7 @@ The [ssl_server_name](https://docs.trafficserver.apache.org/en/8.0.x/admin-guide The following example shows mapping of the inbound SNI hostname coming from the client, and the actual broker service URL where request should be redirected. For example, if the client sends the SNI header `pulsar-broker1`, the proxy creates a TLS tunnel by redirecting request to the `pulsar-broker1:6651` service URL. ``` + server_config = { { fqdn = 'pulsar-broker-vip', @@ -67,6 +74,7 @@ server_config = { tunnel_route = 'pulsar-broker2:6651' }, } + ``` After you configure the `ssl_server_name.config` and `records.config` files, the ATS-proxy server handles SNI routing and creates TCP tunnel between the client and the broker. @@ -75,6 +83,7 @@ After you configure the `ssl_server_name.config` and `records.config` files, the ATS SNI-routing works only with TLS. You need to enable TLS for the ATS proxy and brokers first, configure the SNI routing protocol, and then connect Pulsar clients to brokers through ATS proxy. Pulsar clients support SNI routing by connecting to the proxy, and sending the target broker URL to the SNI header. This process is processed internally. You only need to configure the following proxy configuration initially when you create a Pulsar client to use the SNI routing protocol. ``` + String brokerServiceUrl = “pulsar+ssl://pulsar-broker-vip:6651/”; String proxyUrl = “pulsar+ssl://ats-proxy:443”; ClientBuilder clientBuilder = PulsarClient.builder() @@ -85,12 +94,13 @@ ClientBuilder clientBuilder = PulsarClient.builder() .proxyServiceUrl(proxyUrl, ProxyProtocol.SNI) .operationTimeout(1000, TimeUnit.MILLISECONDS); -Map authParams = new HashMap<>(); +Map authParams = new HashMap(); authParams.put("tlsCertFile", TLS_CLIENT_CERT_FILE_PATH); authParams.put("tlsKeyFile", TLS_CLIENT_KEY_FILE_PATH); clientBuilder.authentication(AuthenticationTls.class.getName(), authParams); PulsarClient pulsarClient = clientBuilder.build(); + ``` #### Pulsar geo-replication with SNI routing @@ -103,19 +113,24 @@ In this example, a Pulsar cluster is deployed into two separate regions, `us-wes (a) Configure the cluster metadata for `us-east` with `us-east` broker service URL and `us-east` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://east-broker-vip:6651 \ --url http://east-broker-vip:8080 \ --proxy-protocol SNI \ --proxy-url pulsar+ssl://east-ats-proxy:443 + ``` (b) Configure the cluster metadata for `us-west` with `us-west` broker service URL and `us-west` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://west-broker-vip:6651 \ --url http://west-broker-vip:8080 \ --proxy-protocol SNI \ --proxy-url pulsar+ssl://west-ats-proxy:443 + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-replication.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-replication.md index 09f16e4064cfd..0be527ab82a2f 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-replication.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-replication.md @@ -1,9 +1,13 @@ --- id: concepts-replication title: Geo Replication -sidebar_label: Geo Replication +sidebar_label: "Geo Replication" original_id: concepts-replication --- -Pulsar enables messages to be produced and consumed in different geo-locations. For instance, your application may be publishing data in one region or market and you would like to process it for consumption in other regions or markets. [Geo-replication](administration-geo.md) in Pulsar enables you to do that. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar enables messages to be produced and consumed in different geo-locations. For instance, your application may be publishing data in one region or market and you would like to process it for consumption in other regions or markets. [Geo-replication](administration-geo) in Pulsar enables you to do that. diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-tiered-storage.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-tiered-storage.md index c00f1096d5e8e..8207661cabfaa 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-tiered-storage.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-tiered-storage.md @@ -1,10 +1,14 @@ --- id: concepts-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" original_id: concepts-tiered-storage --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar's segment oriented architecture allows for topic backlogs to grow very large, effectively without limit. However, this can become expensive over time. One way to alleviate this cost is to use Tiered Storage. With tiered storage, older messages in the backlog can be moved from BookKeeper to a cheaper storage mechanism, while still allowing clients to access the backlog as if nothing had changed. @@ -15,4 +19,4 @@ One way to alleviate this cost is to use Tiered Storage. With tiered storage, ol Pulsar currently supports S3, Google Cloud Storage (GCS), and filesystem for [long term store](https://pulsar.apache.org/docs/en/cookbooks-tiered-storage/). Offloading to long term storage triggered via a Rest API or command line interface. The user passes in the amount of topic data they wish to retain on BookKeeper, and the broker will copy the backlog data to long term storage. The original data will then be deleted from BookKeeper after a configured delay (4 hours by default). -> For a guide for setting up tiered storage, see the [Tiered storage cookbook](cookbooks-tiered-storage.md). +> For a guide for setting up tiered storage, see the [Tiered storage cookbook](cookbooks-tiered-storage). diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-topic-compaction.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-topic-compaction.md index 3aca60f71c21c..c685721e83c70 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-topic-compaction.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-topic-compaction.md @@ -1,13 +1,17 @@ --- id: concepts-topic-compaction title: Topic Compaction -sidebar_label: Topic Compaction +sidebar_label: "Topic Compaction" original_id: concepts-topic-compaction --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Pulsar was built with highly scalable [persistent storage](concepts-architecture-overview.md#persistent-storage) of message data as a primary objective. Pulsar topics enable you to persistently store as many unacknowledged messages as you need while preserving message ordering. By default, Pulsar stores *all* unacknowledged/unprocessed messages produced on a topic. Accumulating many unacknowledged messages on a topic is necessary for many Pulsar use cases but it can also be very time intensive for Pulsar consumers to "rewind" through the entire log of messages. -> For a more practical guide to topic compaction, see the [Topic compaction cookbook](cookbooks-compaction.md). +> For a more practical guide to topic compaction, see the [Topic compaction cookbook](cookbooks-compaction). For some use cases consumers don't need a complete "image" of the topic log. They may only need a few values to construct a more "shallow" image of the log, perhaps even just the most recent value. For these kinds of use cases Pulsar offers **topic compaction**. When you run compaction on a topic, Pulsar goes through a topic's backlog and removes messages that are *obscured* by later messages, i.e. it goes through the topic on a per-key basis and leaves only the most recent message associated with that key. @@ -15,7 +19,7 @@ Pulsar's topic compaction feature: * Allows for faster "rewind" through topic logs * Applies only to [persistent topics](concepts-architecture-overview.md#persistent-storage) -* Triggered automatically when the backlog reaches a certain size or can be triggered manually via the command line. See the [Topic compaction cookbook](cookbooks-compaction.md) +* Triggered automatically when the backlog reaches a certain size or can be triggered manually via the command line. See the [Topic compaction cookbook](cookbooks-compaction) * Is conceptually and operationally distinct from [retention and expiry](concepts-messaging.md#message-retention-and-expiry). Topic compaction *does*, however, respect retention. If retention has removed a message from the message backlog of a topic, the message will also not be readable from the compacted topic ledger. > #### Topic compaction example: the stock ticker @@ -24,7 +28,7 @@ Pulsar's topic compaction feature: ## How topic compaction works -When topic compaction is triggered [via the CLI](cookbooks-compaction.md), Pulsar will iterate over the entire topic from beginning to end. For each key that it encounters the compaction routine will keep a record of the latest occurrence of that key. +When topic compaction is triggered [via the CLI](cookbooks-compaction), Pulsar will iterate over the entire topic from beginning to end. For each key that it encounters the compaction routine will keep a record of the latest occurrence of that key. After that, the broker will create a new [BookKeeper ledger](concepts-architecture-overview.md#ledgers) and make a second iteration through each message on the topic. For each message, if the key matches the latest occurrence of that key, then the key's data payload, message ID, and metadata will be written to the newly created ledger. If the key doesn't match the latest then the message will be skipped and left alone. If any given message has an empty payload, it will be skipped and considered deleted (akin to the concept of [tombstones](https://en.wikipedia.org/wiki/Tombstone_(data_store)) in key-value databases). At the end of this second iteration through the topic, the newly created BookKeeper ledger is closed and two things are written to the topic's metadata: the ID of the BookKeeper ledger and the message ID of the last compacted message (this is known as the **compaction horizon** of the topic). Once this metadata is written compaction is complete. diff --git a/site2/website-next/versioned_docs/version-2.7.3/concepts-transactions.md b/site2/website-next/versioned_docs/version-2.7.3/concepts-transactions.md index 12a32e879f057..9e44d4f4e0258 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/concepts-transactions.md +++ b/site2/website-next/versioned_docs/version-2.7.3/concepts-transactions.md @@ -1,10 +1,14 @@ --- id: transactions title: Transactions -sidebar_label: Overview +sidebar_label: "Overview" original_id: transactions --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + Transactional semantics enable event streaming applications to consume, process, and produce messages in one atomic operation. In Pulsar, a producer or consumer can work with messages across multiple topics and partitions and ensure those messages are processed as a single unit. The following concepts help you understand Pulsar transactions. diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-compaction.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-compaction.md index 6a06829decefb..ebcf4ba3237a7 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-compaction.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-compaction.md @@ -1,7 +1,7 @@ --- id: cookbooks-compaction title: Topic compaction -sidebar_label: Topic compaction +sidebar_label: "Topic compaction" original_id: cookbooks-compaction --- @@ -143,3 +143,4 @@ Message msg = MessageBuilder.create() compactedTopicProducer.send(msg); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-deduplication.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-deduplication.md index 74165999ac8f9..f6dc69a52ab2c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-deduplication.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-deduplication.md @@ -1,7 +1,7 @@ --- id: cookbooks-deduplication title: Message deduplication -sidebar_label: Message deduplication +sidebar_label: "Message deduplication" original_id: cookbooks-deduplication --- @@ -128,6 +128,7 @@ producer = client.create_producer( send_timeout_millis=0) ``` +
    @@ -152,6 +153,7 @@ Producer producer; Result result = client.createProducer(topic, producerConfig, producer); ``` +
    diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-encryption.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-encryption.md index 67f4bf9b0f929..c782e69ddd047 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-encryption.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-encryption.md @@ -1,7 +1,7 @@ --- id: cookbooks-encryption title: Pulsar Encryption -sidebar_label: Encryption +sidebar_label: "Encryption" original_id: cookbooks-encryption --- @@ -39,6 +39,7 @@ openssl ecparam -name secp521r1 -genkey -param_enc explicit -out test_ecdsa_priv openssl ec -in test_ecdsa_privkey.pem -pubout -outform pkcs8 -out test_ecdsa_pubkey.pem ``` + 2. Add the public and private key to the key management and configure your producers to retrieve public keys and consumers clients to retrieve private keys. 3. Implement CryptoKeyReader::getPublicKey() interface from producer and CryptoKeyReader::getPrivateKey() interface from consumer, which will be invoked by Pulsar client to load the key. 4. Add encryption key to producer configuration: conf.addEncryptionKey("myapp.key") @@ -96,6 +97,7 @@ for (int i = 0; i < 10; i++) { pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -171,6 +173,7 @@ conf.addEncryptionKey("myapp.messagekey1"); conf.addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypting encrypted messages at the consumer application: Consumers require access one of the private keys to decrypt messages produced by the producer. If you would like to receive encrypted messages, create a public/private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-message-queue.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-message-queue.md index 7ec715310cb34..f30da44a1bbad 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-message-queue.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-message-queue.md @@ -1,7 +1,7 @@ --- id: cookbooks-message-queue title: Using Pulsar as a message queue -sidebar_label: Message queue +sidebar_label: "Message queue" original_id: cookbooks-message-queue --- @@ -127,3 +127,4 @@ if err != nil { } ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-non-persistent.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-non-persistent.md index 8c5f0046f2524..f10d8374041ac 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-non-persistent.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-non-persistent.md @@ -1,7 +1,7 @@ --- id: cookbooks-non-persistent title: Non-persistent messaging -sidebar_label: Non-persistent messaging +sidebar_label: "Non-persistent messaging" original_id: cookbooks-non-persistent --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-partitioned.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-partitioned.md index d5609816b4977..9442a1ef2c32a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-partitioned.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-partitioned.md @@ -1,7 +1,7 @@ --- id: cookbooks-partitioned title: Partitioned topics -sidebar_label: Partitioned Topics +sidebar_label: "Partitioned Topics" original_id: cookbooks-partitioned --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-retention-expiry.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-retention-expiry.md index f0b25d58ea61d..60441a3f28f66 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-retention-expiry.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-retention-expiry.md @@ -1,7 +1,7 @@ --- id: cookbooks-retention-expiry title: Message retention and expiry -sidebar_label: Message retention and expiry +sidebar_label: "Message retention and expiry" original_id: cookbooks-retention-expiry --- @@ -79,6 +79,7 @@ You can set a retention policy for a namespace by specifying the namespace, a si } ]}> + You can use the [`set-retention`](reference-pulsar-admin.md#namespaces-set-retention) subcommand and specify a namespace, a size limit using the `-s`/`--size` flag, and a time limit using the `-t`/`--time` flag. In the following example, the size limit is set to 10 GB and the time limit is set to 3 hours for each topic within the `my-tenant/my-ns` namespace. @@ -407,6 +408,7 @@ $ pulsar-admin namespaces remove-message-ttl my-tenant/my-ns admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + ## Delete messages from namespaces If you do not have any retention period and that you never have much of a backlog, the upper limit for retaining messages, which are acknowledged, equals to the Pulsar segment rollover period + entry log rollover period + (garbage collection interval * garbage collection ratios). diff --git a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md index 0517884a7ba73..0f2052fb44d4d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md +++ b/site2/website-next/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md @@ -1,7 +1,7 @@ --- id: cookbooks-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" original_id: cookbooks-tiered-storage --- @@ -11,11 +11,9 @@ import TabItem from '@theme/TabItem'; Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -55,7 +53,7 @@ Currently we support driver of types: > though it requires that you specify an endpoint url using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if > using a S3 compatible data store, other than AWS. -```conf +``` managedLedgerOffloadDriver=aws-s3 @@ -70,7 +68,7 @@ Everything that you store in Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` s3ManagedLedgerOffloadBucket=pulsar-topic-offload @@ -81,7 +79,7 @@ but a recommended configuration. If it is not configured, It will use the defaul With AWS S3, the default region is `US East (N. Virginia)`. Page [AWS Regions and Endpoints](https://docs.aws.amazon.com/general/latest/gr/rande.html) contains more information. -```conf +``` s3ManagedLedgerOffloadRegion=eu-west-3 @@ -100,7 +98,7 @@ Once you have created a set of credentials in the AWS IAM console, they can be c If you are on AWS instance with an instance profile that provides credentials, Pulsar will use these credentials if no other mechanism is provided -2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ```conf/pulsar_env.sh```. +2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ``` ```bash @@ -122,7 +120,9 @@ PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessK 4. Set the access credentials in ```~/.aws/credentials```. -```conf [default] +``` + +[default] aws_access_key_id=ABC123456789 aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c @@ -132,7 +132,7 @@ aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c If you want to assume an IAM role, this can be done via specifying the following: -```conf +``` s3ManagedLedgerOffloadRole= s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload @@ -147,7 +147,8 @@ This will use the `DefaultAWSCredentialsProviderChain` for assuming this role. Pulsar also provides some knobs to configure the size of requests sent to AWS S3. -- ```s3ManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of +- ``` + a "part" sent during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```s3ManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from AWS S3. Default is 1MB. @@ -160,7 +161,7 @@ Buckets are the basic containers that hold your data. Everything that you store Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` gcsManagedLedgerOffloadBucket=pulsar-topic-offload @@ -172,7 +173,7 @@ a recommended configuration. If it is not configured, It will use the default re Regarding GCS, buckets are default created in the `us multi-regional location`, page [Bucket Locations](https://cloud.google.com/storage/docs/bucket-locations) contains more information. -```conf +``` gcsManagedLedgerOffloadRegion=europe-west3 @@ -197,7 +198,7 @@ To generate service account credentials or view the public credentials that you' > Notes: Make ensure that the service account you create has permission to operate GCS, you need to assign **Storage Admin** permission to your service account in [here](https://cloud.google.com/storage/docs/access-control/iam). -```conf +``` gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804d5e6a6f33.json" @@ -207,7 +208,8 @@ gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804 Pulsar also provides some knobs to configure the size of requests sent to GCS. -- ```gcsManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of a "part" sent +- ``` + during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```gcsManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from GCS. Default is 1MB. @@ -221,16 +223,17 @@ In both cases, these should not be touched unless you know what you are doing. You can configure the connection address in the `broker.conf` file. -```conf +``` fileSystemURI="hdfs://127.0.0.1:9000" ``` + #### Configure Hadoop profile path The configuration file is stored in the Hadoop profile path. It contains various settings, such as base path, authentication, and so on. -```conf +``` fileSystemProfilePath="../conf/filesystem_offload_core_site.xml" @@ -240,7 +243,7 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us **Example** -```conf +``` fs.defaultFS @@ -271,7 +274,6 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us io.map.index.interval 128 - ``` @@ -280,7 +282,7 @@ For more information about the configurations in `org.apache.hadoop.io.MapFile`, Namespace policies can be configured to offload data automatically once a threshold is reached. The threshold is based on the size of data that the topic has stored on the pulsar cluster. Once the topic reaches the threshold, an offload operation will be triggered. Setting a negative value to the threshold will disable automatic offloading. Setting the threshold to 0 will cause the broker to offload data as soon as it possiby can. -```bash +``` $ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace @@ -295,7 +297,7 @@ Offloading can manually triggered through a REST endpoint on the Pulsar broker. When triggering offload, you must specify the maximum size, in bytes, of backlog which will be retained locally on the bookkeeper. The offload mechanism will offload segments from the start of the topic backlog until this condition is met. -```bash +``` $ bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 @@ -304,7 +306,7 @@ Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages be The command to triggers an offload will not wait until the offload operation has completed. To check the status of the offload, use offload-status. -```bash +``` $ bin/pulsar-admin topics offload-status my-tenant/my-namespace/topic1 Offload is currently running @@ -313,7 +315,7 @@ Offload is currently running To wait for offload to complete, add the -w flag. -```bash +``` $ bin/pulsar-admin topics offload-status -w my-tenant/my-namespace/topic1 Offload was a success @@ -322,7 +324,7 @@ Offload was a success If there is an error offloading, the error will be propagated to the offload-status command. -```bash +``` $ bin/pulsar-admin topics offload-status persistent://public/default/topic1 Error in offload @@ -330,5 +332,5 @@ null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-aws.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-aws.md index b554ab0885721..78defa1e3e5ec 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-aws.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-aws.md @@ -1,7 +1,7 @@ --- id: deploy-aws title: Deploying a Pulsar cluster on AWS using Terraform and Ansible -sidebar_label: Amazon Web Services +sidebar_label: "Amazon Web Services" original_id: deploy-aws --- @@ -9,7 +9,7 @@ import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -> For instructions on deploying a single Pulsar cluster manually rather than using Terraform and Ansible, see [Deploying a Pulsar cluster on bare metal](deploy-bare-metal.md). For instructions on manually deploying a multi-cluster Pulsar instance, see [Deploying a Pulsar instance on bare metal](deploy-bare-metal-multi-cluster.md). +> For instructions on deploying a single Pulsar cluster manually rather than using Terraform and Ansible, see [Deploying a Pulsar cluster on bare metal](deploy-bare-metal.md). For instructions on manually deploying a multi-cluster Pulsar instance, see [Deploying a Pulsar instance on bare metal](deploy-bare-metal-multi-cluster). One of the easiest ways to get a Pulsar [cluster](reference-terminology.md#cluster) running on [Amazon Web Services](https://aws.amazon.com/) (AWS) is to use the [Terraform](https://terraform.io) infrastructure provisioning tool and the [Ansible](https://www.ansible.com) server automation tool. Terraform can create the resources necessary for running the Pulsar cluster---[EC2](https://aws.amazon.com/ec2/) instances, networking and security infrastructure, etc.---While Ansible can install and run Pulsar on the provisioned resources. @@ -24,7 +24,9 @@ In order to install a Pulsar cluster on AWS using Terraform and Ansible, you nee You also need to make sure that you are currently logged into your AWS account via the `aws` tool: ```bash + $ aws configure + ``` ## Installation @@ -32,7 +34,9 @@ $ aws configure You can install Ansible on Linux or macOS using pip. ```bash + $ pip install ansible + ``` You can install Terraform using the instructions [here](https://www.terraform.io/intro/getting-started/install.html). @@ -40,8 +44,10 @@ You can install Terraform using the instructions [here](https://www.terraform.io You also need to have the Terraform and Ansible configuration for Pulsar locally on your machine. You can find them in the [GitHub repository](https://github.com/apache/pulsar) of Pulsar, which you can fetch using Git commands: ```bash + $ git clone https://github.com/apache/pulsar $ cd pulsar/deployment/terraform-ansible/aws + ``` ## SSH setup @@ -54,27 +60,40 @@ $ cd pulsar/deployment/terraform-ansible/aws > > 1. update `ansible.cfg` with following values: > + > ```shell +> > private_key_file=~/.ssh/pulsar_aws +> +> > ``` + > > 2. update `terraform.tfvars` with following values: > + > ```shell +> > public_key_path=~/.ssh/pulsar_aws.pub +> +> > ``` In order to create the necessary AWS resources using Terraform, you need to create an SSH key. Enter the following commands to create a private SSH key in `~/.ssh/id_rsa` and a public key in `~/.ssh/id_rsa.pub`: ```bash + $ ssh-keygen -t rsa + ``` Do *not* enter a passphrase (hit **Enter** instead when the prompt comes out). Enter the following command to verify that a key has been created: ```bash + $ ls ~/.ssh id_rsa id_rsa.pub + ``` ## Create AWS resources using Terraform @@ -82,24 +101,30 @@ id_rsa id_rsa.pub To start building AWS resources with Terraform, you need to install all Terraform dependencies. Enter the following command: ```bash + $ terraform init # This will create a .terraform folder + ``` After that, you can apply the default Terraform configuration by entering this command: ```bash + $ terraform apply + ``` Then you see this prompt below: ```bash + Do you want to perform these actions? Terraform will perform the actions described above. Only 'yes' will be accepted to approve. Enter a value: + ``` Type `yes` and hit **Enter**. Applying the configuration could take several minutes. When the configuration applying finishes, you can see `Apply complete!` along with some other information, including the number of resources created. @@ -143,13 +168,17 @@ All EC2 instances for the cluster run in the [us-west-2](http://docs.aws.amazon. When you apply the Terraform configuration by entering the command `terraform apply`, Terraform outputs a value for the `pulsar_service_url`. The value should look something like this: ``` + pulsar://pulsar-elb-1800761694.us-west-2.elb.amazonaws.com:6650 + ``` You can fetch that value at any time by entering the command `terraform output pulsar_service_url` or parsing the `terraform.tstate` file (which is JSON, even though the filename does not reflect that): ```bash + $ cat terraform.tfstate | jq .modules[0].outputs.pulsar_service_url.value + ``` ### Destroy your cluster @@ -157,7 +186,9 @@ $ cat terraform.tfstate | jq .modules[0].outputs.pulsar_service_url.value At any point, you can destroy all AWS resources associated with your cluster using Terraform's `destroy` command: ```bash + $ terraform destroy + ``` ## Setup Disks @@ -167,10 +198,12 @@ Before you run the Pulsar playbook, you need to mount the disks to the correct d To setup disks on bookie nodes, enter this command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ setup-disk.yaml + ``` After that, the disks is mounted under `/mnt/journal` as journal disk, and `/mnt/storage` as ledger disk. @@ -180,25 +213,29 @@ Remember to enter this command just only once. If you attempt to enter this comm Once you have created the necessary AWS resources using Terraform, you can install and run Pulsar on the Terraform-created EC2 instances using Ansible. -(Optional) If you want to use any [built-in IO connectors](io-connectors.md) , edit the `Download Pulsar IO packages` task in the `deploy-pulsar.yaml` file and uncomment the connectors you want to use. +(Optional) If you want to use any [built-in IO connectors](io-connectors) , edit the `Download Pulsar IO packages` task in the `deploy-pulsar.yaml` file and uncomment the connectors you want to use. To run the playbook, enter this command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ ../deploy-pulsar.yaml + ``` If you have created a private SSH key at a location different from `~/.ssh/id_rsa`, you can specify the different location using the `--private-key` flag in the following command: ```bash + $ ansible-playbook \ --user='ec2-user' \ --inventory=`which terraform-inventory` \ --private-key="~/.ssh/some-non-default-key" \ ../deploy-pulsar.yaml + ``` ## Access the cluster @@ -208,24 +245,30 @@ You can now access your running Pulsar using the unique Pulsar connection URL fo For a quick demonstration of accessing the cluster, we can use the Python client for Pulsar and the Python shell. First, install the Pulsar Python module using pip: ```bash + $ pip install pulsar-client + ``` Now, open up the Python shell using the `python` command: ```bash + $ python + ``` Once you are in the shell, enter the following command: ```python + >>> import pulsar >>> client = pulsar.Client('pulsar://pulsar-elb-1800761694.us-west-2.elb.amazonaws.com:6650') # Make sure to use your connection URL >>> producer = client.create_producer('persistent://public/default/test-topic') >>> producer.send('Hello world') >>> client.close() + ``` If all of these commands are successful, Pulsar clients can now use your cluster! diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal-multi-cluster.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal-multi-cluster.md index 5a17cda056692..a6d99b11553ce 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal-multi-cluster.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal-multi-cluster.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal-multi-cluster title: Deploying a multi-cluster on bare metal -sidebar_label: Bare metal multi-cluster +sidebar_label: "Bare metal multi-cluster" original_id: deploy-bare-metal-multi-cluster --- @@ -13,17 +13,17 @@ import TabItem from '@theme/TabItem'; > > 1. Single-cluster Pulsar installations should be sufficient for all but the most ambitious use cases. If you are interested in experimenting with > Pulsar or using it in a startup or on a single team, you had better opt for a single cluster. For instructions on deploying a single cluster, -> see the guide [here](deploy-bare-metal.md). +> see the guide [here](deploy-bare-metal). > -> 2. If you want to use all builtin [Pulsar IO](io-overview.md) connectors in your Pulsar deployment, you need to download `apache-pulsar-io-connectors` +> 2. If you want to use all builtin [Pulsar IO](io-overview) connectors in your Pulsar deployment, you need to download `apache-pulsar-io-connectors` > package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you -> run a separate cluster of function workers for [Pulsar Functions](functions-overview.md). +> run a separate cluster of function workers for [Pulsar Functions](functions-overview). > -> 3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders` +> 3. If you want to use [Tiered Storage](concepts-tiered-storage) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders` > package and install `apache-pulsar-offloaders` under `offloaders` directory in the pulsar directory on every broker node. For more details of how to configure -> this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md). +> this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage). -A Pulsar *instance* consists of multiple Pulsar clusters working in unison. You can distribute clusters across data centers or geographical regions and replicate the clusters amongst themselves using [geo-replication](administration-geo.md). Deploying a multi-cluster Pulsar instance involves the following basic steps: +A Pulsar *instance* consists of multiple Pulsar clusters working in unison. You can distribute clusters across data centers or geographical regions and replicate the clusters amongst themselves using [geo-replication](administration-geo). Deploying a multi-cluster Pulsar instance involves the following basic steps: * Deploying two separate [ZooKeeper](#deploy-zookeeper) quorums: a [local](#deploy-local-zookeeper) quorum for each cluster in the instance and a [configuration store](#configuration-store) quorum for instance-wide tasks * Initializing [cluster metadata](#cluster-metadata-initialization) for each cluster @@ -33,7 +33,7 @@ A Pulsar *instance* consists of multiple Pulsar clusters working in unison. You If you want to deploy a single Pulsar cluster, see [Clusters and Brokers](getting-started-standalone.md#start-the-cluster). > #### Run Pulsar locally or on Kubernetes? -> This guide shows you how to deploy Pulsar in production in a non-Kubernetes environment. If you want to run a standalone Pulsar cluster on a single machine for development purposes, see the [Setting up a local cluster](getting-started-standalone.md) guide. If you want to run Pulsar on [Kubernetes](https://kubernetes.io), see the [Pulsar on Kubernetes](deploy-kubernetes.md) guide, which includes sections on running Pulsar on Kubernetes on [Google Kubernetes Engine](deploy-kubernetes#pulsar-on-google-kubernetes-engine) and on [Amazon Web Services](deploy-kubernetes#pulsar-on-amazon-web-services). +> This guide shows you how to deploy Pulsar in production in a non-Kubernetes environment. If you want to run a standalone Pulsar cluster on a single machine for development purposes, see the [Setting up a local cluster](getting-started-standalone.md) guide. If you want to run Pulsar on [Kubernetes](https://kubernetes.io), see the [Pulsar on Kubernetes](deploy-kubernetes) guide, which includes sections on running Pulsar on Kubernetes on [Google Kubernetes Engine](deploy-kubernetes#pulsar-on-google-kubernetes-engine) and on [Amazon Web Services](deploy-kubernetes#pulsar-on-amazon-web-services). ## System requirement Pulsar is currently available for **MacOS** and **Linux**. In order to use Pulsar, you need to install Java 8 from [Oracle download center](http://www.oracle.com/). @@ -51,14 +51,18 @@ To get started running Pulsar, download a binary tarball release in one of the f * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget 'https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=pulsar/pulsar-@pulsar:version@/apache-pulsar-@pulsar:version@-bin.tar.gz' -O apache-pulsar-@pulsar:version@-bin.tar.gz + ``` Once you download the tarball, untar it and `cd` into the resulting directory: ```bash + $ tar xvfz apache-pulsar-@pulsar:version@-bin.tar.gz $ cd apache-pulsar-@pulsar:version@ + ``` ## What your package contains @@ -67,9 +71,9 @@ The Pulsar binary package initially contains the following directories: Directory | Contains :---------|:-------- -`bin` | [Command-line tools](reference-cli-tools.md) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) +`bin` | [Command-line tools](reference-cli-tools) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) `conf` | Configuration files for Pulsar, including for [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more -`examples` | A Java JAR file containing example [Pulsar Functions](functions-overview.md) +`examples` | A Java JAR file containing example [Pulsar Functions](functions-overview) `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files that Pulsar uses `licenses` | License files, in `.txt` form, for various components of the Pulsar codebase @@ -78,7 +82,7 @@ The following directories are created once you begin running Pulsar: Directory | Contains :---------|:-------- `data` | The data storage directory that ZooKeeper and BookKeeper use -`instances` | Artifacts created for [Pulsar Functions](functions-overview.md) +`instances` | Artifacts created for [Pulsar Functions](functions-overview) `logs` | Logs that the installation creates @@ -101,9 +105,11 @@ You need to stand up one local ZooKeeper cluster *per Pulsar cluster* for deploy To begin, add all ZooKeeper servers to the quorum configuration specified in the [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) file. Add a `server.N` line for each node in the cluster to the configuration, where `N` is the number of the ZooKeeper node. The following is an example for a three-node cluster: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` On each host, you need to specify the ID of the node in the `myid` file of each node, which is in `data/zookeeper` folder of each server by default (you can change the file location via the [`dataDir`](reference-configuration.md#zookeeper-dataDir) parameter). @@ -113,8 +119,10 @@ On each host, you need to specify the ID of the node in the `myid` file of each On a ZooKeeper server at `zk1.us-west.example.com`, for example, you could set the `myid` value like this: ```shell + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com` the command looks like `echo 2 > data/zookeeper/myid` and so on. @@ -122,7 +130,9 @@ On `zk2.us-west.example.com` the command looks like `echo 2 > data/zookeeper/myi Once you add each server to the `zookeeper.conf` configuration and each server has the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start zookeeper + ``` ### Deploy the configuration store @@ -135,13 +145,15 @@ If you deploy a [single-cluster](#single-cluster-pulsar-instance) instance, you If your Pulsar instance consists of just one cluster, then you can deploy a configuration store on the same machines as the local ZooKeeper quorum but run on different TCP ports. -To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorom uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: +To deploy a ZooKeeper configuration store in a single-cluster instance, add the same ZooKeeper servers that the local quorum uses to the configuration file in [`conf/global_zookeeper.conf`](reference-configuration.md#configuration-store) using the same method for [local ZooKeeper](#local-zookeeper), but make sure to use a different port (2181 is the default for ZooKeeper). The following is an example that uses port 2184 for a three-node ZooKeeper cluster: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 server.3=zk3.us-west.example.com:2185:2186 + ``` As before, create the `myid` files for each server on `data/global-zookeeper/myid`. @@ -159,7 +171,9 @@ For example, assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also assume, each cluster has its own local ZK servers named such as the following: ``` + zk[1-3].${CLUSTER}.example.com + ``` In this scenario if you want to pick the quorum participants from few clusters and @@ -170,6 +184,7 @@ This method guarantees that writes to configuration store is possible even if on The ZK configuration in all the servers looks like: ```properties + clientPort=2184 server.1=zk1.us-west.example.com:2185:2186 server.2=zk2.us-west.example.com:2185:2186 @@ -186,12 +201,15 @@ server.12=zk3.eu-central.example.com:2185:2186:observer server.13=zk1.ap-south.example.com:2185:2186:observer server.14=zk2.ap-south.example.com:2185:2186:observer server.15=zk3.ap-south.example.com:2185:2186:observer + ``` Additionally, ZK observers need to have the following parameters: ```properties + peerType=observer + ``` ##### Start the service @@ -199,7 +217,9 @@ peerType=observer Once your configuration store configuration is in place, you can start up the service using [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) ```shell + $ bin/pulsar-daemon start configuration-store + ``` ## Cluster metadata initialization @@ -209,6 +229,7 @@ Once you set up the cluster-specific ZooKeeper and configuration store quorums f You can initialize this metadata using the [`initialize-cluster-metadata`](reference-cli-tools.md#pulsar-initialize-cluster-metadata) command of the [`pulsar`](reference-cli-tools.md#pulsar) CLI tool. The following is an example: ```shell + $ bin/pulsar initialize-cluster-metadata \ --cluster us-west \ --zookeeper zk1.us-west.example.com:2181 \ @@ -217,6 +238,7 @@ $ bin/pulsar initialize-cluster-metadata \ --web-service-url-tls https://pulsar.us-west.example.com:8443/ \ --broker-service-url pulsar://pulsar.us-west.example.com:6650/ \ --broker-service-url-tls pulsar+ssl://pulsar.us-west.example.com:6651/ + ``` As you can see from the example above, you need to specify the following: @@ -227,7 +249,7 @@ As you can see from the example above, you need to specify the following: * The web service URL for the cluster * A broker service URL enabling interaction with the [brokers](reference-terminology.md#broker) in the cluster -If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. +If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. Make sure to run `initialize-cluster-metadata` for each cluster in your instance. @@ -248,12 +270,17 @@ You can start a bookie in two ways: in the foreground or as a background daemon. To start a bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` You can verify that the bookie works properly using the `bookiesanity` command for the [BookKeeper shell](reference-cli-tools.md#bookkeeper-shell): + ```shell + $ bin/bookkeeper shell bookiesanity + ``` This command creates a new ledger on the local bookie, writes a few entries, reads them back and finally deletes the ledger. @@ -261,7 +288,9 @@ This command creates a new ledger on the local bookie, writes a few entries, rea After you have started all bookies, you can use the `simpletest` command for [BookKeeper shell](reference-cli-tools.md#shell) on any bookie node, to verify that all bookies in the cluster are running. ```bash + $ bin/bookkeeper shell simpletest --ensemble --writeQuorum --ackQuorum --numEntries + ``` Bookie hosts are responsible for storing message data on disk. In order for bookies to provide optimal performance, having a suitable hardware configuration is essential for the bookies. The following are key dimensions for bookie hardware capacity. @@ -292,6 +321,7 @@ You also need to specify the name of the [cluster](reference-terminology.md#clus The following is an example configuration: ```properties + # Local ZooKeeper servers zookeeperServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 @@ -311,6 +341,7 @@ webServicePort=8080 # Port to use to server HTTPS request webServicePortTls=8443 + ``` ### Broker hardware @@ -322,20 +353,24 @@ Pulsar brokers do not require any special hardware since they do not use the loc You can start a broker in the background by using [nohup](https://en.wikipedia.org/wiki/Nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```shell + $ bin/pulsar-daemon start broker + ``` You can also start brokers in the foreground by using [`pulsar broker`](reference-cli-tools.md#broker): ```shell + $ bin/pulsar broker + ``` ## Service discovery -[Clients](getting-started-clients.md) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. Pulsar provides a built-in service discovery mechanism that you can set up using the instructions [immediately below](#service-discovery-setup). +[Clients](getting-started-clients) connecting to Pulsar brokers need to be able to communicate with an entire Pulsar instance using a single URL. Pulsar provides a built-in service discovery mechanism that you can set up using the instructions [immediately below](#service-discovery-setup). -You can also use your own service discovery system if you want. If you use your own system, you only need to satisfy just one requirement: when a client performs an HTTP request to an [endpoint](reference-configuration.md) for a Pulsar cluster, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to *some* active broker in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. +You can also use your own service discovery system if you want. If you use your own system, you only need to satisfy just one requirement: when a client performs an HTTP request to an [endpoint](reference-configuration) for a Pulsar cluster, such as `http://pulsar.us-west.example.com:8080`, the client needs to be redirected to *some* active broker in the desired cluster, whether via DNS, an HTTP or IP redirect, or some other means. > #### Service discovery already provided by many scheduling systems > Many large-scale deployment systems, such as [Kubernetes](deploy-kubernetes), have service discovery systems built in. If you run Pulsar on such a system, you may not need to provide your own service discovery mechanism. @@ -343,33 +378,39 @@ You can also use your own service discovery system if you want. If you use your ### Service discovery setup -The service discovery mechanism that included with Pulsar maintains a list of active brokers, which stored in ZooKeeper, and supports lookup using HTTP and also the [binary protocol](developing-binary-protocol.md) of Pulsar. +The service discovery mechanism that included with Pulsar maintains a list of active brokers, which stored in ZooKeeper, and supports lookup using HTTP and also the [binary protocol](developing-binary-protocol) of Pulsar. To get started setting up the built-in service of discovery of Pulsar, you need to change a few parameters in the [`conf/discovery.conf`](reference-configuration.md#service-discovery) configuration file. Set the [`zookeeperServers`](reference-configuration.md#service-discovery-zookeeperServers) parameter to the ZooKeeper quorum connection string of the cluster and the [`configurationStoreServers`](reference-configuration.md#service-discovery-configurationStoreServers) setting to the [configuration store](reference-terminology.md#configuration-store) quorum connection string. ```properties + # Zookeeper quorum connection string zookeeperServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 # Global configuration store connection string configurationStoreServers=zk1.us-west.example.com:2184,zk2.us-west.example.com:2184,zk3.us-west.example.com:2184 + ``` To start the discovery service: ```shell + $ bin/pulsar-daemon start discovery + ``` ## Admin client and verification -At this point your Pulsar instance should be ready to use. You can now configure client machines that can serve as [administrative clients](admin-api-overview.md) for each cluster. You can use the [`conf/client.conf`](reference-configuration.md#client) configuration file to configure admin clients. +At this point your Pulsar instance should be ready to use. You can now configure client machines that can serve as [administrative clients](admin-api-overview) for each cluster. You can use the [`conf/client.conf`](reference-configuration.md#client) configuration file to configure admin clients. The most important thing is that you point the [`serviceUrl`](reference-configuration.md#client-serviceUrl) parameter to the correct service URL for the cluster: ```properties + serviceUrl=http://pulsar.us-west.example.com:8080/ + ``` ## Provision new tenants @@ -380,9 +421,11 @@ Pulsar is built as a fundamentally multi-tenant system. If a new tenant wants to use the system, you need to create a new one. You can create a new tenant by using the [`pulsar-admin`](reference-pulsar-admin.md#tenants) CLI tool: ```shell + $ bin/pulsar-admin tenants create test-tenant \ --allowed-clusters us-west \ --admin-roles test-admin-role + ``` In this command, users who identify with `test-admin-role` role can administer the configuration for the `test-tenant` tenant. The `test-tenant` tenant can only use the `us-west` cluster. From now on, this tenant can manage its resources. @@ -393,7 +436,9 @@ Once you create a tenant, you need to create [namespaces](reference-terminology. The first step is to create a namespace. A namespace is an administrative unit that can contain many topics. A common practice is to create a namespace for each different use case from a single tenant. ```shell + $ bin/pulsar-admin namespaces create test-tenant/ns1 + ``` ##### Test producer and consumer @@ -407,23 +452,32 @@ You can use a topic in the namespace that you have just created. Topics are auto The topic name in this case could be: ```http + persistent://test-tenant/ns1/my-topic + ``` Start a consumer that creates a subscription on the topic and waits for messages: ```shell + $ bin/pulsar-perf consume persistent://test-tenant/ns1/my-topic + ``` Start a producer that publishes messages at a fixed rate and reports stats every 10 seconds: ```shell + $ bin/pulsar-perf produce persistent://test-tenant/ns1/my-topic + ``` To report the topic stats: ```shell + $ bin/pulsar-admin topics stats persistent://test-tenant/ns1/my-topic + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal.md index 3d3599cba945a..3529b4e220572 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-bare-metal.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal title: Deploy a cluster on bare metal -sidebar_label: Bare metal +sidebar_label: "Bare metal" original_id: deploy-bare-metal --- @@ -14,15 +14,15 @@ import TabItem from '@theme/TabItem'; > > 1. Single-cluster Pulsar installations should be sufficient for all but the most ambitious use cases. If you are interested in experimenting with > Pulsar or using Pulsar in a startup or on a single team, it is simplest to opt for a single cluster. If you do need to run a multi-cluster Pulsar instance, -> see the guide [here](deploy-bare-metal-multi-cluster.md). +> see the guide [here](deploy-bare-metal-multi-cluster). > -> 2. If you want to use all builtin [Pulsar IO](io-overview.md) connectors in your Pulsar deployment, you need to download `apache-pulsar-io-connectors` +> 2. If you want to use all builtin [Pulsar IO](io-overview) connectors in your Pulsar deployment, you need to download `apache-pulsar-io-connectors` > package and install `apache-pulsar-io-connectors` under `connectors` directory in the pulsar directory on every broker node or on every function-worker node if you -> have run a separate cluster of function workers for [Pulsar Functions](functions-overview.md). +> have run a separate cluster of function workers for [Pulsar Functions](functions-overview). > -> 3. If you want to use [Tiered Storage](concepts-tiered-storage.md) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders` +> 3. If you want to use [Tiered Storage](concepts-tiered-storage) feature in your Pulsar deployment, you need to download `apache-pulsar-offloaders` > package and install `apache-pulsar-offloaders` under `offloaders` directory in the pulsar directory on every broker node. For more details of how to configure -> this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md). +> this feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage). Deploying a Pulsar cluster involves doing the following (in order): @@ -86,21 +86,25 @@ To get started deploying a Pulsar cluster on bare metal, you need to download a * Using [wget](https://www.gnu.org/software/wget): ```bash + $ wget pulsar:binary_release_url + ``` Once you download the tarball, untar it and `cd` into the resulting directory: ```bash + $ tar xvzf apache-pulsar-@pulsar:version@-bin.tar.gz $ cd apache-pulsar-@pulsar:version@ + ``` The extracted directory contains the following subdirectories: Directory | Contains :---------|:-------- -`bin` |[command-line tools](reference-cli-tools.md) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) +`bin` |[command-line tools](reference-cli-tools) of Pulsar, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/) `conf` | Configuration files for Pulsar, including for [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more `data` | The data storage directory that ZooKeeper and BookKeeper use `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files that Pulsar uses @@ -123,19 +127,23 @@ To get started using builtin connectors, you need to download the connectors tar * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar + ``` Once you download the .nar file, copy the file to directory `connectors` in the pulsar directory. For example, if you download the connector file `pulsar-io-aerospike-@pulsar:version@.nar`: ```bash + $ mkdir connectors $ mv pulsar-io-aerospike-@pulsar:version@.nar connectors $ ls connectors pulsar-io-aerospike-@pulsar:version@.nar ... + ``` ## [Install Tiered Storage Offloaders (optional)](https://pulsar.apache.org/docs/en/next/standalone/#install-tiered-storage-offloaders-optional) @@ -155,12 +163,15 @@ To get started using tiered storage offloaders, you need to download the offload * using [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:offloader_release_url + ``` Once you download the tarball, in the pulsar directory, untar the offloaders package and copy the offloaders as `offloaders` in the pulsar directory: ```bash + $ tar xvfz apache-pulsar-offloaders-@pulsar:version@-bin.tar.gz // you can find a directory named `apache-pulsar-offloaders-@pulsar:version@` in the pulsar directory @@ -170,9 +181,10 @@ $ mv apache-pulsar-offloaders-@pulsar:version@/offloaders offloaders $ ls offloaders tiered-storage-jcloud-@pulsar:version@.nar + ``` -For more details of how to configure tiered storage feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage.md) +For more details of how to configure tiered storage feature, you can refer to the [Tiered storage cookbook](cookbooks-tiered-storage) ## Deploy a ZooKeeper cluster @@ -184,9 +196,11 @@ For more details of how to configure tiered storage feature, you can refer to th To begin, add all ZooKeeper servers to the configuration specified in [`conf/zookeeper.conf`](reference-configuration.md#zookeeper) (in the Pulsar directory that you create [above](#install-the-pulsar-binary-package)). The following is an example: ```properties + server.1=zk1.us-west.example.com:2888:3888 server.2=zk2.us-west.example.com:2888:3888 server.3=zk3.us-west.example.com:2888:3888 + ``` > If you only have one machine on which to deploy Pulsar, you only need to add one server entry in the configuration file. @@ -198,8 +212,10 @@ On each host, you need to specify the ID of the node in the `myid` file, which i For example, on a ZooKeeper server like `zk1.us-west.example.com`, you can set the `myid` value as follows: ```bash + $ mkdir -p data/zookeeper $ echo 1 > data/zookeeper/myid + ``` On `zk2.us-west.example.com`, the command is `echo 2 > data/zookeeper/myid` and so on. @@ -207,7 +223,9 @@ On `zk2.us-west.example.com`, the command is `echo 2 > data/zookeeper/myid` and Once you add each server to the `zookeeper.conf` configuration and have the appropriate `myid` entry, you can start ZooKeeper on all hosts (in the background, using nohup) with the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start zookeeper + ``` > If you plan to deploy Zookeeper with the Bookie on the same node, you @@ -216,7 +234,9 @@ $ bin/pulsar-daemon start zookeeper Start zookeeper with [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool like: ```bash + $ PULSAR_EXTRA_OPTS="-Dstats_server_port=8001" bin/pulsar-daemon start zookeeper + ``` ## Initialize cluster metadata @@ -226,6 +246,7 @@ Once you deploy ZooKeeper for your cluster, you need to write some metadata to Z You can initialize this metadata using the [`initialize-cluster-metadata`](reference-cli-tools.md#pulsar-initialize-cluster-metadata) command of the [`pulsar`](reference-cli-tools.md#pulsar) CLI tool. This command can be run on any machine in your ZooKeeper cluster. The following is an example: ```shell + $ bin/pulsar initialize-cluster-metadata \ --cluster pulsar-cluster-1 \ --zookeeper zk1.us-west.example.com:2181 \ @@ -234,6 +255,7 @@ $ bin/pulsar initialize-cluster-metadata \ --web-service-url-tls https://pulsar.us-west.example.com:8443 \ --broker-service-url pulsar://pulsar.us-west.example.com:6650 \ --broker-service-url-tls pulsar+ssl://pulsar.us-west.example.com:6651 + ``` As you can see from the example above, you will need to specify the following: @@ -244,29 +266,39 @@ Flag | Description `--zookeeper` | A "local" ZooKeeper connection string for the cluster. This connection string only needs to include *one* machine in the ZooKeeper cluster. `--configuration-store` | The configuration store connection string for the entire instance. As with the `--zookeeper` flag, this connection string only needs to include *one* machine in the ZooKeeper cluster. `--web-service-url` | The web service URL for the cluster, plus a port. This URL should be a standard DNS name. The default port is 8080 (you had better not use a different port). -`--web-service-url-tls` | If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster. The default port is 8443 (you had better not use a different port). +`--web-service-url-tls` | If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster. The default port is 8443 (you had better not use a different port). `--broker-service-url` | A broker service URL enabling interaction with the brokers in the cluster. This URL should not use the same DNS name as the web service URL but should use the `pulsar` scheme instead. The default port is 6650 (you had better not use a different port). -`--broker-service-url-tls` | If you use [TLS](security-tls-transport.md), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. The default port is 6651 (you had better not use a different port). +`--broker-service-url-tls` | If you use [TLS](security-tls-transport), you also need to specify a TLS web service URL for the cluster as well as a TLS broker service URL for the brokers in the cluster. The default port is 6651 (you had better not use a different port). > If you do not have a DNS server, you can use multi-host format in the service URL with the following settings: > + > ```properties +> > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` +> > If you want to use an existing BookKeeper cluster, you can add the `--existing-bk-metadata-service-uri` flag as follows: > + > ```properties +> > --existing-bk-metadata-service-uri "zk+null://zk1:2181;zk2:2181/ledgers" \ > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` + > You can obtain the metadata service URI of the existing BookKeeper cluster by using the `bin/bookkeeper shell whatisinstanceid` command. You must enclose the value in double quotes since the multiple metadata service URIs are separated with semicolons. ## Deploy a BookKeeper cluster @@ -276,7 +308,9 @@ Flag | Description You can configure BookKeeper bookies using the [`conf/bookkeeper.conf`](reference-configuration.md#bookkeeper) configuration file. The most important step in configuring bookies for our purposes here is ensuring that [`zkServers`](reference-configuration.md#bookkeeper-zkServers) is set to the connection string for the ZooKeeper cluster. The following is an example: ```properties + zkServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 + ``` Once you appropriately modify the `zkServers` parameter, you can make any other configuration changes that you require. You can find a full listing of the available BookKeeper configuration parameters [here](reference-configuration.md#bookkeeper). However, consulting the [BookKeeper documentation](http://bookkeeper.apache.org/docs/latest/reference/config/) for a more in-depth guide might be a better choice. @@ -286,19 +320,25 @@ Once you apply the desired configuration in `conf/bookkeeper.conf`, you can star To start the bookie in the background, use the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start bookie + ``` To start the bookie in the foreground: ```bash + $ bin/pulsar bookie + ``` You can verify that a bookie works properly by running the `bookiesanity` command on the [BookKeeper shell](reference-cli-tools.md#shell): ```bash + $ bin/bookkeeper shell bookiesanity + ``` This command creates an ephemeral BookKeeper ledger on the local bookie, writes a few entries, reads them back, and finally deletes the ledger. @@ -306,7 +346,9 @@ This command creates an ephemeral BookKeeper ledger on the local bookie, writes After you start all the bookies, you can use `simpletest` command for [BookKeeper shell](reference-cli-tools.md#shell) on any bookie node, to verify all the bookies in the cluster are up running. ```bash + $ bin/bookkeeper shell simpletest --ensemble --writeQuorum --ackQuorum --numEntries + ``` This command creates a `num-bookies` sized ledger on the cluster, writes a few entries, and finally deletes the ledger. @@ -321,28 +363,36 @@ Pulsar brokers are the last thing you need to deploy in your Pulsar cluster. Bro The most important element of broker configuration is ensuring that each broker is aware of the ZooKeeper cluster that you have deployed. Ensure that the [`zookeeperServers`](reference-configuration.md#broker-zookeeperServers) and [`configurationStoreServers`](reference-configuration.md#broker-configurationStoreServers) parameters are correct. In this case, since you only have 1 cluster and no configuration store setup, the `configurationStoreServers` point to the same `zookeeperServers`. ```properties + zookeeperServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 configurationStoreServers=zk1.us-west.example.com:2181,zk2.us-west.example.com:2181,zk3.us-west.example.com:2181 + ``` You also need to specify the cluster name (matching the name that you provided when you [initialize the metadata of the cluster](#initialize-cluster-metadata)): ```properties + clusterName=pulsar-cluster-1 + ``` In addition, you need to match the broker and web service ports provided when you initialize the metadata of the cluster (especially when you use a different port than the default): ```properties + brokerServicePort=6650 brokerServicePortTls=6651 webServicePort=8080 webServicePortTls=8443 + ``` > If you deploy Pulsar in a one-node cluster, you should update the replication settings in `conf/broker.conf` to `1`. > + > ```properties +> > # Number of bookies to use when creating a ledger > managedLedgerDefaultEnsembleSize=1 > @@ -351,25 +401,31 @@ webServicePortTls=8443 > > # Number of guaranteed copies (acks to wait before write is complete) > managedLedgerDefaultAckQuorum=1 +> +> > ``` ### Enable Pulsar Functions (optional) -If you want to enable [Pulsar Functions](functions-overview.md), you can follow the instructions as below: +If you want to enable [Pulsar Functions](functions-overview), you can follow the instructions as below: 1. Edit `conf/broker.conf` to enable functions worker, by setting `functionsWorkerEnabled` to `true`. - ```conf - functionsWorkerEnabled=true - ``` + ```conf + + functionsWorkerEnabled=true + + ``` 2. Edit `conf/functions_worker.yml` and set `pulsarFunctionsCluster` to the cluster name that you provide when you [initialize the metadata of the cluster](#initialize-cluster-metadata). - ```conf - pulsarFunctionsCluster: pulsar-cluster-1 - ``` + ```conf + + pulsarFunctionsCluster: pulsar-cluster-1 + + ``` -If you want to learn more options about deploying the functions worker, check out [Deploy and manage functions worker](functions-worker.md). +If you want to learn more options about deploying the functions worker, check out [Deploy and manage functions worker](functions-worker). ### Start Brokers @@ -378,13 +434,17 @@ You can then provide any other configuration changes that you want in the [`conf You can start a broker in the foreground using the [`pulsar broker`](reference-cli-tools.md#pulsar-broker) command: ```bash + $ bin/pulsar broker + ``` You can start a broker in the background using the [`pulsar-daemon`](reference-cli-tools.md#pulsar-daemon) CLI tool: ```bash + $ bin/pulsar-daemon start broker + ``` Once you successfully start up all the brokers that you intend to use, your Pulsar cluster should be ready to go! @@ -396,24 +456,32 @@ Once your Pulsar cluster is up and running, you should be able to connect with i To use the `pulsar-client` tool, first modify the client configuration file in [`conf/client.conf`](reference-configuration.md#client) in your binary package. You need to change the values for `webServiceUrl` and `brokerServiceUrl`, substituting `localhost` (which is the default), with the DNS name that you assign to your broker/bookie hosts. The following is an example: ```properties + webServiceUrl=http://us-west.example.com:8080 brokerServiceurl=pulsar://us-west.example.com:6650 + ``` > If you do not have a DNS server, you can specify multi-host in service URL as follows: > + > ```properties +> > webServiceUrl=http://host1:8080,host2:8080,host3:8080 > brokerServiceurl=pulsar://host1:6650,host2:6650,host3:6650 +> +> > ``` Once that is complete, you can publish a message to the Pulsar topic: ```bash + $ bin/pulsar-client produce \ persistent://public/default/test \ -n 1 \ -m "Hello Pulsar" + ``` > You may need to use a different cluster name in the topic if you specify a cluster name other than `pulsar-cluster-1`. @@ -421,18 +489,22 @@ $ bin/pulsar-client produce \ This command publishes a single message to the Pulsar topic. In addition, you can subscribe to the Pulsar topic in a different terminal before publishing messages as below: ```bash + $ bin/pulsar-client consume \ persistent://public/default/test \ -n 100 \ -s "consumer-test" \ -t "Exclusive" + ``` Once you successfully publish the above message to the topic, you should see it in the standard output: ```bash + ----- got message ----- Hello Pulsar + ``` ## Run Functions @@ -442,6 +514,7 @@ Hello Pulsar Create an ExclamationFunction `exclamation`. ```bash + bin/pulsar-admin functions create \ --jar examples/api-examples.jar \ --classname org.apache.pulsar.functions.api.examples.ExclamationFunction \ @@ -450,16 +523,22 @@ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ --name exclamation + ``` Check whether the function runs as expected by [triggering](functions-deploying.md#triggering-pulsar-functions) the function. ```bash + bin/pulsar-admin functions trigger --name exclamation --trigger-value "hello world" + ``` You should see the following output: ```shell + hello world! + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-dcos.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-dcos.md index 99a142a273cd3..14a36352e6872 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-dcos.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-dcos.md @@ -1,7 +1,7 @@ --- id: deploy-dcos title: Deploy Pulsar on DC/OS -sidebar_label: DC/OS +sidebar_label: "DC/OS" original_id: deploy-dcos --- @@ -11,7 +11,7 @@ import TabItem from '@theme/TabItem'; > ### Tips > -> If you want to enable all builtin [Pulsar IO](io-overview.md) connectors in your Pulsar deployment, you can choose to use `apachepulsar/pulsar-all` image instead of +> If you want to enable all builtin [Pulsar IO](io-overview) connectors in your Pulsar deployment, you can choose to use `apachepulsar/pulsar-all` image instead of > `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). [DC/OS](https://dcos.io/) (the DataCenter Operating System) is a distributed operating system used for deploying and managing applications and systems on [Apache Mesos](http://mesos.apache.org/). DC/OS is an open-source tool that [Mesosphere](https://mesosphere.com/) creates and maintains . @@ -28,7 +28,9 @@ In order to run Pulsar on DC/OS, you need the following: * The [`PulsarGroups.json`](https://github.com/apache/pulsar/blob/master/deployment/dcos/PulsarGroups.json) configuration file from the Pulsar GitHub repo. ```bash + $ curl -O https://raw.githubusercontent.com/apache/pulsar/master/deployment/dcos/PulsarGroups.json + ``` Each node in the DC/OS-managed Mesos cluster must have at least: @@ -44,7 +46,9 @@ Alternatively, you can change the configuration in `PulsarGroups.json` according You can deploy Pulsar on DC/OS using this command: ```bash + $ dcos marathon group add PulsarGroups.json + ``` This command deploys Docker container instances in three groups, which together comprise a Pulsar cluster: @@ -131,7 +135,9 @@ Now that you have a fully deployed Pulsar cluster, you can run a simple consumer You can clone a [Pulsar Java tutorial](https://github.com/streamlio/pulsar-java-tutorial) repo. This repo contains a simple Pulsar consumer and producer (you can find more information in the `README` file of the repo). ```bash + $ git clone https://github.com/streamlio/pulsar-java-tutorial + ``` Change the `SERVICE_URL` from `pulsar://localhost:6650` to `pulsar://a1.dcos:6650` in both [`ConsumerTutorial.java`](https://github.com/streamlio/pulsar-java-tutorial/blob/master/src/main/java/tutorial/ConsumerTutorial.java) and [`ProducerTutorial.java`](https://github.com/streamlio/pulsar-java-tutorial/blob/master/src/main/java/tutorial/ProducerTutorial.java). @@ -142,7 +148,9 @@ Now, change the message number from 10 to 10000000 in main method of [`ProducerT Now compile the project code using the command below: ```bash + $ mvn clean package + ``` ### Run the consumer and producer @@ -150,13 +158,17 @@ $ mvn clean package Execute this command to run the consumer: ```bash + $ mvn exec:java -Dexec.mainClass="tutorial.ConsumerTutorial" + ``` Execute this command to run the producer: ```bash + $ mvn exec:java -Dexec.mainClass="tutorial.ProducerTutorial" + ``` You can see the producer producing messages and the consumer consuming messages through the DC/OS GUI. @@ -178,10 +190,13 @@ You can shut down and uninstall the `pulsar` application from DC/OS at any time 1. Using the DC/OS GUI, you can choose **Delete** at the right end of Pulsar group. - ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) + ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) 2. You can use the following command: - ```bash - $ dcos marathon group remove /pulsar - ``` + ```bash + + $ dcos marathon group remove /pulsar + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-docker.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-docker.md index 07d0e00ecd4ef..f76318f67daaf 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-docker.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-docker.md @@ -1,7 +1,7 @@ --- id: deploy-docker title: Deploy a cluster on Docker -sidebar_label: Docker +sidebar_label: "Docker" original_id: deploy-docker --- @@ -23,34 +23,42 @@ To run Pulsar on Docker, you need to create a container for each Pulsar componen You can pull a Pulsar image from [Docker Hub](https://hub.docker.com/r/apachepulsar/pulsar-all/tags) with the following command. ``` + docker pull apachepulsar/pulsar-all:latest + ``` ### Create three containers Create containers for ZooKeeper, BookKeeper and broker. In this example, they are named as `zookeeper`, `bookkeeper` and `broker` respectively. You can name them as you want with the `--name` flag. By default, the container names are created randomly. ``` + docker run -it --name bookkeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name zookeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name broker apachepulsar/pulsar-all:latest /bin/bash + ``` ### Create a network To deploy a Pulsar cluster on Docker, you need to create a `network` and connect the containers of ZooKeeper, BookKeeper and broker to this network. The following command creates the network `pulsar`: ``` + docker network create pulsar + ``` ### Connect containers to network Connect the containers of ZooKeeper, BookKeeper and broker to the `pulsar` network with the following commands. ``` + docker network connect pulsar zookeeper docker network connect pulsar bookkeeper docker network connect pulsar broker + ``` To check whether the containers are successfully connected to the network, enter the `docker network inspect pulsar` command. -For detailed information about how to deploy ZooKeeper cluster, BookKeeper cluster, brokers, see [deploy a cluster on bare metal](deploy-bare-metal.md). +For detailed information about how to deploy ZooKeeper cluster, BookKeeper cluster, brokers, see [deploy a cluster on bare metal](deploy-bare-metal). diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-kubernetes.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-kubernetes.md index 4edbbf197b909..ad31619796889 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-kubernetes.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-kubernetes.md @@ -1,7 +1,7 @@ --- id: deploy-kubernetes title: Deploy Pulsar on Kubernetes -sidebar_label: Kubernetes +sidebar_label: "Kubernetes" original_id: deploy-kubernetes --- @@ -10,6 +10,6 @@ import TabItem from '@theme/TabItem'; To get up and running with these charts as fast as possible, in a **non-production** use case, we provide -a [quick start guide](getting-started-helm.md) for Proof of Concept (PoC) deployments. +a [quick start guide](getting-started-helm) for Proof of Concept (PoC) deployments. -To configure and install a Pulsar cluster on Kubernetes for production usage, follow the complete [Installation Guide](helm-install.md). +To configure and install a Pulsar cluster on Kubernetes for production usage, follow the complete [Installation Guide](helm-install). diff --git a/site2/website-next/versioned_docs/version-2.7.3/deploy-monitoring.md b/site2/website-next/versioned_docs/version-2.7.3/deploy-monitoring.md index a1002f2cac3c5..6923caa805da8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/deploy-monitoring.md +++ b/site2/website-next/versioned_docs/version-2.7.3/deploy-monitoring.md @@ -1,7 +1,7 @@ --- id: deploy-monitoring title: Monitor -sidebar_label: Monitor +sidebar_label: "Monitor" original_id: deploy-monitoring --- @@ -22,13 +22,17 @@ You can collect Pulsar broker metrics from brokers and export the metrics in JSO * *Destination dumps*, which contain stats for each individual topic. You can fetch the destination dumps using the command below: ```shell + bin/pulsar-admin broker-stats destinations + ``` * Broker metrics, which contain the broker information and topics stats aggregated at namespace level. You can fetch the broker metrics by using the following command: ```shell + bin/pulsar-admin broker-stats monitoring-metrics + ``` All the message rates are updated every minute. @@ -36,7 +40,9 @@ All the message rates are updated every minute. The aggregated broker metrics are also exposed in the [Prometheus](https://prometheus.io) format at: ```shell + http://$BROKER_ADDRESS:8080/metrics + ``` ### ZooKeeper stats @@ -44,8 +50,10 @@ http://$BROKER_ADDRESS:8080/metrics The local ZooKeeper, configuration store server and clients that are shipped with Pulsar can expose detailed stats through Prometheus. ```shell + http://$LOCAL_ZK_SERVER:8000/metrics http://$GLOBAL_ZK_SERVER:8001/metrics + ``` The default port of local ZooKeeper is `8000` and the default port of configuration store is `8001`. You can change the default port of local ZooKeeper and configuration store by specifying system property `stats_server_port`. @@ -57,7 +65,9 @@ You can configure the stats frameworks for BookKeeper by modifying the `statsPro The default BookKeeper configuration enables the Prometheus exporter. The configuration is included with Pulsar distribution. ```shell + http://$BOOKIE_ADDRESS:8000/metrics + ``` The default port for bookie is `8000`. You can change the port by configuring `prometheusStatsHttpPort` in the `conf/bookkeeper.conf` file. @@ -66,10 +76,12 @@ The default port for bookie is `8000`. You can change the port by configuring `p The acknowledgment state is persistent to the ledger first. When the acknowledgment state fails to be persistent to the ledger, they are persistent to ZooKeeper. To track the stats of acknowledgement, you can configure the metrics for the managed cursor. ``` -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"") + +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:") brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"") + ``` Those metrics are added in the Prometheus interface, you can monitor and check the metrics stats in the Grafana. @@ -78,7 +90,7 @@ Those metrics are added in the Prometheus interface, you can monitor and check t You can use Prometheus to collect all the metrics exposed for Pulsar components and set up [Grafana](https://grafana.com/) dashboards to display the metrics and monitor your Pulsar cluster. For details, refer to [Prometheus guide](https://prometheus.io/docs/introduction/getting_started/). -When you run Pulsar on bare metal, you can provide the list of nodes to be probed. When you deploy Pulsar in a Kubernetes cluster, the monitoring is setup automatically. For details, refer to [Kubernetes instructions](helm-deploy.md). +When you run Pulsar on bare metal, you can provide the list of nodes to be probed. When you deploy Pulsar in a Kubernetes cluster, the monitoring is setup automatically. For details, refer to [Kubernetes instructions](helm-deploy). ## Dashboards @@ -86,7 +98,7 @@ When you collect time series statistics, the major problem is to make sure the n ### Pulsar per-topic dashboard -The per-topic dashboard instructions are available at [Pulsar manager](administration-pulsar-manager.md). +The per-topic dashboard instructions are available at [Pulsar manager](administration-pulsar-manager). ### Grafana @@ -97,9 +109,11 @@ When you deploy Pulsar on Kubernetes, a `pulsar-grafana` Docker image is enabled Enter the command below to use the dashboard manually: ```shell + docker run -p3000:3000 \ -e PROMETHEUS_URL=http://$PROMETHEUS_HOST:9090/ \ apachepulsar/pulsar-grafana:latest + ``` The following are some Grafana dashboards examples: diff --git a/site2/website-next/versioned_docs/version-2.7.3/developing-binary-protocol.md b/site2/website-next/versioned_docs/version-2.7.3/developing-binary-protocol.md index f547375073cc8..3a202a38bcc4a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/developing-binary-protocol.md +++ b/site2/website-next/versioned_docs/version-2.7.3/developing-binary-protocol.md @@ -1,7 +1,7 @@ --- id: develop-binary-protocol title: Pulsar binary protocol specification -sidebar_label: Binary protocol +sidebar_label: "Binary protocol" original_id: develop-binary-protocol --- @@ -350,7 +350,7 @@ message CommandFlow { Parameters: * `consumer_id` → Id of an already established consumer * `messagePermits` → Number of additional permits to grant to the broker for - pushing more messages + pushing more messages ##### Command Message diff --git a/site2/website-next/versioned_docs/version-2.7.3/developing-cpp.md b/site2/website-next/versioned_docs/version-2.7.3/developing-cpp.md index d661ea1e54bd6..76c5671431f82 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/developing-cpp.md +++ b/site2/website-next/versioned_docs/version-2.7.3/developing-cpp.md @@ -1,7 +1,7 @@ --- id: develop-cpp title: Building Pulsar C++ client -sidebar_label: Building Pulsar C++ client +sidebar_label: "Building Pulsar C++ client" original_id: develop-cpp --- @@ -115,3 +115,4 @@ $ cmake . $ make ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/developing-load-manager.md b/site2/website-next/versioned_docs/version-2.7.3/developing-load-manager.md index de443b10025aa..ed60d5ced6193 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/developing-load-manager.md +++ b/site2/website-next/versioned_docs/version-2.7.3/developing-load-manager.md @@ -1,7 +1,7 @@ --- id: develop-load-manager title: Modular load manager -sidebar_label: Modular load manager +sidebar_label: "Modular load manager" original_id: develop-load-manager --- @@ -19,11 +19,11 @@ There are two ways that you can enable the modular load manager: 2. Using the `pulsar-admin` tool. Here's an example: ```shell - + $ pulsar-admin update-dynamic-config \ - --config loadManagerClassName \ - --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl - + --config loadManagerClassName \ + --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl + ``` You can use the same method to change back to the original value. In either case, any mistake in specifying the load manager will cause Pulsar to default to `SimpleLoadManagerImpl`. @@ -34,111 +34,113 @@ There are a few different ways to determine which load manager is being used: 1. Use `pulsar-admin` to examine the `loadManagerClassName` element: - ```shell - + ```shell + $ bin/pulsar-admin brokers get-all-dynamic-config { - "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" + "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" } - + ``` If there is no `loadManagerClassName` element, then the default load manager is used. 2. Consult a ZooKeeper load report. With the module load manager, the load report in `/loadbalance/brokers/...` will have many differences. for example the `systemResourceUsage` sub-elements (`bandwidthIn`, `bandwidthOut`, etc.) are now all at the top level. Here is an example load report from the module load manager: - ```json - - { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 4.256510416666667 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 5.287239583333333 - }, - "bundles": [], - "cpu": { - "limit": 2400.0, - "usage": 5.7353247655435915 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - } - } - - ``` - - With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: - - ```json - - { - "systemResourceUsage": { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 0.0 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 0.0 - }, - "cpu": { - "limit": 2400.0, - "usage": 0.0 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - }, - "memory": { - "limit": 8192.0, - "usage": 3903.0 - } - } - } - - ``` + ```json + + { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 4.256510416666667 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 5.287239583333333 + }, + "bundles": [], + "cpu": { + "limit": 2400.0, + "usage": 5.7353247655435915 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + } + } + + ``` + + With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: + + ```json + + { + "systemResourceUsage": { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 0.0 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 0.0 + }, + "cpu": { + "limit": 2400.0, + "usage": 0.0 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + }, + "memory": { + "limit": 8192.0, + "usage": 3903.0 + } + } + } + + ``` 3. The command-line [broker monitor](reference-cli-tools.md#monitor-brokers) will have a different output format depending on which load manager implementation is being used. - Here is an example from the modular load manager: - - ``` - =================================================================================================================== - ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |4 |0 || - ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - =================================================================================================================== - - ``` - - Here is an example from the simple load manager: - - ``` - =================================================================================================================== - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |0 |0 || - ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || - ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.20 |1.89 | |1.27 |3.21 |3.21 || - ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || - ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || - =================================================================================================================== - - ``` + Here is an example from the modular load manager: + + ``` + + =================================================================================================================== + ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |4 |0 || + ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + =================================================================================================================== + + ``` + + Here is an example from the simple load manager: + + ``` + + =================================================================================================================== + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |0 |0 || + ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || + ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.20 |1.89 | |1.27 |3.21 |3.21 || + ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || + ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || + =================================================================================================================== + + ``` It is important to note that the module load manager is _centralized_, meaning that all requests to assign a bundle---whether it's been seen before or whether this is the first time---only get handled by the _lead_ broker (which can change over time). To determine the current lead broker, examine the `/loadbalance/leader` node in ZooKeeper. diff --git a/site2/website-next/versioned_docs/version-2.7.3/developing-tools.md b/site2/website-next/versioned_docs/version-2.7.3/developing-tools.md index 8f068a291aecb..f560f9d79162a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/developing-tools.md +++ b/site2/website-next/versioned_docs/version-2.7.3/developing-tools.md @@ -1,7 +1,7 @@ --- id: develop-tools title: Simulation tools -sidebar_label: Simulation tools +sidebar_label: "Simulation tools" original_id: develop-tools --- @@ -24,6 +24,7 @@ send signals to clients to start incurring load. The client implementation is in To Start a simulation client, use the `pulsar-perf` script with the command `simulation-client` as follows: ``` + pulsar-perf simulation-client --port --service-url ``` @@ -39,6 +40,7 @@ command with. To start a simulation controller, use the `pulsar-perf` script with the command `simulation-controller` as follows: ``` + pulsar-perf simulation-controller --cluster --client-port --clients @@ -51,32 +53,32 @@ names. In all cases, the BASE name of the tenants, namespaces, and topics are us `my_namespace`, and the topic name is `my_topic`. The controller can perform the following actions: * Create a topic with a producer and a consumer - * `trade [--rate ] - [--rand-rate ,] - [--size ]` + * `trade [--rate ] + [--rand-rate ,] + [--size ]` * Create a group of topics with a producer and a consumer - * `trade_group [--rate ] - [--rand-rate ,] - [--separation ] [--size ] - [--topics-per-namespace ]` + * `trade_group [--rate ] + [--rand-rate ,] + [--separation ] [--size ] + [--topics-per-namespace ]` * Change the configuration of an existing topic - * `change [--rate ] - [--rand-rate ,] - [--size ]` + * `change [--rate ] + [--rand-rate ,] + [--size ]` * Change the configuration of a group of topics - * `change_group [--rate ] [--rand-rate ,] - [--size ] [--topics-per-namespace ]` + * `change_group [--rate ] [--rand-rate ,] + [--size ] [--topics-per-namespace ]` * Shutdown a previously created topic - * `stop ` + * `stop ` * Shutdown a previously created group of topics - * `stop_group ` + * `stop_group ` * Copy the historical data from one ZooKeeper to another and simulate based on the message rates and sizes in that history * `copy [--rate-multiplier value]` * Simulate the load of the historical data on the current ZooKeeper (should be same ZooKeeper being simulated on) - * `simulate [--rate-multiplier value]` + * `simulate [--rate-multiplier value]` * Stream the latest data from the given active ZooKeeper to simulate the real-time load of that ZooKeeper. - * `stream [--rate-multiplier value]` + * `stream [--rate-multiplier value]` The "group" arguments in these commands allow the user to create or affect multiple topics at once. Groups are created when calling the `trade_group` command, and all topics from these groups may be subsequently modified or stopped @@ -105,6 +107,7 @@ console as it is updated using watchers. To start a broker monitor, use the `monitor-brokers` command in the `pulsar-perf` script: ``` + pulsar-perf monitor-brokers --connect-string ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-debug.md b/site2/website-next/versioned_docs/version-2.7.3/functions-debug.md index f98429beaddcd..2587b48eff857 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-debug.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-debug.md @@ -29,8 +29,8 @@ A Pulsar Function is a function with inputs and outputs, you can test a Pulsar F For example, if you have the following Pulsar Function: - ```java + import java.util.function.Function; public class JavaNativeExclamationFunction implements Function { @@ -39,27 +39,32 @@ public class JavaNativeExclamationFunction implements Function { return String.format("%s!", input); } } + ``` You can write a simple unit test to test Pulsar Function. -> #### Tip -> Pulsar uses testng for testing. +:::tip +Pulsar uses testng for testing. + +::: ```java + @Test public void testJavaNativeExclamationFunction() { JavaNativeExclamationFunction exclamation = new JavaNativeExclamationFunction(); String output = exclamation.apply("foo"); Assert.assertEquals(output, "foo!"); } + ``` The following Pulsar Function implements the `org.apache.pulsar.functions.api.Function` interface. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -69,21 +74,26 @@ public class ExclamationFunction implements Function { return String.format("%s!", input); } } + ``` In this situation, you can write a unit test for this function as well. Remember to mock the `Context` parameter. The following is an example. -> #### Tip -> Pulsar uses testng for testing. +:::tip +Pulsar uses testng for testing. + +::: ```java + @Test public void testExclamationFunction() { ExclamationFunction exclamation = new ExclamationFunction(); String output = exclamation.process("foo", mock(Context.class)); Assert.assertEquals(output, "foo!"); } + ``` ## Debug with localrun mode @@ -91,13 +101,16 @@ When you run a Pulsar Function in localrun mode, it launches an instance of the In this mode, a Pulsar Function consumes and produces actual data to a Pulsar cluster, and mirrors how the function actually runs in a Pulsar cluster. -> Note -> Currently, debugging with localrun mode is only supported by Pulsar Functions written in Java. You need Pulsar version 2.4.0 or later to do the following. Even though localrun is available in versions earlier than Pulsar 2.4.0, you cannot debug with localrun mode programmatically or run Functions as threads. +:::note -You can launch your function in the following manner. +Currently, debugging with localrun mode is only supported by Pulsar Functions written in Java. You need Pulsar version 2.4.0 or later to do the following. Even though localrun is available in versions earlier than Pulsar 2.4.0, you cannot debug with localrun mode programmatically or run Functions as threads. + +::: +You can launch your function in the following manner. ```java + FunctionConfig functionConfig = new FunctionConfig(); functionConfig.setName(functionName); functionConfig.setInputs(Collections.singleton(sourceTopic)); @@ -107,14 +120,15 @@ functionConfig.setOutput(sinkTopic); LocalRunner localRunner = LocalRunner.builder().functionConfig(functionConfig).build(); localRunner.start(true); + ``` So you can debug functions using an IDE easily. Set breakpoints and manually step through a function to debug with real data. The following example illustrates how to programmatically launch a function in localrun mode. - ```java + public class ExclamationFunction implements Function { @Override @@ -133,12 +147,13 @@ public static void main(String[] args) throws Exception { LocalRunner localRunner = LocalRunner.builder().functionConfig(functionConfig).build(); localRunner.start(false); } + ``` To use localrun mode programmatically, add the following dependency. - ```xml + org.apache.pulsar pulsar-functions-local-runner @@ -149,8 +164,11 @@ To use localrun mode programmatically, add the following dependency. For complete code samples, see [here](https://github.com/jerrypeng/pulsar-functions-demos/tree/master/debugging). -> Note -> Debugging with localrun mode for Pulsar Functions written in other languages will be supported soon. +:::note + +Debugging with localrun mode for Pulsar Functions written in other languages will be supported soon. + +::: ## Use log topic @@ -160,8 +178,8 @@ In Pulsar Functions, you can generate log information defined in functions to a **Example** - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -181,17 +199,19 @@ public class LoggingFunction implements Function { return null; } } + ``` As shown in the example above, you can get the logger via `context.getLogger()` and assign the logger to the `LOG` variable of `slf4j`, so you can define your desired log information in a function using the `LOG` variable. Meanwhile, you need to specify the topic to which the log information is produced. **Example** - ```bash + $ bin/pulsar-admin functions create \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` ## Use Functions CLI @@ -204,9 +224,11 @@ With [Pulsar Functions CLI](reference-pulsar-admin.md#functions), you can debug * `list` * `trigger` -> **Tip** -> -> For complete commands of **Pulsar Functions CLI**, see [here](reference-pulsar-admin.md#functions)。 +:::tip + +For complete commands of **Pulsar Functions CLI**, see [here](reference-pulsar-admin.md#functions)。 + +::: ### `get` @@ -214,9 +236,10 @@ Get information about a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions get options + ``` **Options** @@ -228,32 +251,37 @@ $ pulsar-admin functions get options |`--namespace`|The namespace of a Pulsar Function. |`--tenant`|The tenant of a Pulsar Function. -> **Tip** -> -> `--fqfn` consists of `--name`, `--namespace` and `--tenant`, so you can specify either `--fqfn` or `--name`, `--namespace` and `--tenant`. +:::tip + +`--fqfn` consists of `--name`, `--namespace` and `--tenant`, so you can specify either `--fqfn` or `--name`, `--namespace` and `--tenant`. + +::: **Example** You can specify `--fqfn` to get information about a Pulsar Function. - ```bash + $ ./bin/pulsar-admin functions get public/default/ExclamationFunctio6 + ``` -Optionally, you can specify `--name`, `--namespace` and `--tenant` to get information about a Pulsar Function. +Optionally, you can specify `--name`, `--namespace` and `--tenant` to get information about a Pulsar Function. ```bash + $ ./bin/pulsar-admin functions get \ --tenant public \ --namespace default \ --name ExclamationFunctio6 + ``` As shown below, the `get` command shows input, output, runtime, and other information about the _ExclamationFunctio6_ function. - ```json + { "tenant": "public", "namespace": "default", @@ -272,6 +300,7 @@ As shown below, the `get` command shows input, output, runtime, and other inform "autoAck": true, "parallelism": 1 } + ``` ### `status` @@ -280,9 +309,10 @@ Check the current status of a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions status options + ``` **Options** @@ -297,18 +327,19 @@ $ pulsar-admin functions status options **Example** - ```bash + $ ./bin/pulsar-admin functions status \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ + ``` As shown below, the `status` command shows the number of instances, running instances, the instance running under the _ExclamationFunctio6_ function, received messages, successfully processed messages, system exceptions, the average latency and so on. - ```json + { "numInstances" : 1, "numRunning" : 1, @@ -330,6 +361,7 @@ As shown below, the `status` command shows the number of instances, running inst } } ] } + ``` ### `stats` @@ -338,9 +370,10 @@ Get the current stats of a Pulsar Function. **Usage** - ```bash + $ pulsar-admin functions stats options + ``` **Options** @@ -355,18 +388,19 @@ $ pulsar-admin functions stats options **Example** - ```bash + $ ./bin/pulsar-admin functions stats \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ + ``` The output is shown as follows: - ```json + { "receivedTotal" : 1, "processedSuccessfullyTotal" : 1, @@ -401,6 +435,7 @@ The output is shown as follows: } } ] } + ``` ### `list` @@ -409,9 +444,10 @@ List all Pulsar Functions running under a specific tenant and namespace. **Usage** - ```bash + $ pulsar-admin functions list options + ``` **Options** @@ -423,19 +459,22 @@ $ pulsar-admin functions list options **Example** - ```bash + $ ./bin/pulsar-admin functions list \ --tenant public \ --namespace default + ``` -As shown below, the `list` command returns three functions running under the _public_ tenant and the _default_ namespace. +As shown below, the `list` command returns three functions running under the _public_ tenant and the _default_ namespace. ```text + ExclamationFunctio1 ExclamationFunctio2 ExclamationFunctio3 + ``` ### `trigger` @@ -444,9 +483,10 @@ Trigger a specified Pulsar Function with a supplied value. This command simulate **Usage** - ```bash + $ pulsar-admin functions trigger options + ``` **Options** @@ -463,29 +503,35 @@ $ pulsar-admin functions trigger options **Example** - ```bash + $ ./bin/pulsar-admin functions trigger \ --tenant public \ --namespace default \ --name ExclamationFunctio6 \ --topic persistent://public/default/my-topic-1 \ --trigger-value "hello pulsar functions" + ``` As shown below, the `trigger` command returns the following result: - ```text + This is my function! + ``` -> #### **Note** -> You must specify the [entire topic name](getting-started-pulsar.md#topic-names) when using the `--topic` option. Otherwise, the following error occurs. -> -> +:::note + +You must specify the [entire topic name](getting-started-pulsar.md#topic-names) when using the `--topic` option. Otherwise, the following error occurs. + ```text ->Function in trigger function has unidentified topic -> ->Reason: Function in trigger function has unidentified topic ->``` + +Function in trigger function has unidentified topic +Reason: Function in trigger function has unidentified topic + +``` + +::: + diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-deploy.md b/site2/website-next/versioned_docs/version-2.7.3/functions-deploy.md index f694e239942ec..03eb08c8b3efb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-deploy.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-deploy.md @@ -13,12 +13,12 @@ import TabItem from '@theme/TabItem'; To deploy and manage Pulsar Functions, you need to have a Pulsar cluster running. There are several options for this: -* You can run a [standalone cluster](getting-started-standalone.md) locally on your own machine. -* You can deploy a Pulsar cluster on [Kubernetes](deploy-kubernetes.md), [Amazon Web Services](deploy-aws.md), [bare metal](deploy-bare-metal.md), [DC/OS](deploy-dcos.md), and more. +* You can run a [standalone cluster](getting-started-standalone) locally on your own machine. +* You can deploy a Pulsar cluster on [Kubernetes](deploy-kubernetes.md), [Amazon Web Services](deploy-aws.md), [bare metal](deploy-bare-metal.md), [DC/OS](deploy-dcos), and more. If you run a non-[standalone](reference-terminology.md#standalone) cluster, you need to obtain the service URL for the cluster. How you obtain the service URL depends on how you deploy your Pulsar cluster. -If you want to deploy and trigger Python user-defined functions, you need to install [the pulsar python client](http://pulsar.apache.org/docs/en/client-libraries-python/) on all the machines running [functions workers](functions-worker.md). +If you want to deploy and trigger Python user-defined functions, you need to install [the pulsar python client](http://pulsar.apache.org/docs/en/client-libraries-python/) on all the machines running [functions workers](functions-worker). ## Command-line interface @@ -44,12 +44,13 @@ Pulsar service URL | `pulsar://localhost:6650` Take the `create` command as an example. - ```bash + $ bin/pulsar-admin functions create \ --jar my-pulsar-functions.jar \ --classname org.example.MyFunction \ --inputs my-function-input-topic1,my-function-input-topic2 + ``` The function has default values for the function name (`MyFunction`), tenant (`public`), namespace (`default`), subscription type (`SHARED`), processing guarantees (`ATLEAST_ONCE`), and Pulsar service URL (`pulsar://localhost:6650`). @@ -58,48 +59,52 @@ The function has default values for the function name (`MyFunction`), tenant (`p If you run a Pulsar Function in **local run** mode, it runs on the machine from which you enter the commands (on your laptop, an [AWS EC2](https://aws.amazon.com/ec2/) instance, and so on). The following is a [`localrun`](reference-pulsar-admin.md#localrun) command example. - ```bash + $ bin/pulsar-admin functions localrun \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/input-1 \ --output persistent://public/default/output-1 + ``` By default, the function connects to a Pulsar cluster running on the same machine, via a local [broker](reference-terminology.md#broker) service URL of `pulsar://localhost:6650`. If you use local run mode to run a function but connect it to a non-local Pulsar cluster, you can specify a different broker URL using the `--brokerServiceUrl` flag. The following is an example. - ```bash + $ bin/pulsar-admin functions localrun \ --broker-service-url pulsar://my-cluster-host:6650 \ # Other function parameters + ``` ## Cluster mode When you run a Pulsar Function in **cluster** mode, the function code is uploaded to a Pulsar broker and runs *alongside the broker* rather than in your [local environment](#local-run-mode). You can run a function in cluster mode using the [`create`](reference-pulsar-admin.md#create-1) command. - ```bash + $ bin/pulsar-admin functions create \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/input-1 \ --output persistent://public/default/output-1 + ``` ### Update functions in cluster mode You can use the [`update`](reference-pulsar-admin.md#update-1) command to update a Pulsar Function running in cluster mode. The following command updates the function created in the [cluster mode](#cluster-mode) section. - ```bash + $ bin/pulsar-admin functions update \ --py myfunc.py \ --classname myfunc.SomeFunction \ --inputs persistent://public/default/new-input-topic \ --output persistent://public/default/new-output-topic + ``` ### Parallelism @@ -108,40 +113,44 @@ Pulsar Functions run as processes or threads, which are called **instances**. Wh When you create a function, you can specify the *parallelism* of a function (the number of instances to run). You can set the parallelism factor using the `--parallelism` flag of the [`create`](reference-pulsar-admin.md#functions-create) command. - ```bash + $ bin/pulsar-admin functions create \ --parallelism 3 \ # Other function info + ``` You can adjust the parallelism of an already created function using the [`update`](reference-pulsar-admin.md#update-1) interface. - ```bash + $ bin/pulsar-admin functions update \ --parallelism 5 \ # Other function + ``` If you specify a function configuration via YAML, use the `parallelism` parameter. The following is a config file example. - ```yaml + # function-config.yaml parallelism: 3 inputs: - persistent://public/default/input-1 output: persistent://public/default/output-1 # other parameters + ``` The following is corresponding update command. - ```bash + $ bin/pulsar-admin functions update \ --function-config-file function-config.yaml + ``` ### Function instance resources @@ -156,14 +165,15 @@ Disk space | The number of bytes | Docker The following function creation command allocates 8 cores, 8 GB of RAM, and 10 GB of disk space to a function. - ```bash + $ bin/pulsar-admin functions create \ --jar target/my-functions.jar \ --classname org.example.functions.MyFunction \ --cpu 8 \ --ram 8589934592 \ --disk 10737418240 + ``` > #### Resources are *per instance* @@ -177,17 +187,18 @@ If a Pulsar Function is running in [cluster mode](#cluster-mode), you can **trig To learn how to trigger a function, you can start with Python function that returns a simple string based on the input. - ```python + # myfunc.py def process(input): return "This function has been triggered with a value of {0}".format(input) + ``` You can run the function in [local run mode](functions-deploy.md#local-run-mode). - ```bash + $ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ @@ -196,34 +207,38 @@ $ bin/pulsar-admin functions create \ --classname myfunc \ --inputs persistent://public/default/in \ --output persistent://public/default/out + ``` Then assign a consumer to listen on the output topic for messages from the `myfunc` function with the [`pulsar-client consume`](reference-cli-tools.md#consume) command. - ```bash + $ bin/pulsar-client consume persistent://public/default/out \ --subscription-name my-subscription --num-messages 0 # Listen indefinitely + ``` And then you can trigger the function. - ```bash + $ bin/pulsar-admin functions trigger \ --tenant public \ --namespace default \ --name myfunc \ --trigger-value "hello world" + ``` The consumer listening on the output topic produces something as follows in the log. - ``` + ----- got message ----- This function has been triggered with a value of hello world + ``` > #### Topic info is not required diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-develop.md b/site2/website-next/versioned_docs/version-2.7.3/functions-develop.md index f8b7993f02b74..a1520fe5d14d3 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-develop.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-develop.md @@ -36,6 +36,7 @@ The language-native function, which adds an exclamation point to all incoming st ```Java + import java.util.function.Function; public class JavaNativeExclamationFunction implements Function { @@ -44,29 +45,37 @@ public class JavaNativeExclamationFunction implements Function { return String.format("%s!", input); } } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeExclamationFunction.java). ```python + def process(input): return "{}!".format(input) + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/native_exclamation_function.py). -> Note -> You can write Pulsar Functions in python2 or python3. However, Pulsar only looks for `python` as the interpreter. -> -> If you're running Pulsar Functions on an Ubuntu system that only supports python3, you might fail to -> start the functions. In this case, you can create a symlink. Your system will fail if -> you subsequently install any other package that depends on Python 2.x. A solution is under development in [Issue 5518](https://github.com/apache/pulsar/issues/5518). -> -> +:::note + +You can write Pulsar Functions in python2 or python3. However, Pulsar only looks for `python` as the interpreter. +If you're running Pulsar Functions on an Ubuntu system that only supports python3, you might fail to +start the functions. In this case, you can create a symlink. Your system will fail if +you subsequently install any other package that depends on Python 2.x. A solution is under development in [Issue 5518](https://github.com/apache/pulsar/issues/5518). + ```bash -> sudo update-alternatives --install /usr/bin/python python /usr/bin/python3 10 ->``` + +sudo update-alternatives --install /usr/bin/python python /usr/bin/python3 10 + +``` + +::: @@ -92,6 +101,7 @@ The following example uses Pulsar Functions SDK. ```Java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -101,13 +111,16 @@ public class ExclamationFunction implements Function { return String.format("%s!", input); } } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/ExclamationFunction.java). ```python + from pulsar import Function class ExclamationFunction(Function): @@ -116,13 +129,16 @@ class ExclamationFunction(Function): def process(self, input, context): return input + '!' + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/exclamation_function.py). ```Go + package main import ( @@ -140,7 +156,9 @@ func HandleRequest(ctx context.Context, in []byte) error{ func main() { pf.Start(HandleRequest) } + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f1626a53a1fe2e65dd25f377c1127/pulsar-function-go/examples/inputFunc/inputFunc.go#L20-L36). @@ -170,17 +188,20 @@ SerDe stands for **Ser**ialization and **De**serialization. Pulsar Functions use } ]}> + When you write Pulsar Functions in Java, the following basic Java types are built in and supported by default: `String`, `Double`, `Integer`, `Float`, `Long`, `Short`, and `Byte`. To customize Java types, you need to implement the following interface. - ```java + public interface SerDe { T deserialize(byte[] input); byte[] serialize(T input); } + ``` + SerDe works in the following ways in Java Functions. - If the input and output topics have schema, Pulsar Functions use schema for SerDe. - If the input or output topics do not exist, Pulsar Functions adopt the following rules to determine SerDe: @@ -190,12 +211,13 @@ SerDe works in the following ways in Java Functions. + In Python, the default SerDe is identity, meaning that the type is serialized as whatever type the producer function returns. You can specify the SerDe when [creating](functions-deploy.md#cluster-mode) or [running](functions-deploy.md#local-run-mode) functions. - ```bash + $ bin/pulsar-admin functions create \ --tenant public \ --namespace default \ @@ -205,6 +227,7 @@ $ bin/pulsar-admin functions create \ --custom-serde-inputs '{"input-topic-1":"Serde1","input-topic-2":"Serde2"}' \ --output-serde-classname Serde3 \ --output output-topic-1 + ``` This case contains two input topics: `input-topic-1` and `input-topic-2`, each of which is mapped to a different SerDe class (the map must be specified as a JSON string). The output topic, `output-topic-1`, uses the `Serde3` class for SerDe. At the moment, all Pulsar Functions logic, include processing function and SerDe classes, must be contained within a single Python file. @@ -225,6 +248,7 @@ Custom SerDe | When you require explicit control over SerDe, potentially for per + Currently, the feature is not available in Go. @@ -248,8 +272,8 @@ Imagine that you're writing Pulsar Functions that are processing tweet objects, ]}> - ```java + public class Tweet { private String username; private String tweetContent; @@ -261,12 +285,13 @@ public class Tweet { // Standard setters and getters } + ``` To pass `Tweet` objects directly between Pulsar Functions, you need to provide a custom SerDe class. In the example below, `Tweet` objects are basically strings in which the username and tweet content are separated by a `|`. - ```java + package com.example.serde; import org.apache.pulsar.functions.api.SerDe; @@ -284,6 +309,7 @@ public class TweetSerde implements SerDe { return "%s|%s".format(input.getUsername(), input.getTweetContent()).getBytes(); } } + ``` To apply this customized SerDe to a particular Pulsar Function, you need to: @@ -293,12 +319,13 @@ To apply this customized SerDe to a particular Pulsar Function, you need to: The following is an example of [`create`](reference-pulsar-admin.md#create-1) operation. - ```bash + $ bin/pulsar-admin functions create \ --jar /path/to/your.jar \ --output-serde-classname com.example.serde.TweetSerde \ # Other function attributes + ``` > #### Custom SerDe classes must be packaged with your function JARs @@ -307,12 +334,13 @@ $ bin/pulsar-admin functions create \ - ```python + class Tweet(object): def __init__(self, username, tweet_content): self.username = username self.tweet_content = tweet_content + ``` In order to use this class in Pulsar Functions, you have two options: @@ -320,19 +348,21 @@ In order to use this class in Pulsar Functions, you have two options: 1. You can specify `PickleSerDe`, which applies the [`pickle`](https://docs.python.org/3/library/pickle.html) library SerDe. 2. You can create your own SerDe class. The following is an example. + ```python + + from pulsar import SerDe + + class TweetSerDe(SerDe): + + def serialize(self, input): + return bytes("{0}|{1}".format(input.username, input.tweet_content)) + + def deserialize(self, input_bytes): + tweet_components = str(input_bytes).split('|') + return Tweet(tweet_components[0], tweet_componentsp[1]) + + ``` -```python -from pulsar import SerDe - -class TweetSerDe(SerDe): - - def serialize(self, input): - return bytes("{0}|{1}".format(input.username, input.tweet_content)) - - def deserialize(self, input_bytes): - tweet_components = str(input_bytes).split('|') - return Tweet(tweet_components[0], tweet_componentsp[1]) -``` For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/custom_object_function.py). @@ -378,10 +408,11 @@ Java, Python and Go SDKs provide access to a **context object** that can be used } ]}> -The [Context](https://github.com/apache/pulsar/blob/master/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Context.java) interface provides a number of methods that you can use to access the function [context](#context). The various method signatures for the `Context` interface are listed as follows. +The [Context](https://github.com/apache/pulsar/blob/master/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Context.java) interface provides a number of methods that you can use to access the function [context](#context). The various method signatures for the `Context` interface are listed as follows. ```java + public interface Context { Record getCurrentRecord(); Collection getInputTopics(); @@ -414,12 +445,13 @@ public interface Context { PulsarAdmin getPulsarAdmin(); PulsarAdmin getPulsarAdmin(String clusterName); } + ``` The following example uses several methods available via the `Context` object. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -444,12 +476,14 @@ public class ContextFunction implements Function { return null; } } + ``` ``` + class ContextImpl(pulsar.Context): def get_message_id(self): ... @@ -510,12 +544,14 @@ class ContextImpl(pulsar.Context): ... def get_state(self, key): ... + ``` ``` + func (c *FunctionContext) GetInstanceID() int { return c.instanceConf.instanceID } @@ -567,12 +603,13 @@ func (c *FunctionContext) GetCurrentRecord() pulsar.Message { func (c *FunctionContext) NewOutputMessage(topic string) pulsar.Producer { return c.outputMessage(topic) } + ``` The following example uses several methods available via the `Context` object. - ``` + import ( "context" "fmt" @@ -586,6 +623,7 @@ func contextFunc(ctx context.Context) { fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) } } + ``` For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f1626a53a1fe2e65dd25f377c1127/pulsar-function-go/examples/contextFunc/contextFunc.go#L29-L34). @@ -597,12 +635,13 @@ For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f ### User config When you run or update Pulsar Functions created using SDK, you can pass arbitrary key/values to them with the command line with the `--user-config` flag. Key/values must be specified as JSON. The following function creation command passes a user configured key/value to a function. - ```bash + $ bin/pulsar-admin functions create \ --name word-filter \ # Other function configs --user-config '{"forbidden-word":"rosebud"}' + ``` -The Java SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. +The Java SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. ```bash + $ bin/pulsar-admin functions create \ # Other function configs --user-config '{"word-of-the-day":"verdure"}' + ``` To access that value in a Java function: - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -654,29 +695,32 @@ public class UserConfigFunction implements Function { return null; } } + ``` The `UserConfigFunction` function will log the string `"The word of the day is verdure"` every time the function is invoked (which means every time a message arrives). The `word-of-the-day` user config will be changed only when the function is updated with a new config value via the command line. You can also access the entire user config map or set a default value in case no value is present: - ```java + // Get the whole config map Map allConfigs = context.getUserConfigMap(); // Get value or resort to default String wotd = context.getUserConfigValueOrDefault("word-of-the-day", "perspicacious"); + ``` > For all key/value pairs passed to Java functions, both the key *and* the value are `String`. To set the value to be a different type, you need to deserialize from the `String` type. -In Python function, you can access the configuration value like this. +In Python function, you can access the configuration value like this. ```python + from pulsar import Function class WordFilter(Function): @@ -690,21 +734,23 @@ class WordFilter(Function): # Otherwise publish the message else: return input + ``` The Python SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. - ```bash + $ bin/pulsar-admin functions create \ # Other function configs \ --user-config '{"word-of-the-day":"verdure"}' + ``` To access that value in a Python function: - ```python + from pulsar import Function class UserConfigFunction(Function): @@ -715,23 +761,26 @@ class UserConfigFunction(Function): logger.warn('No word of the day provided') else: logger.info("The word of the day is {0}".format(wotd)) + ``` + The Go SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. - ```bash + $ bin/pulsar-admin functions create \ --go path/to/go/binary --user-config '{"word-of-the-day":"lackadaisical"}' + ``` To access that value in a Go function: - ```go + func contextFunc(ctx context.Context) { fc, ok := pf.FromContext(ctx) if !ok { @@ -746,6 +795,7 @@ func contextFunc(ctx context.Context) { logutil.Infof("The word of the day is %s", wotd.(string)) } } + ``` @@ -771,10 +821,11 @@ func contextFunc(ctx context.Context) { } ]}> -Pulsar Functions that use the Java SDK have access to an [SLF4j](https://www.slf4j.org/) [`Logger`](https://www.slf4j.org/api/org/apache/log4j/Logger.html) object that can be used to produce logs at the chosen log level. The following example logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. +Pulsar Functions that use the Java SDK have access to an [SLF4j](https://www.slf4j.org/) [`Logger`](https://www.slf4j.org/api/org/apache/log4j/Logger.html) object that can be used to produce logs at the chosen log level. The following example logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -794,17 +845,19 @@ public class LoggingFunction implements Function { return null; } } + ``` If you want your function to produce logs, you need to specify a log topic when creating or running the function. The following is an example. - ```bash + $ bin/pulsar-admin functions create \ --jar my-functions.jar \ --classname my.package.LoggingFunction \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` All logs produced by `LoggingFunction` above can be accessed via the `persistent://public/default/logging-function-logs` topic. @@ -813,8 +866,8 @@ All logs produced by `LoggingFunction` above can be accessed via the `persistent Additionally, you can use the XML file, `functions_log4j2.xml`, to customize the function log level. To customize the function log level, create or update `functions_log4j2.xml` in your Pulsar conf directory (for example, `/etc/pulsar/` on bare-metal, or `/pulsar/conf` on Kubernetes) to contain contents such as: - ```xml + pulsar-functions-instance 30 @@ -837,7 +890,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in Console SYSTEM_OUT - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -846,7 +899,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in ${sys:pulsar.function.log.dir}/${sys:pulsar.function.log.file}-%d{MM-dd-yyyy}-%i.log.gz true - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -879,7 +932,7 @@ To customize the function log level, create or update `functions_log4j2.xml` in ${sys:pulsar.function.log.dir}/${sys:pulsar.function.log.file}.bk-%d{MM-dd-yyyy}-%i.log.gz true - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n @@ -925,19 +978,24 @@ To customize the function log level, create or update `functions_log4j2.xml` in + ``` The properties set like: ```xml + pulsar.log.level debug + ``` + propagate to places where they are referenced, such as: ```xml + ${sys:pulsar.log.level} @@ -945,12 +1003,14 @@ propagate to places where they are referenced, such as: ${sys:pulsar.log.level} + ``` + In the above example, debug level logging would be applied to ALL function logs. This may be more verbose than you desire. To be more selective, you can apply different log levels to different classes or modules. For example: - ```xml + com.example.module info @@ -959,10 +1019,13 @@ This may be more verbose than you desire. To be more selective, you can apply di ${sys:pulsar.log.appender} + ``` + You can be more specific as well, such as applying a more verbose log level to a class in the module, such as: ```xml + com.example.module.className debug @@ -971,6 +1034,7 @@ You can be more specific as well, such as applying a more verbose log level to a Console + ``` Each `` entry allows you to output the log to a target specified in the definition of the Appender. @@ -979,28 +1043,34 @@ Additivity pertains to whether log messages will be duplicated if multiple Logge To disable additivity, specify ```xml + false + ``` + as shown in examples above. Disabling additivity prevents duplication of log messages when one or more `` entries contain classes or modules that overlap. The `` is defined in the `` section, such as: ```xml + Console SYSTEM_OUT - %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} - %msg%n + %d{ISO8601_OFFSET_DATE_TIME_HHMM} [%t] %-5level %logger{36} - %msg%n + ``` -Pulsar Functions that use the Python SDK have access to a logging object that can be used to produce logs at the chosen log level. The following example function that logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. +Pulsar Functions that use the Python SDK have access to a logging object that can be used to produce logs at the chosen log level. The following example function that logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```python + from pulsar import Function class LoggingFunction(Function): @@ -1011,17 +1081,19 @@ class LoggingFunction(Function): logger.warn("A warning was received in message {0}".format(context.get_message_id())) else: logger.info("Message {0} received\nContent: {1}".format(msg_id, input)) + ``` If you want your function to produce logs on a Pulsar topic, you need to specify a **log topic** when creating or running the function. The following is an example. - ```bash + $ bin/pulsar-admin functions create \ --py logging_function.py \ --classname logging_function.LoggingFunction \ --log-topic logging-function-logs \ # Other function configs + ``` All logs produced by `LoggingFunction` above can be accessed via the `logging-function-logs` topic. @@ -1029,10 +1101,11 @@ Additionally, you can specify the function log level through the broker XML file -The following Go Function example shows different log levels based on the function input. +The following Go Function example shows different log levels based on the function input. ``` + import ( "context" @@ -1052,6 +1125,7 @@ func loggerFunc(ctx context.Context, input []byte) { func main() { pf.Start(loggerFunc) } + ``` When you use `logTopic` related functionalities in Go Function, import `github.com/apache/pulsar/pulsar-function-go/logutil`, and you do not have to use the `getLogger()` context object. @@ -1078,8 +1152,8 @@ Pulsar Functions using the Java SDK has access to the Pulsar admin client, which Below is an example of how to use the Pulsar admin client exposed from the Function `context`. - ``` + import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1105,17 +1179,19 @@ public class CursorManagementFunction implements Function { return null; } } + ``` If you want your function to get access to the Pulsar admin client, you need to enable this feature by setting `exposeAdminClientEnabled=true` in the `functions_worker.yml` file. You can test whether this feature is enabled or not using the command `pulsar-admin functions localrun` with the flag `--web-service-url`. - ``` + $ bin/pulsar-admin functions localrun \ --jar my-functions.jar \ --classname my.package.CursorManagementFunction \ --web-service-url http://pulsar-web-service:8080 \ # Other function configs + ``` @@ -1144,10 +1220,11 @@ Pulsar Functions can publish arbitrary metrics to the metrics interface which ca } ]}> -You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. +You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1165,16 +1242,18 @@ public class MetricRecorderFunction implements Function { return null; } } + ``` -> For instructions on reading and using metrics, see the [Monitoring](deploy-monitoring.md) guide. +> For instructions on reading and using metrics, see the [Monitoring](deploy-monitoring) guide. -You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. The following is an example. +You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. The following is an example. ```python + from pulsar import Function class MetricRecorderFunction(Function): @@ -1183,9 +1262,12 @@ class MetricRecorderFunction(Function): if input == 11: context.record_metric('elevens-count', 1) + ``` + + Currently, the feature is not available in Go. @@ -1193,11 +1275,11 @@ Currently, the feature is not available in Go. ### Access metrics -To access metrics created by Pulsar Functions, refer to [Monitoring](deploy-monitoring.md) in Pulsar. +To access metrics created by Pulsar Functions, refer to [Monitoring](deploy-monitoring) in Pulsar. ## Security -If you want to enable security on Pulsar Functions, first you should enable security on [Functions Workers](functions-worker.md). For more details, refer to [Security settings](functions-worker.md#security-settings). +If you want to enable security on Pulsar Functions, first you should enable security on [Functions Workers](functions-worker). For more details, refer to [Security settings](functions-worker.md#security-settings). Pulsar Functions can support the following providers: @@ -1225,10 +1307,11 @@ At the same time, Pulsar Functions provides two interfaces, **SecretsProvider** } ]}> -You can get secret provider using the [`Context`](#context) object. The following is an example: +You can get secret provider using the [`Context`](#context) object. The following is an example: ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -1249,14 +1332,16 @@ public class GetSecretProviderFunction implements Function { return null; } } + ``` -You can get secret provider using the [`Context`](#context) object. The following is an example: +You can get secret provider using the [`Context`](#context) object. The following is an example: ```python + from pulsar import Function class GetSecretProviderFunction(Function): @@ -1267,11 +1352,12 @@ class GetSecretProviderFunction(Function): logger.warn('No secret provider') else: logger.info("The secret provider is {0}".format(secret_provider)) -``` +``` + Currently, the feature is not available in Go. @@ -1287,8 +1373,11 @@ States are key-value pairs, where the key is a string and the value is arbitrary You can access states within Pulsar Java Functions using the `putState`, `putStateAsync`, `getState`, `getStateAsync`, `incrCounter`, `incrCounterAsync`, `getCounter`, `getCounterAsync` and `deleteState` calls on the context object. You can access states within Pulsar Python Functions using the `putState`, `getState`, `incrCounter`, `getCounter` and `deleteState` calls on the context object. You can also manage states using the [querystate](#query-state) and [putstate](#putstate) options to `pulsar-admin functions`. -> Note -> State storage is not available in Go. +:::note + +State storage is not available in Go. + +::: ### API @@ -1305,26 +1394,28 @@ You can access states within Pulsar Java Functions using the `putState`, `putSta } ]}> + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](functions-develop.md#context) object when you are using Java SDK functions. #### incrCounter - ```java + /** * Increment the builtin distributed counter referred by key * @param key The name of the key * @param amount The amount to be incremented */ void incrCounter(String key, long amount); + ``` The application can use `incrCounter` to change the counter of a given `key` by the given `amount`. #### incrCounterAsync - ```java + /** * Increment the builtin distributed counter referred by key * but dont wait for the completion of the increment operation @@ -1333,14 +1424,15 @@ The application can use `incrCounter` to change the counter of a given `key` by * @param amount The amount to be incremented */ CompletableFuture incrCounterAsync(String key, long amount); + ``` The application can use `incrCounterAsync` to asynchronously change the counter of a given `key` by the given `amount`. #### getCounter - ```java + /** * Retrieve the counter value for the key. * @@ -1348,6 +1440,7 @@ The application can use `incrCounterAsync` to asynchronously change the counter * @return the amount of the counter value for this key */ long getCounter(String key); + ``` The application can use `getCounter` to retrieve the counter of a given `key` mutated by `incrCounter`. @@ -1357,8 +1450,8 @@ general key/value state. #### getCounterAsync - ```java + /** * Retrieve the counter value for the key, but don't wait * for the operation to be completed @@ -1367,14 +1460,15 @@ general key/value state. * @return the amount of the counter value for this key */ CompletableFuture getCounterAsync(String key); + ``` The application can use `getCounterAsync` to asynchronously retrieve the counter of a given `key` mutated by `incrCounterAsync`. #### putState - ```java + /** * Update the state value for the key. * @@ -1382,12 +1476,13 @@ The application can use `getCounterAsync` to asynchronously retrieve the counter * @param value state value of the key */ void putState(String key, ByteBuffer value); + ``` #### putStateAsync - ```java + /** * Update the state value for the key, but don't wait for the operation to be completed * @@ -1395,14 +1490,15 @@ The application can use `getCounterAsync` to asynchronously retrieve the counter * @param value state value of the key */ CompletableFuture putStateAsync(String key, ByteBuffer value); + ``` The application can use `putStateAsync` to asynchronously update the state of a given `key`. #### getState - ```java + /** * Retrieve the state value for the key. * @@ -1410,12 +1506,13 @@ The application can use `putStateAsync` to asynchronously update the state of a * @return the state value for the key. */ ByteBuffer getState(String key); + ``` #### getStateAsync - ```java + /** * Retrieve the state value for the key, but don't wait for the operation to be completed * @@ -1423,33 +1520,37 @@ The application can use `putStateAsync` to asynchronously update the state of a * @return the state value for the key. */ CompletableFuture getStateAsync(String key); + ``` The application can use `getStateAsync` to asynchronously retrieve the state of a given `key`. #### deleteState - ```java + /** * Delete the state value for the key. * * @param key name of the key */ + ``` Counters and binary values share the same keyspace, so this deletes either type. + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](#context) object when you are using Python SDK functions. #### incr_counter - ```python + def incr_counter(self, key, amount): - """incr the counter of a given key in the managed state""" + ""incr the counter of a given key in the managed state"" + ``` Application can use `incr_counter` to change the counter of a given `key` by the given `amount`. @@ -1457,10 +1558,11 @@ If the `key` does not exist, a new key is created. #### get_counter - ```python + def get_counter(self, key): """get the counter of a given key in the managed state""" + ``` Application can use `get_counter` to retrieve the counter of a given `key` mutated by `incrCounter`. @@ -1470,28 +1572,31 @@ general key/value state. #### put_state - ```python + def put_state(self, key, value): """update the value of a given key in the managed state""" + ``` The key is a string, and the value is arbitrary binary data. #### get_state - ```python + def get_state(self, key): """get the value of a given key in the managed state""" + ``` #### del_counter - ```python + def del_counter(self, key): """delete the counter of a given key in the managed state""" + ``` Counters and binary values share the same keyspace, so this deletes either type. @@ -1506,8 +1611,8 @@ A Pulsar Function can use the [State API](#api) for storing state into Pulsar's and retrieving state back from Pulsar's state storage. Additionally Pulsar also provides CLI commands for querying its state. - ```shell + $ bin/pulsar-admin functions querystate \ --tenant \ --namespace \ @@ -1515,6 +1620,7 @@ $ bin/pulsar-admin functions querystate \ --state-storage-url \ --key \ [---watch] + ``` If `--watch` is specified, the CLI will watch the value of the provided `state-key`. @@ -1535,11 +1641,11 @@ If `--watch` is specified, the CLI will watch the value of the provided `state-k ]}> -{@inject: github:`WordCountFunction`:/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/WordCountFunction.java} is a very good example +{@inject: github:WordCountFunction:/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/WordCountFunction.java} is a very good example demonstrating on how Application can easily store `state` in Pulsar Functions. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1552,6 +1658,7 @@ public class WordCountFunction implements Function { return null; } } + ``` The logic of this `WordCount` function is pretty simple and straightforward: @@ -1562,14 +1669,15 @@ The logic of this `WordCount` function is pretty simple and straightforward: - ```python + from pulsar import Function class WordCount(Function): def process(self, item, context): for word in item.split(): context.incr_counter(word, 1) + ``` The logic of this `WordCount` function is pretty simple and straightforward: diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-metrics.md b/site2/website-next/versioned_docs/version-2.7.3/functions-metrics.md index 6812ec7bb411c..2fe073599c626 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-metrics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-metrics.md @@ -1,7 +1,7 @@ --- id: functions-metrics title: Metrics for Pulsar Functions -sidebar_label: Metrics +sidebar_label: "Metrics" original_id: functions-metrics --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-overview.md b/site2/website-next/versioned_docs/version-2.7.3/functions-overview.md index 5d2d41bd7f0d6..9e92f5ac71803 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-overview.md @@ -1,7 +1,7 @@ --- id: functions-overview title: Pulsar Functions overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: functions-overview --- @@ -60,8 +60,8 @@ If you implement the classic word count example using Pulsar Functions, it looks To write the function in Java with [Pulsar Functions SDK for Java](functions-develop.md#available-apis), you can write the function as follows. - ```java + package org.example.functions; import org.apache.pulsar.functions.api.Context; @@ -80,12 +80,13 @@ public class WordCountFunction implements Function { return null; } } + ``` Bundle and build the JAR file to be deployed, and then deploy it in your Pulsar cluster using the [command line](functions-deploy.md#command-line-interface) as follows. - ```bash + $ bin/pulsar-admin functions create \ --jar target/my-jar-with-dependencies.jar \ --classname org.example.functions.WordCountFunction \ @@ -94,6 +95,7 @@ $ bin/pulsar-admin functions create \ --name word-count \ --inputs persistent://public/default/sentences \ --output persistent://public/default/count + ``` ### Content-based routing example @@ -106,8 +108,8 @@ For example, a function takes items (strings) as input and publishes them to eit If you implement this routing functionality in Python, it looks something like this: - ```python + from pulsar import Function class RoutingFunction(Function): @@ -131,12 +133,13 @@ class RoutingFunction(Function): else: warning = "The item {0} is neither a fruit nor a vegetable".format(item) context.get_logger().warn(warning) + ``` If this code is stored in `~/router.py`, then you can deploy it in your Pulsar cluster using the [command line](functions-deploy.md#command-line-interface) as follows. - ```bash + $ bin/pulsar-admin functions create \ --py ~/router.py \ --classname router.RoutingFunction \ @@ -144,6 +147,7 @@ $ bin/pulsar-admin functions create \ --namespace default \ --name route-fruit-veg \ --inputs persistent://public/default/basket-items + ``` ### Functions, messages and message types @@ -155,15 +159,16 @@ Pulsar Functions take byte arrays as inputs and spit out byte arrays as output. ## Fully Qualified Function Name (FQFN) Each Pulsar Function has a **Fully Qualified Function Name** (FQFN) that consists of three elements: the function tenant, namespace, and function name. FQFN looks like this: - ```http + tenant/namespace/name + ``` FQFNs enable you to create multiple functions with the same name provided that they are in different namespaces. ## Supported languages -Currently, you can write Pulsar Functions in Java, Python, and Go. For details, refer to [Develop Pulsar Functions](functions-develop.md). +Currently, you can write Pulsar Functions in Java, Python, and Go. For details, refer to [Develop Pulsar Functions](functions-develop). ## Processing guarantees Pulsar Functions provide three different messaging semantics that you can apply to any function. @@ -178,12 +183,13 @@ Delivery semantics | Description ### Apply processing guarantees to a function You can set the processing guarantees for a Pulsar Function when you create the Function. The following [`pulsar-function create`](reference-pulsar-admin.md#create-1) command creates a function with effectively-once guarantees applied. - ```bash + $ bin/pulsar-admin functions create \ --name my-effectively-once-function \ --processing-guarantees EFFECTIVELY_ONCE \ # Other function configs + ``` The available options for `--processing-guarantees` are: @@ -197,9 +203,11 @@ The available options for `--processing-guarantees` are: ### Update the processing guarantees of a function You can change the processing guarantees applied to a function using the [`update`](reference-pulsar-admin.md#update-1) command. The following is an example. - ```bash + $ bin/pulsar-admin functions update \ --processing-guarantees ATMOST_ONCE \ # Other function configs + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-package.md b/site2/website-next/versioned_docs/version-2.7.3/functions-package.md index 07e497cb5c998..3e1ccfeabbba4 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-package.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-package.md @@ -11,12 +11,15 @@ import TabItem from '@theme/TabItem'; You can package Pulsar functions in Java, Python, and Go. Packaging the window function in Java is the same as [packaging a function in Java](#java). -> **Note** -> Currently, the window function is not available in Python and Go. +:::note + +Currently, the window function is not available in Python and Go. + +::: ## Prerequisite -Before running a Pulsar function, you need to start Pulsar. You can [run a standalone Pulsar in Docker](getting-started-docker.md), or [run Pulsar in Kubernetes](getting-started-helm.md). +Before running a Pulsar function, you need to start Pulsar. You can [run a standalone Pulsar in Docker](getting-started-docker.md), or [run Pulsar in Kubernetes](getting-started-helm). To check whether the Docker image starts, you can use the `docker ps` command. @@ -26,151 +29,158 @@ To package a function in Java, complete the following steps. 1. Create a new maven project with a pom file. In the following code sample, the value of `mainClass` is your package name. - -```Java - - - 4.0.0 - - java-function - java-function - 1.0-SNAPSHOT - - - - org.apache.pulsar - pulsar-functions-api - 2.6.0 - - - - - - - maven-assembly-plugin - - false - - jar-with-dependencies - - - - org.example.test.ExclamationFunction - - - - - - make-assembly - package - - assembly - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - 8 - 8 - - - - - - -``` + ```Java + + + + 4.0.0 + + java-function + java-function + 1.0-SNAPSHOT + + + + org.apache.pulsar + pulsar-functions-api + 2.6.0 + + + + + + + maven-assembly-plugin + + false + + jar-with-dependencies + + + + org.example.test.ExclamationFunction + + + + + + make-assembly + package + + assembly + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + + + ``` 2. Write a Java function. + ``` + + package org.example.test; -``` - package org.example.test; - - import java.util.function.Function; + import java.util.function.Function; - public class ExclamationFunction implements Function { - @Override - public String apply(String s) { - return "This is my function!"; - } - } -``` - - For the imported package, you can use one of the following interfaces: - - Function interface provided by Java 8: `java.util.function.Function` - - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` - - The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. - - The following example uses `org.apache.pulsar.functions.api.Function` interface with context. - - -``` - package org.example.functions; - import org.apache.pulsar.functions.api.Context; - import org.apache.pulsar.functions.api.Function; - - import java.util.Arrays; - public class WordCountFunction implements Function { - // This function is invoked every time a message is published to the input topic - @Override - public Void process(String input, Context context) throws Exception { - Arrays.asList(input.split(" ")).forEach(word -> { - String counterKey = word.toLowerCase(); - context.incrCounter(counterKey, 1); - }); - return null; + public class ExclamationFunction implements Function { + @Override + public String apply(String s) { + return "This is my function!"; } - } -``` + } + + ``` + + For the imported package, you can use one of the following interfaces: + - Function interface provided by Java 8: `java.util.function.Function` + - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` + + The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. + + The following example uses `org.apache.pulsar.functions.api.Function` interface with context. + + ``` + + package org.example.functions; + import org.apache.pulsar.functions.api.Context; + import org.apache.pulsar.functions.api.Function; + + import java.util.Arrays; + public class WordCountFunction implements Function { + // This function is invoked every time a message is published to the input topic + @Override + public Void process(String input, Context context) throws Exception { + Arrays.asList(input.split(" ")).forEach(word -> { + String counterKey = word.toLowerCase(); + context.incrCounter(counterKey, 1); + }); + return null; + } + } + + ``` 3. Package the Java function. + ```bash + + mvn package + + ``` -```bash - mvn package -``` - - After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. + After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. 4. Run the Java function. - (1) Copy the packaged jar file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Java function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --jar java-function-1.0-SNAPSHOT.jar \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name JavaFunction -``` - - The following log indicates that the Java function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the packaged jar file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Java function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname org.example.test.ExclamationFunction \ + --jar java-function-1.0-SNAPSHOT.jar \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name JavaFunction + + ``` + + The following log indicates that the Java function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Python @@ -186,69 +196,74 @@ To package a function with **one python file** in Python, complete the following 1. Write a Python function. + ``` + + from pulsar import Function // import the Function module from Pulsar -``` - from pulsar import Function // import the Function module from Pulsar - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass - def process(self, input, context): - return input + '!' -``` + def process(self, input, context): + return input + '!' + + ``` - In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. + In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. - `process()` mainly has two parameters: + `process()` mainly has two parameters: - - `input` represents your input. + - `input` represents your input. - - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. + - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. 2. Install a Python client. - The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. + The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. - -```bash - pip install python-client==2.6.0 -``` + ```bash + + pip install pulsar-client==2.6.0 + + ``` 3. Run the Python Function. - (1) Copy the Python function file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Python function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --py \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name PythonFunction -``` - - The following log indicates that the Python function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the Python function file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname . \ + --py \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` + + The following log indicates that the Python function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### ZIP file @@ -256,58 +271,64 @@ To package a function with the **ZIP file** in Python, complete the following st 1. Prepare the ZIP file. - The following is required when packaging the ZIP file of the Python Function. - - -```text - Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: - "func/src" - "func/requirements.txt" - "func/deps" -``` - Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. - - -```text - . - ├── deps - │   └── sh-1.12.14-py2.py3-none-any.whl - └── src - └── exclamation.py -``` + The following is required when packaging the ZIP file of the Python Function. + + ```text + + Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: + "func/src" + "func/requirements.txt" + "func/deps" + + ``` + + Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. + + ```text + + . + ├── deps + │   └── sh-1.12.14-py2.py3-none-any.whl + └── src + └── exclamation.py + + ``` 2. Run the Python Function. - (1) Copy the ZIP file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Python function using the following command. - - -```bash - ./bin/pulsar-admin functions localrun \ - --classname exclamation \ - --py \ - --inputs persistent://public/default/in-topic \ - --output persistent://public/default/out-topic \ - --tenant public \ - --namespace default \ - --name PythonFunction -``` - - The following log indicates that the Python function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the ZIP file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname exclamation \ + --py \ + --inputs persistent://public/default/in-topic \ + --output persistent://public/default/out-topic \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` + + The following log indicates that the Python function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### PIP @@ -315,50 +336,54 @@ The PIP method is only supported in Kubernetes runtime. To package a function wi 1. Configure the `functions_worker.yml` file. - -```text - #### Kubernetes Runtime #### - installUserCodeDependencies: true -``` + ```text + + #### Kubernetes Runtime #### + installUserCodeDependencies: true + + ``` 2. Write your Python Function. + ``` + + from pulsar import Function + import js2xml -``` - from pulsar import Function - import js2xml - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass - def process(self, input, context): - // add your logic - return input + '!' -``` + def process(self, input, context): + // add your logic + return input + '!' + + ``` - You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. + You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. 3. Generate the `whl` file. - -```shell script - $ cd $PULSAR_HOME/pulsar-functions/scripts/python - $ chmod +x generate.sh - $ ./generate.sh - # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 -``` - - The output is written in `/path/to/python/output`: - - -```text - -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl - -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz - -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl -``` + ```shell script + + $ cd $PULSAR_HOME/pulsar-functions/scripts/python + $ chmod +x generate.sh + $ ./generate.sh + # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 + + ``` + + The output is written in `/path/to/python/output`: + + ```text + + -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl + -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz + -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl + + ``` ## Go @@ -366,99 +391,107 @@ To package a function in Go, complete the following steps. 1. Write a Go function. - Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". + Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". + ``` + + import ( + "context" + "fmt" -``` - import ( - "context" - "fmt" + "github.com/apache/pulsar/pulsar-function-go/pf" + ) - "github.com/apache/pulsar/pulsar-function-go/pf" - ) + func HandleRequest(ctx context.Context, input []byte) error { + fmt.Println(string(input) + "!") + return nil + } - func HandleRequest(ctx context.Context, input []byte) error { - fmt.Println(string(input) + "!") - return nil - } + func main() { + pf.Start(HandleRequest) + } + + ``` + + You can use context to connect to the Go function. - func main() { - pf.Start(HandleRequest) + ``` + + if fc, ok := pf.FromContext(ctx); ok { + fmt.Printf("function ID is:%s, ", fc.GetFuncID()) + fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) } -``` - - You can use context to connect to the Go function. - - -``` - if fc, ok := pf.FromContext(ctx); ok { - fmt.Printf("function ID is:%s, ", fc.GetFuncID()) - fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) - } -``` - - When writing a Go function, remember that - - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. - - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: - - -``` - func () - func () error - func (input) error - func () (output, error) - func (input) (output, error) - func (context.Context) error - func (context.Context, input) error - func (context.Context) (output, error) - func (context.Context, input) (output, error) -``` + + ``` + + When writing a Go function, remember that + - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. + - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: + + ``` + + func () + func () error + func (input) error + func () (output, error) + func (input) (output, error) + func (context.Context) error + func (context.Context, input) error + func (context.Context) (output, error) + func (context.Context, input) (output, error) + + ``` 2. Build the Go function. - -``` - go build .go -``` + ``` + + go build .go + + ``` 3. Run the Go Function. - (1) Copy the Go function file to the Pulsar image. - - -```bash - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar -``` - - (2) Run the Go function with the following command. - - -``` - ./bin/pulsar-admin functions localrun \ - --go [your go function path] - --inputs [input topics] \ - --output [output topic] \ - --tenant [default:public] \ - --namespace [default:default] \ - --name [custom unique go function name] -``` - - The following log indicates that the Go function starts successfully. - - -```text - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... -``` + (1) Copy the Go function file to the Pulsar image. + + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` + + (2) Run the Go function with the following command. + + ``` + + ./bin/pulsar-admin functions localrun \ + --go [your go function path] + --inputs [input topics] \ + --output [output topic] \ + --tenant [default:public] \ + --namespace [default:default] \ + --name [custom unique go function name] + + ``` + + The following log indicates that the Go function starts successfully. + + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Start Functions in cluster mode If you want to start a function in cluster mode, replace `localrun` with `create` in the commands above. The following log indicates that your function starts successfully. - -```text + ```text + "Created successfully" -``` + + ``` For information about parameters on `--classname`, `--jar`, `--py`, `--go`, `--inputs`, run the command `./bin/pulsar-admin functions` or see [here](reference-pulsar-admin.md#functions). diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-runtime.md b/site2/website-next/versioned_docs/version-2.7.3/functions-runtime.md index f9e1891f3ae60..9621cd6c2e2ab 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-runtime.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-runtime.md @@ -15,8 +15,11 @@ You can use the following methods to run functions. - *Process*: Invoke functions in processes forked by functions worker. - *Kubernetes*: Submit functions as Kubernetes StatefulSets by functions worker. -> **Note** -> Pulsar supports adding labels to the Kubernetes StatefulSets and services while launching functions, which facilitates selecting the target Kubernetes objects. +:::note + +Pulsar supports adding labels to the Kubernetes StatefulSets and services while launching functions, which facilitates selecting the target Kubernetes objects. + +::: The differences of the thread and process modes are: - Thread mode: when a function runs in thread mode, it runs on the same Java virtual machine (JVM) with functions worker. @@ -25,11 +28,12 @@ The differences of the thread and process modes are: ## Configure thread runtime It is easy to configure *Thread* runtime. In most cases, you do not need to configure anything. You can customize the thread group name with the following settings: - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory functionRuntimeFactoryConfigs: threadGroupName: "Your Function Container Group" + ``` *Thread* runtime is only supported in Java function. @@ -37,8 +41,8 @@ functionRuntimeFactoryConfigs: ## Configure process runtime When you enable *Process* runtime, you do not need to configure anything. - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.process.ProcessRuntimeFactory functionRuntimeFactoryConfigs: # the directory for storing the function logs @@ -49,6 +53,7 @@ functionRuntimeFactoryConfigs: pythonInstanceLocation: # change the extra dependencies location: extraFunctionDependenciesDir: + ``` *Process* runtime is supported in Java, Python, and Go functions. @@ -61,16 +66,18 @@ The manifests, generated by the functions worker, include a `StatefulSet`, a `Se The Kubernetes runtime supports secrets, so you can create a Kubernetes secret and expose it as an environment variable in the pod. The Kubernetes runtime is extensible, you can implement classes and customize the way how to generate Kubernetes manifests, how to pass auth data to pods, and how to integrate secrets. -> **Tip** -> -> For the rules of translating Pulsar object names into Kubernetes resource labels, see [here](admin-api-overview.md#how-to-define-pulsar-resource-names-when-running-pulsar-in-kubernetes). +:::tip + +For the rules of translating Pulsar object names into Kubernetes resource labels, see [here](admin-api-overview.md#how-to-define-pulsar-resource-names-when-running-pulsar-in-kubernetes). + +::: ### Basic configuration It is easy to configure Kubernetes runtime. You can just uncomment the settings of `kubernetesContainerFactory` in the `functions_worker.yaml` file. The following is an example. - ```yaml + functionRuntimeFactoryClassName: org.apache.pulsar.functions.runtime.kubernetes.KubernetesRuntimeFactory functionRuntimeFactoryConfigs: # uri to kubernetes cluster, leave it to empty and it will use the kubernetes settings in function worker @@ -107,6 +114,7 @@ functionRuntimeFactoryConfigs: extraFunctionDependenciesDir: # Additional memory padding added on top of the memory requested by the function per on a per instance basis percentMemoryPadding: 10 + ``` If you run functions worker embedded in a broker on Kubernetes, you can use the default settings. @@ -117,10 +125,11 @@ If you run functions worker standalone (that is, not embedded) on Kubernetes, yo For example, both Pulsar brokers and Function Workers run in the `pulsar` K8S namespace. The brokers have a service called `brokers` and the functions worker has a service called `func-worker`. The settings are as follows: - ```yaml + pulsarServiceUrl: pulsar://broker.pulsar:6650 // or pulsar+ssl://broker.pulsar:6651 if using TLS pulsarAdminUrl: http://func-worker.pulsar:8080 // or https://func-worker:8443 if using TLS + ``` ### Run RBAC in Kubernetes clusters @@ -134,8 +143,8 @@ If you run RBAC in your Kubernetes cluster, make sure that the service account y The following is sufficient: - ```yaml + apiVersion: rbac.authorization.k8s.io/v1beta1 kind: ClusterRole metadata: @@ -171,11 +180,13 @@ roleRef: subjectsKubernetesSec: - kind: ServiceAccount name: functions-worker + ``` If the service-account is not properly configured, an error message similar to this is displayed: ```bash + 22:04:27.696 [Timer-0] ERROR org.apache.pulsar.functions.runtime.KubernetesRuntimeFactory - Error while trying to fetch configmap example-pulsar-4qvmb5gur3c6fc9dih0x1xn8b-function-worker-config at namespace pulsar io.kubernetes.client.ApiException: Forbidden at io.kubernetes.client.ApiClient.handleResponse(ApiClient.java:882) ~[io.kubernetes-client-java-2.0.0.jar:?] @@ -186,6 +197,7 @@ io.kubernetes.client.ApiException: Forbidden at org.apache.pulsar.functions.runtime.KubernetesRuntimeFactory$1.run(KubernetesRuntimeFactory.java:275) [org.apache.pulsar-pulsar-functions-runtime-2.4.0-42c3bf949.jar:2.4.0-42c3bf949] at java.util.TimerThread.mainLoop(Timer.java:555) [?:1.8.0_212] at java.util.TimerThread.run(Timer.java:505) [?:1.8.0_212] + ``` ### Integrate Kubernetes secrets @@ -194,8 +206,8 @@ In order to safely distribute secrets, Pulasr Functions can reference Kubernetes You can create a secret in the namespace where your functions are deployed. For example, you deploy functions to the `pulsar-func` Kubernetes namespace, and you have a secret named `database-creds` with a field name `password`, which you want to mount in the pod as an environment variable called `DATABASE_PASSWORD`. The following functions configuration enables you to reference that secret and mount the value as an environment variable in the pod. - ```Yaml + tenant: "mytenant" namespace: "mynamespace" name: "myfunction" @@ -218,17 +230,21 @@ The `org.apache.pulsar.functions.auth.KubernetesFunctionAuthProvider` interface Pulsar includes an implementation of this interface for token authentication, and distributes the certificate authority via the same implementation. The configuration is similar as follows: - ```Yaml + functionAuthProviderClassName: org.apache.pulsar.functions.auth.KubernetesSecretsTokenAuthProvider + ``` For token authentication, the functions worker captures the token that is used to deploy (or update) the function. The token is saved as a secret and mounted into the pod. For custom authentication or TLS, you need to implement this interface or use an alternative mechanism to provide authentication. If you use token authentication and TLS encryption to secure the communication with the cluster, Pulsar passes your certificate authority (CA) to the client, so the client obtains what it needs to authenticate the cluster, and trusts the cluster with your signed certificate. -> **Note** -> If you use tokens that expire when deploying functions, these tokens will expire. +:::note + +If you use tokens that expire when deploying functions, these tokens will expire. + +::: ### Run clusters with authentication @@ -236,8 +252,8 @@ When you run a functions worker in a standalone process (that is, not embedded i For example, if you use token authentication, you need to configure the following properties in the `function-worker.yml` file. - ```Yaml + clientAuthenticationPlugin: org.apache.pulsar.client.impl.auth.AuthenticationToken clientAuthenticationParameters: file:///etc/pulsar/token/admin-token.txt configurationStoreServers: zookeeper-cluster:2181 # auth requires a connection to zookeeper @@ -251,10 +267,14 @@ superUserRoles: properties: tokenSecretKey: file:///etc/pulsar/jwt/secret # if using a secret token tokenPublicKey: file:///etc/pulsar/jwt/public.key # if using public/private key tokens + ``` -> **Note** -> You must configure both the Function Worker authorization or authentication for the server to authenticate requests and configure the client to be authenticated to communicate with the broker. +:::note + +You must configure both the Function Worker authorization or authentication for the server to authenticate requests and configure the client to be authenticated to communicate with the broker. + +::: ### Customize Kubernetes runtime @@ -268,8 +288,8 @@ Pulsar includes a built-in implementation. To use the basic implementation, set Below is an example of `customRuntimeOptions`. - ```json + { "jobName": "jobname", // the k8s pod name to run this function instance "jobNamespace": "namespace", // the k8s namespace to run this function in @@ -300,6 +320,7 @@ Below is an example of `customRuntimeOptions`. } } } + ``` ## Run clusters with geo-replication @@ -308,16 +329,18 @@ If you run multiple clusters tied together with geo-replication, it is important For example, if you have two clusters: `east-1` and `west-1`, you can configure the functions workers for `east-1` and `west-1` perspectively as follows. - ```Yaml + pulsarFunctionsCluster: east-1 pulsarFunctionsNamespace: public/functions-east-1 -``` +``` ```Yaml + pulsarFunctionsCluster: west-1 pulsarFunctionsNamespace: public/functions-west-1 + ``` This ensures the two different Functions Workers use distinct sets of topics for their internal coordination. @@ -328,8 +351,8 @@ When configuring a standalone functions worker, you need to configure properties You need to configure the following required properties. - ```Yaml + workerPort: 8080 workerPortTls: 8443 # when using TLS tlsCertificateFilePath: /etc/pulsar/tls/tls.crt # when using TLS @@ -340,3 +363,4 @@ pulsarWebServiceUrl: http://broker.pulsar:8080/ # or https://pulsar-prod-broker. useTls: true # when using TLS, critical! ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/functions-worker.md b/site2/website-next/versioned_docs/version-2.7.3/functions-worker.md index 3a4e1720570e6..edb8d962fb049 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/functions-worker.md +++ b/site2/website-next/versioned_docs/version-2.7.3/functions-worker.md @@ -8,14 +8,17 @@ original_id: functions-worker import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -Before using Pulsar Functions, you need to learn how to set up Pulsar Functions worker and how to [configure Functions runtime](functions-runtime.md). +Before using Pulsar Functions, you need to learn how to set up Pulsar Functions worker and how to [configure Functions runtime](functions-runtime). Pulsar `functions-worker` is a logic component to run Pulsar Functions in cluster mode. Two options are available, and you can select either based on your requirements. - [run with brokers](#run-functions-worker-with-brokers) - [run it separately](#run-functions-worker-separately) in a different broker -> Note -> The `--- Service Urls---` lines in the following diagrams represent Pulsar service URLs that Pulsar client and admin use to connect to a Pulsar cluster. +:::note + +The `--- Service Urls---` lines in the following diagrams represent Pulsar service URLs that Pulsar client and admin use to connect to a Pulsar cluster. + +::: ## Run Functions-worker with brokers @@ -25,9 +28,10 @@ The following diagram illustrates the deployment of functions-workers running al To enable functions-worker running as part of a broker, you need to set `functionsWorkerEnabled` to `true` in the `broker.conf` file. - ```conf + functionsWorkerEnabled=true + ``` If the `functionsWorkerEnabled` is set to `true`, the functions-worker is started as part of a broker. You need to configure the `conf/functions_worker.yml` file to customize your functions_worker. @@ -57,35 +61,41 @@ If you want to use Stateful-Functions related functions (for example, `putState Currently, the service uses the NAR package, so you need to set the configuration in `bookkeeper.conf`. - -```text - extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent -``` + ```text + + extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent + + ``` After starting bookie, use the following methods to check whether the streamStorage service is started correctly. Input: + ```shell + + telnet localhost 4181 + + ``` -```shell - telnet localhost 4181 -``` Output: -```text - Trying 127.0.0.1... - Connected to localhost. - Escape character is '^]'. -``` + ```text + + Trying 127.0.0.1... + Connected to localhost. + Escape character is '^]'. + + ``` 2. Turn on this function in `functions_worker.yml`. + ```text + + stateStorageServiceUrl: bk://:4181 + + ``` -```text - stateStorageServiceUrl: bk://:4181 -``` - - `bk-service-url` is the service URL pointing to the BookKeeper table service. + `bk-service-url` is the service URL pointing to the BookKeeper table service. ### Start Functions-worker with broker @@ -93,16 +103,18 @@ Once you have configured the `functions_worker.yml` file, you can start or resta And then you can use the following command to verify if `functions-worker` is running well. - ```bash + curl :8080/admin/v2/worker/cluster + ``` After entering the command above, a list of active function workers in the cluster is returned. The output is similar to the following. - ```json + [{"workerId":"","workerHostname":"","port":8080}] + ``` ## Run Functions-worker separately @@ -111,8 +123,11 @@ This section illustrates how to run `functions-worker` as a separate process in ![assets/functions-worker-separated.png](/assets/functions-worker-separated.png) -> Note -> In this mode, make sure `functionsWorkerEnabled` is set to `false`, so you won't start `functions-worker` with brokers by mistake. +:::note + +In this mode, make sure `functionsWorkerEnabled` is set to `false`, so you won't start `functions-worker` with brokers by mistake. + +::: ### Configure Functions-worker to run separately @@ -152,8 +167,8 @@ If you want to enable security on functions workers, you *should*: To enable TLS transport encryption, configure the following settings. - ``` + useTLS: true pulsarServiceUrl: pulsar+ssl://localhost:6651/ pulsarWebServiceUrl: https://localhost:8443 @@ -165,86 +180,97 @@ tlsTrustCertsFilePath: /path/to/ca.cert.pem // The path to trusted certificates used by the Pulsar client to authenticate with Pulsar brokers brokerClientTrustCertsFilePath: /path/to/ca.cert.pem + ``` -For details on TLS encryption, refer to [Transport Encryption using TLS](security-tls-transport.md). +For details on TLS encryption, refer to [Transport Encryption using TLS](security-tls-transport). ##### Enable Authentication Provider To enable authentication on Functions Worker, you need to configure the following settings. -> Note -> Substitute the *providers list* with the providers you want to enable. +:::note + +Substitute the *providers list* with the providers you want to enable. +::: ``` + authenticationEnabled: true authenticationProviders: [ provider1, provider2 ] + ``` For *TLS Authentication* provider, follow the example below to add the necessary settings. -See [TLS Authentication](security-tls-authentication.md) for more details. - +See [TLS Authentication](security-tls-authentication) for more details. ``` + brokerClientAuthenticationPlugin: org.apache.pulsar.client.impl.auth.AuthenticationTls brokerClientAuthenticationParameters: tlsCertFile:/path/to/admin.cert.pem,tlsKeyFile:/path/to/admin.key-pk8.pem authenticationEnabled: true authenticationProviders: ['org.apache.pulsar.broker.authentication.AuthenticationProviderTls'] + ``` For *SASL Authentication* provider, add `saslJaasClientAllowedIds` and `saslJaasBrokerSectionName` under `properties` if needed. - ``` + properties: saslJaasClientAllowedIds: .*pulsar.* saslJaasBrokerSectionName: Broker + ``` For *Token Authentication* provider, add necessary settings for `properties` if needed. -See [Token Authentication](security-jwt.md) for more details. +See [Token Authentication](security-jwt) for more details. ``` + properties: tokenSecretKey: file://my/secret.key # If using public/private - # tokenPublicKey: file:///path/to/public.key + # tokenPublicKey: file:///path/to/public.key + ``` ##### Enable Authorization Provider To enable authorization on Functions Worker, you need to configure `authorizationEnabled`, `authorizationProvider` and `configurationStoreServers`. The authentication provider connects to `configurationStoreServers` to receive namespace policies. - ```yaml + authorizationEnabled: true authorizationProvider: org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider configurationStoreServers: + ``` You should also configure a list of superuser roles. The superuser roles are able to access any admin API. The following is a configuration example. - ```yaml + superUserRoles: - role1 - role2 - role3 + ``` ##### Enable End-to-End Encryption You can use the public and private key pair that the application configures to perform encryption. Only the consumers with a valid key can decrypt the encrypted messages. -To enable End-to-End encryption on Functions Worker, you can set it by specifying `--producer-config` in the command line terminal, for more information, please refer to [here](security-encryption.md). +To enable End-to-End encryption on Functions Worker, you can set it by specifying `--producer-config` in the command line terminal, for more information, please refer to [here](security-encryption). We include the relevant configuration information of `CryptoConfig` into `ProducerConfig`. The specific configurable field information about `CryptoConfig` is as follows: - ```text + public class CryptoConfig { private String cryptoKeyReaderClassName; private Map cryptoKeyReaderConfig; @@ -254,6 +280,7 @@ public class CryptoConfig { private ConsumerCryptoFailureAction consumerCryptoFailureAction; } + ``` - `producerCryptoFailureAction`: define the action if producer fail to encrypt data one of `FAIL`, `SEND`. @@ -271,9 +298,10 @@ If authentication is enabled on the BookKeeper cluster, you need configure the B Once you have finished configuring the `functions_worker.yml` configuration file, you can use the following command to start a `functions-worker`: - ```bash + bin/pulsar functions-worker + ``` ### Configure Proxies for Functions-workers @@ -291,10 +319,11 @@ start proxies. To enable routing functions related admin requests to `functions-worker` in a proxy, you can edit the `proxy.conf` file to modify the following settings: - ```conf + functionWorkerWebServiceURL= functionWorkerWebServiceURLTLS= + ``` ## Compare the Run-with-Broker and Run-separately modes @@ -315,9 +344,10 @@ Use the `Run-separately` mode in the following cases: **Error message: Namespace missing local cluster name in clusters list** - ``` + Failed to get partitioned topic metadata: org.apache.pulsar.client.api.PulsarClientException$BrokerMetadataException: Namespace missing local cluster name in clusters list: local_cluster=xyz ns=public/functions clusters=[standalone] + ``` The error message prompts when either of the cases occurs: @@ -332,16 +362,18 @@ If any of these cases happens, follow the instructions below to fix the problem: 2. Get the current clusters list of `public/functions` namespace. - ```bash + bin/pulsar-admin namespaces get-clusters public/functions + ``` 3. Check if the cluster is in the clusters list. If the cluster is not in the list, add it to the list and update the clusters list. - ```bash + bin/pulsar-admin namespaces set-clusters --clusters , public/functions + ``` 4. After setting the cluster successfully, enable functions worker by setting `functionsWorkerEnabled=true`. diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-clients.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-clients.md index 7860f0a136aa5..4194e347e2a09 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-clients.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-clients.md @@ -1,7 +1,7 @@ --- id: client-libraries title: Pulsar client libraries -sidebar_label: Overview +sidebar_label: "Overview" original_id: client-libraries --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-concepts-and-architecture.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-concepts-and-architecture.md index e5131f15da9a1..be300de5ab86c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-concepts-and-architecture.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-concepts-and-architecture.md @@ -1,10 +1,14 @@ --- id: concepts-architecture title: Pulsar concepts and architecture -sidebar_label: Concepts and architecture +sidebar_label: "Concepts and architecture" original_id: concepts-architecture --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-docker.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-docker.md index 13b75d21799b0..2d4dc043786d7 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-docker.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-docker.md @@ -1,10 +1,14 @@ --- id: standalone-docker title: Set up a standalone Pulsar in Docker -sidebar_label: Run Pulsar in Docker +sidebar_label: "Run Pulsar in Docker" original_id: standalone-docker --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + For local development and testing, you can run Pulsar in standalone mode on your own machine within a Docker container. @@ -16,13 +20,15 @@ and follow the instructions for your OS. * For MacOS, Linux, and Windows: ```shell + $ docker run -it \ - -p 6650:6650 \ - -p 8080:8080 \ - --mount source=pulsardata,target=/pulsar/data \ - --mount source=pulsarconf,target=/pulsar/conf \ - apachepulsar/pulsar:{{pulsar:version}} \ - bin/pulsar standalone + -p 6650:6650 \ + -p 8080:8080 \ + --mount source=pulsardata,target=/pulsar/data \ + --mount source=pulsarconf,target=/pulsar/conf \ + apachepulsar/pulsar:@pulsar:version@ \ + bin/pulsar standalone + ``` A few things to note about this command: @@ -33,33 +39,40 @@ time the container is restarted. For details on the volumes you can use `docker If you start Pulsar successfully, you will see `INFO`-level log messages like this: ``` + 2017-08-09 22:34:04,030 - INFO - [main:WebService@213] - Web Service started at http://127.0.0.1:8080 2017-08-09 22:34:04,038 - INFO - [main:PulsarService@335] - messaging service is ready, bootstrap service on port=8080, broker url=pulsar://127.0.0.1:6650, cluster=standalone, configs=org.apache.pulsar.broker.ServiceConfiguration@4db60246 ... + ``` -> #### Tip -> -> When you start a local standalone cluster, a `public/default` +:::tip + +When you start a local standalone cluster, a `public/default` + +::: + namespace is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). ## Use Pulsar in Docker -Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python.md) -and [C++](client-libraries-cpp.md). If you're running a local standalone cluster, you can +Pulsar offers client libraries for [Java](client-libraries-java.md), [Go](client-libraries-go.md), [Python](client-libraries-python) +and [C++](client-libraries-cpp). If you're running a local standalone cluster, you can use one of these root URLs to interact with your cluster: * `pulsar://localhost:6650` * `http://localhost:8080` -The following example will guide you get started with Pulsar quickly by using the [Python](client-libraries-python.md) +The following example will guide you get started with Pulsar quickly by using the [Python](client-libraries-python) client API. Install the Pulsar Python client library directly from [PyPI](https://pypi.org/project/pulsar-client/): ```shell + $ pip install pulsar-client + ``` ### Consume a message @@ -67,6 +80,7 @@ $ pip install pulsar-client Create a consumer and subscribe to the topic: ```python + import pulsar client = pulsar.Client('pulsar://localhost:6650') @@ -79,6 +93,7 @@ while True: consumer.acknowledge(msg) client.close() + ``` ### Produce a message @@ -86,6 +101,7 @@ client.close() Now start a producer to send some test messages: ```python + import pulsar client = pulsar.Client('pulsar://localhost:6650') @@ -95,22 +111,26 @@ for i in range(10): producer.send(('hello-pulsar-%d' % i).encode('utf-8')) client.close() + ``` ## Get the topic statistics In Pulsar, you can use REST, Java, or command-line tools to control every aspect of the system. -For details on APIs, refer to [Admin API Overview](admin-api-overview.md). +For details on APIs, refer to [Admin API Overview](admin-api-overview). In the simplest example, you can use curl to probe the stats for a particular topic: ```shell + $ curl http://localhost:8080/admin/v2/persistent/public/default/my-topic/stats | python -m json.tool + ``` The output is something like this: ```json + { "averageMsgSize": 0.0, "msgRateIn": 0.0, @@ -158,4 +178,6 @@ The output is something like this: } } } + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-helm.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-helm.md index 8e1cc5a03111b..c5c428a1be4d2 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-helm.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-helm.md @@ -1,10 +1,14 @@ --- id: kubernetes-helm title: Get started in Kubernetes -sidebar_label: Run Pulsar in Kubernetes +sidebar_label: "Run Pulsar in Kubernetes" original_id: kubernetes-helm --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + This section guides you through every step of installing and running Apache Pulsar with Helm on Kubernetes quickly, including the following sections: - Install the Apache Pulsar on Kubernetes using Helm @@ -13,7 +17,7 @@ This section guides you through every step of installing and running Apache Puls - Produce and consume messages using Pulsar clients - Monitor Apache Pulsar status with Prometheus and Grafana -For deploying a Pulsar cluster for production usage, read the documentation on [how to configure and install a Pulsar Helm chart](helm-deploy.md). +For deploying a Pulsar cluster for production usage, read the documentation on [how to configure and install a Pulsar Helm chart](helm-deploy). ## Prerequisite @@ -21,120 +25,148 @@ For deploying a Pulsar cluster for production usage, read the documentation on [ - kubectl 1.14.0+ - Helm 3.0+ -> #### Tip -> For the following steps, step 2 and step 3 are for **developers** and step 4 and step 5 are for **administrators**. +:::tip + +For the following steps, step 2 and step 3 are for **developers** and step 4 and step 5 are for **administrators**. + +::: ## Step 0: Prepare a Kubernetes cluster -Before installing a Pulsar Helm chart, you have to create a Kubernetes cluster. You can follow [the instructions](helm-prepare.md) to prepare a Kubernetes cluster. +Before installing a Pulsar Helm chart, you have to create a Kubernetes cluster. You can follow [the instructions](helm-prepare) to prepare a Kubernetes cluster. We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start guide. To prepare a Kubernetes cluster, follow these steps: 1. Create a Kubernetes cluster on Minikube. - ```bash - minikube start --memory=8192 --cpus=4 --kubernetes-version= - ``` + ```bash + + minikube start --memory=8192 --cpus=4 --kubernetes-version= + + ``` - The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. + The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. 2. Set `kubectl` to use Minikube. - ```bash - kubectl config use-context minikube - ``` + ```bash + + kubectl config use-context minikube + + ``` 3. To use the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) with the local Kubernetes cluster on Minikube, enter the command below: - ```bash - minikube dashboard - ``` - The command automatically triggers opening a webpage in your browser. + ```bash + + minikube dashboard + + ``` + + The command automatically triggers opening a webpage in your browser. ## Step 1: Install Pulsar Helm chart 0. Add Pulsar charts repo. - ```bash - helm repo add apache https://pulsar.apache.org/charts - ``` + ```bash + + helm repo add apache https://pulsar.apache.org/charts + + ``` - ```bash - helm repo update - ``` + ```bash + + helm repo update + + ``` 1. Clone the Pulsar Helm chart repository. - ```bash - git clone https://github.com/apache/pulsar-helm-chart - cd pulsar-helm-chart - ``` + ```bash + + git clone https://github.com/apache/pulsar-helm-chart + cd pulsar-helm-chart + + ``` 2. Run the script `prepare_helm_release.sh` to create secrets required for installing the Apache Pulsar Helm chart. The username `pulsar` and password `pulsar` are used for logging into the Grafana dashboard and Pulsar Manager. - ```bash - ./scripts/pulsar/prepare_helm_release.sh \ - -n pulsar \ - -k pulsar-mini \ - -c - ``` + ```bash + + ./scripts/pulsar/prepare_helm_release.sh \ + -n pulsar \ + -k pulsar-mini \ + -c + + ``` 3. Use the Pulsar Helm chart to install a Pulsar cluster to Kubernetes. > **NOTE** > You need to specify `--set initialize=true` when installing Pulsar the first time. This command installs and starts Apache Pulsar. - ```bash - helm install \ - --values examples/values-minikube.yaml \ - --set initialize=true \ - --namespace pulsar \ - pulsar-mini apache/pulsar - ``` + ```bash + + helm install \ + --values examples/values-minikube.yaml \ + --set initialize=true \ + --namespace pulsar \ + pulsar-mini apache/pulsar + + ``` 4. Check the status of all pods. - ```bash - kubectl get pods -n pulsar - ``` - - If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. - - **Output** - - ```bash - NAME READY STATUS RESTARTS AGE - pulsar-mini-bookie-0 1/1 Running 0 9m27s - pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s - pulsar-mini-broker-0 1/1 Running 0 9m27s - pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s - pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s - pulsar-mini-proxy-0 1/1 Running 0 9m27s - pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s - pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s - pulsar-mini-toolset-0 1/1 Running 0 9m27s - pulsar-mini-zookeeper-0 1/1 Running 0 9m27s - ``` + ```bash + + kubectl get pods -n pulsar + + ``` + + If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. + + **Output** + + ```bash + + NAME READY STATUS RESTARTS AGE + pulsar-mini-bookie-0 1/1 Running 0 9m27s + pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s + pulsar-mini-broker-0 1/1 Running 0 9m27s + pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s + pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s + pulsar-mini-proxy-0 1/1 Running 0 9m27s + pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s + pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s + pulsar-mini-toolset-0 1/1 Running 0 9m27s + pulsar-mini-zookeeper-0 1/1 Running 0 9m27s + + ``` 5. Check the status of all services in the namespace `pulsar`. - ```bash - kubectl get services -n pulsar - ``` - - **Output** - - ```bash - NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE - pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m - pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m - pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m - pulsar-mini-prometheus ClusterIP None 9090/TCP 11m - pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m - pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m - pulsar-mini-toolset ClusterIP None 11m - pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m - ``` + ```bash + + kubectl get services -n pulsar + + ``` + + **Output** + + ```bash + + NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE + pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m + pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m + pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m + pulsar-mini-prometheus ClusterIP None 9090/TCP 11m + pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m + pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m + pulsar-mini-toolset ClusterIP None 11m + pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m + + ``` ## Step 2: Use pulsar-admin to create Pulsar tenants/namespaces/topics @@ -142,65 +174,85 @@ We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start 1. Enter the `toolset` container. - ```bash - kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash - ``` + ```bash + + kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash + + ``` 2. In the `toolset` container, create a tenant named `apache`. - ```bash - bin/pulsar-admin tenants create apache - ``` + ```bash + + bin/pulsar-admin tenants create apache + + ``` - Then you can list the tenants to see if the tenant is created successfully. + Then you can list the tenants to see if the tenant is created successfully. - ```bash - bin/pulsar-admin tenants list - ``` + ```bash + + bin/pulsar-admin tenants list + + ``` - You should see a similar output as below. The tenant `apache` has been successfully created. + You should see a similar output as below. The tenant `apache` has been successfully created. - ```bash - "apache" - "public" - "pulsar" - ``` + ```bash + + "apache" + "public" + "pulsar" + + ``` 3. In the `toolset` container, create a namespace named `pulsar` in the tenant `apache`. - ```bash - bin/pulsar-admin namespaces create apache/pulsar - ``` + ```bash + + bin/pulsar-admin namespaces create apache/pulsar + + ``` - Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. + Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. - ```bash - bin/pulsar-admin namespaces list apache - ``` + ```bash + + bin/pulsar-admin namespaces list apache + + ``` - You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. + You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. - ```bash - "apache/pulsar" - ``` + ```bash + + "apache/pulsar" + + ``` 4. In the `toolset` container, create a topic `test-topic` with `4` partitions in the namespace `apache/pulsar`. - ```bash - bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 - ``` + ```bash + + bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 + + ``` 5. In the `toolset` container, list all the partitioned topics in the namespace `apache/pulsar`. - ```bash - bin/pulsar-admin topics list-partitioned-topics apache/pulsar - ``` + ```bash + + bin/pulsar-admin topics list-partitioned-topics apache/pulsar + + ``` - Then you can see all the partitioned topics in the namespace `apache/pulsar`. + Then you can see all the partitioned topics in the namespace `apache/pulsar`. - ```bash - "persistent://apache/pulsar/test-topic" - ``` + ```bash + + "persistent://apache/pulsar/test-topic" + + ``` ## Step 3: Use Pulsar client to produce and consume messages @@ -209,13 +261,17 @@ You can use the Pulsar client to create producers and consumers to produce and c By default, the Pulsar Helm chart exposes the Pulsar cluster through a Kubernetes `LoadBalancer`. In Minikube, you can use the following command to check the proxy service. ```bash + kubectl get services -n pulsar | grep pulsar-mini-proxy + ``` You will see a similar output as below. ```bash + pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 28m + ``` This output tells what are the node ports that Pulsar cluster's binary port and HTTP port are mapped to. The port after `80:` is the HTTP port while the port after `6650:` is the binary port. @@ -223,12 +279,15 @@ This output tells what are the node ports that Pulsar cluster's binary port and Then you can find the IP address and exposed ports of your Minikube server by running the following command. ```bash + minikube service pulsar-mini-proxy -n pulsar + ``` **Output** ```bash + |-----------|-------------------|-------------|-------------------------| | NAMESPACE | NAME | TARGET PORT | URL | |-----------|-------------------|-------------|-------------------------| @@ -242,12 +301,16 @@ minikube service pulsar-mini-proxy -n pulsar | pulsar | pulsar-mini-proxy | | http://127.0.0.1:61853 | | | | | http://127.0.0.1:61854 | |-----------|-------------------|-------------|------------------------| + ``` At this point, you can get the service URLs to connect to your Pulsar client. Here are URL examples: + ``` + webServiceUrl=http://127.0.0.1:61853/ brokerServiceUrl=pulsar://127.0.0.1:61854/ + ``` Then you can proceed with the following steps: @@ -256,95 +319,109 @@ Then you can proceed with the following steps: 2. Decompress the tarball based on your download file. - ```bash - tar -xf .tar.gz - ``` + ```bash + + tar -xf .tar.gz + + ``` 3. Expose `PULSAR_HOME`. - (1) Enter the directory of the decompressed download file. + (1) Enter the directory of the decompressed download file. - (2) Expose `PULSAR_HOME` as the environment variable. + (2) Expose `PULSAR_HOME` as the environment variable. - ```bash - export PULSAR_HOME=$(pwd) - ``` + ```bash + + export PULSAR_HOME=$(pwd) + + ``` 4. Configure the Pulsar client. - In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. + In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. 5. Create a subscription to consume messages from `apache/pulsar/test-topic`. - ```bash - bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 - ``` + ```bash + + bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 + + ``` 6. Open a new terminal. In the new terminal, create a producer and send 10 messages to the `test-topic` topic. - ```bash - bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 - ``` + ```bash + + bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 + + ``` 7. Verify the results. - - From the producer side - - **Output** - - The messages have been produced successfully. - - ```bash - 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced - ``` - - - From the consumer side - - **Output** - - At the same time, you can receive the messages as below. - - ```bash - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ``` + - From the producer side + + **Output** + + The messages have been produced successfully. + + ```bash + + 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced + + ``` + + - From the consumer side + + **Output** + + At the same time, you can receive the messages as below. + + ```bash + + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + + ``` ## Step 4: Use Pulsar Manager to manage the cluster -[Pulsar Manager](administration-pulsar-manager.md) is a web-based GUI management tool for managing and monitoring Pulsar. +[Pulsar Manager](administration-pulsar-manager) is a web-based GUI management tool for managing and monitoring Pulsar. 1. By default, the `Pulsar Manager` is exposed as a separate `LoadBalancer`. You can open the Pulsar Manager UI using the following command: - ```bash - minikube service -n pulsar pulsar-mini-pulsar-manager - ``` + ```bash + + minikube service -n pulsar pulsar-mini-pulsar-manager + + ``` 2. The Pulsar Manager UI will be open in your browser. You can use the username `pulsar` and password `pulsar` to log into Pulsar Manager. 3. In Pulsar Manager UI, you can create an environment. - - Click `New Environment` button in the top-left corner. - - Type `pulsar-mini` for the field `Environment Name` in the popup window. - - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. - - Click `Confirm` button in the popup window. + - Click `New Environment` button in the top-left corner. + - Type `pulsar-mini` for the field `Environment Name` in the popup window. + - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. + - Click `Confirm` button in the popup window. 4. After successfully created an environment, you are redirected to the `tenants` page of that environment. Then you can create `tenants`, `namespaces` and `topics` using the Pulsar Manager. @@ -354,9 +431,11 @@ Grafana is an open-source visualization tool, which can be used for visualizing 1. By default, the Grafana is exposed as a separate `LoadBalancer`. You can open the Grafana UI using the following command: - ```bash - minikube service pulsar-mini-grafana -n pulsar - ``` + ```bash + + minikube service pulsar-mini-grafana -n pulsar + + ``` 2. The Grafana UI is open in your browser. You can use the username `pulsar` and password `pulsar` to log into the Grafana Dashboard. diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-pulsar.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-pulsar.md index 6f9ce8ac59d11..d1dd8bf605992 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-pulsar.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-pulsar.md @@ -1,17 +1,21 @@ --- id: pulsar-2.0 title: Pulsar 2.0 -sidebar_label: Pulsar 2.0 +sidebar_label: "Pulsar 2.0" original_id: pulsar-2.0 --- -Pulsar 2.0 is a major new release for Pulsar that brings some bold changes to the platform, including [simplified topic names](#topic-names), the addition of the [Pulsar Functions](functions-overview.md) feature, some terminology changes, and more. +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + +Pulsar 2.0 is a major new release for Pulsar that brings some bold changes to the platform, including [simplified topic names](#topic-names), the addition of the [Pulsar Functions](functions-overview) feature, some terminology changes, and more. ## New features in Pulsar 2.0 Feature | Description :-------|:----------- -[Pulsar Functions](functions-overview.md) | A lightweight compute option for Pulsar +[Pulsar Functions](functions-overview) | A lightweight compute option for Pulsar ## Major changes @@ -26,8 +30,11 @@ Previously, Pulsar had a concept of properties. A property is essentially the ex Prior to version 2.0, *all* Pulsar topics had the following form: ```http + {persistent|non-persistent}://property/cluster/namespace/topic + ``` + Two important changes have been made in Pulsar 2.0: * There is no longer a [cluster component](#no-cluster) @@ -40,7 +47,9 @@ Two important changes have been made in Pulsar 2.0: The cluster component has been removed from topic names. Thus, all topic names now have the following form: ```http + {persistent|non-persistent}://tenant/namespace/topic + ``` > Existing topics that use the legacy name format will continue to work without any change, and there are no plans to change that. diff --git a/site2/website-next/versioned_docs/version-2.7.3/getting-started-standalone.md b/site2/website-next/versioned_docs/version-2.7.3/getting-started-standalone.md index 0e2fc3c4af84c..51e75384d75af 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/getting-started-standalone.md +++ b/site2/website-next/versioned_docs/version-2.7.3/getting-started-standalone.md @@ -1,15 +1,19 @@ --- +slug: / id: standalone title: Set up a standalone Pulsar locally -sidebar_label: Run Pulsar locally +sidebar_label: "Run Pulsar locally" original_id: standalone -slug: / --- +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + + For local development and testing, you can run Pulsar in standalone mode on your machine. The standalone mode includes a Pulsar broker, the necessary ZooKeeper and BookKeeper components running inside of a single Java Virtual Machine (JVM) process. > #### Pulsar in production? -> If you're looking to run a full production Pulsar installation, see the [Deploying a Pulsar instance](deploy-bare-metal.md) guide. +> If you're looking to run a full production Pulsar installation, see the [Deploying a Pulsar instance](deploy-bare-metal) guide. ## Install Pulsar standalone @@ -19,14 +23,17 @@ This tutorial guides you through every step of the installation process. Pulsar is currently available for **MacOS** and **Linux**. To use Pulsar, you need to install Java 8 from [Oracle download center](http://www.oracle.com/). -> #### Tip -> By default, Pulsar allocates 2G JVM heap memory to start. It can be changed in `conf/pulsar_env.sh` file under `PULSAR_MEM`. This is extra options passed into JVM. +:::tip + +By default, Pulsar allocates 2G JVM heap memory to start. It can be changed in `conf/pulsar_env.sh` file under `PULSAR_MEM`. This is extra options passed into JVM. + +::: ### Install Pulsar using binary release To get started with Pulsar, download a binary tarball release in one of the following ways: -* download from the Apache mirror (Pulsar {{pulsar:version}} binary release) +* download from the Apache mirror (Pulsar @pulsar:version@ binary release) * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -35,14 +42,18 @@ To get started with Pulsar, download a binary tarball release in one of the foll * use [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:binary_release_url + ``` After you download the tarball, untar it and use the `cd` command to navigate to the resulting directory: ```bash -$ tar xvfz apache-pulsar-{{pulsar:version}}-bin.tar.gz -$ cd apache-pulsar-{{pulsar:version}} + +$ tar xvfz apache-pulsar-@pulsar:version@-bin.tar.gz +$ cd apache-pulsar-@pulsar:version@ + ``` #### What your package contains @@ -53,7 +64,7 @@ Directory | Contains :---------|:-------- `bin` | Pulsar's command-line tools, such as [`pulsar`](reference-cli-tools.md#pulsar) and [`pulsar-admin`](https://pulsar.apache.org/tools/pulsar-admin/). `conf` | Configuration files for Pulsar, including [broker configuration](reference-configuration.md#broker), [ZooKeeper configuration](reference-configuration.md#zookeeper), and more. -`examples` | A Java JAR file containing [Pulsar Functions](functions-overview.md) example. +`examples` | A Java JAR file containing [Pulsar Functions](functions-overview) example. `lib` | The [JAR](https://en.wikipedia.org/wiki/JAR_(file_format)) files used by Pulsar. `licenses` | License files, in the`.txt` form, for various components of the Pulsar [codebase](https://github.com/apache/pulsar). @@ -62,23 +73,24 @@ These directories are created once you begin running Pulsar. Directory | Contains :---------|:-------- `data` | The data storage directory used by ZooKeeper and BookKeeper. -`instances` | Artifacts created for [Pulsar Functions](functions-overview.md). +`instances` | Artifacts created for [Pulsar Functions](functions-overview). `logs` | Logs created by the installation. -> #### Tip -> If you want to use builtin connectors and tiered storage offloaders, you can install them according to the following instructions: -> -> * [Install builtin connectors (optional)](#install-builtin-connectors-optional) -> * [Install tiered storage offloaders (optional)](#install-tiered-storage-offloaders-optional) -> -> Otherwise, skip this step and perform the next step [Start Pulsar standalone](#start-pulsar-standalone). Pulsar can be successfully installed without installing bulitin connectors and tiered storage offloaders. +:::tip + +If you want to use builtin connectors and tiered storage offloaders, you can install them according to the following instructions: +* [Install builtin connectors (optional)](#install-builtin-connectors-optional) +* [Install tiered storage offloaders (optional)](#install-tiered-storage-offloaders-optional) +Otherwise, skip this step and perform the next step [Start Pulsar standalone](#start-pulsar-standalone). Pulsar can be successfully installed without installing bulitin connectors and tiered storage offloaders. + +::: ### Install builtin connectors (optional) Since `2.1.0-incubating` release, Pulsar releases a separate binary distribution, containing all the `builtin` connectors. To enable those `builtin` connectors, you can download the connectors tarball release in one of the following ways: -* download from the Apache mirror Pulsar IO Connectors {{pulsar:version}} release +* download from the Apache mirror Pulsar IO Connectors @pulsar:version@ release * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -87,39 +99,46 @@ To enable those `builtin` connectors, you can download the connectors tarball re * use [wget](https://www.gnu.org/software/wget): ```shell - $ wget pulsar:connector_release_url/{connector}-{{pulsar:version}}.nar + + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar + ``` After you download the nar file, copy the file to the `connectors` directory in the pulsar directory. -For example, if you download the `pulsar-io-aerospike-{{pulsar:version}}.nar` connector file, enter the following commands: +For example, if you download the `pulsar-io-aerospike-@pulsar:version@.nar` connector file, enter the following commands: ```bash + $ mkdir connectors -$ mv pulsar-io-aerospike-{{pulsar:version}}.nar connectors +$ mv pulsar-io-aerospike-@pulsar:version@.nar connectors $ ls connectors -pulsar-io-aerospike-{{pulsar:version}}.nar +pulsar-io-aerospike-@pulsar:version@.nar ... + ``` -> #### Note -> -> * If you are running Pulsar in a bare metal cluster, make sure `connectors` tarball is unzipped in every pulsar directory of the broker -> (or in every pulsar directory of function-worker if you are running a separate worker cluster for Pulsar Functions). -> -> * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos.md)), -> you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). +:::note + +* If you are running Pulsar in a bare metal cluster, make sure `connectors` tarball is unzipped in every pulsar directory of the broker +(or in every pulsar directory of function-worker if you are running a separate worker cluster for Pulsar Functions). +* If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), +you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). + +::: ### Install tiered storage offloaders (optional) -> #### Tip -> -> Since `2.2.0` release, Pulsar releases a separate binary distribution, containing the tiered storage offloaders. -> To enable tiered storage feature, follow the instructions below; otherwise skip this section. +:::tip + +Since `2.2.0` release, Pulsar releases a separate binary distribution, containing the tiered storage offloaders. +To enable tiered storage feature, follow the instructions below; otherwise skip this section. + +::: -To get started with [tiered storage offloaders](concepts-tiered-storage.md), you need to download the offloaders tarball release on every broker node in one of the following ways: +To get started with [tiered storage offloaders](concepts-tiered-storage), you need to download the offloaders tarball release on every broker node in one of the following ways: -* download from the Apache mirror Pulsar Tiered Storage Offloaders {{pulsar:version}} release +* download from the Apache mirror Pulsar Tiered Storage Offloaders @pulsar:version@ release * download from the Pulsar [downloads page](pulsar:download_page_url) @@ -128,55 +147,67 @@ To get started with [tiered storage offloaders](concepts-tiered-storage.md), you * use [wget](https://www.gnu.org/software/wget): ```shell + $ wget pulsar:offloader_release_url + ``` After you download the tarball, untar the offloaders package and copy the offloaders as `offloaders` in the pulsar directory: ```bash -$ tar xvfz apache-pulsar-offloaders-{{pulsar:version}}-bin.tar.gz -// you will find a directory named `apache-pulsar-offloaders-{{pulsar:version}}` in the pulsar directory +$ tar xvfz apache-pulsar-offloaders-@pulsar:version@-bin.tar.gz + +// you will find a directory named `apache-pulsar-offloaders-@pulsar:version@` in the pulsar directory // then copy the offloaders -$ mv apache-pulsar-offloaders-{{pulsar:version}}/offloaders offloaders +$ mv apache-pulsar-offloaders-@pulsar:version@/offloaders offloaders $ ls offloaders -tiered-storage-jcloud-{{pulsar:version}}.nar +tiered-storage-jcloud-@pulsar:version@.nar + ``` -For more information on how to configure tiered storage, see [Tiered storage cookbook](cookbooks-tiered-storage.md). +For more information on how to configure tiered storage, see [Tiered storage cookbook](cookbooks-tiered-storage). -> #### Note -> -> * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's pulsar directory. -> -> * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos.md)), -> you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. +:::note + +* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's pulsar directory. +* If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), +you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + +::: ## Start Pulsar standalone Once you have an up-to-date local copy of the release, you can start a local cluster using the [`pulsar`](reference-cli-tools.md#pulsar) command, which is stored in the `bin` directory, and specifying that you want to start Pulsar in standalone mode. ```bash + $ bin/pulsar standalone + ``` If you have started Pulsar successfully, you will see `INFO`-level log messages like this: ```bash + 2017-06-01 14:46:29,192 - INFO - [main:WebSocketService@95] - Configuration Store cache started 2017-06-01 14:46:29,192 - INFO - [main:AuthenticationService@61] - Authentication is disabled 2017-06-01 14:46:29,192 - INFO - [main:WebSocketService@108] - Pulsar WebSocket Service started + ``` -> #### Tip -> -> * The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. +:::tip + +* The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. + +::: + You can also run the service as a background process using the `pulsar-daemon start standalone` command. For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). > -> * By default, there is no encryption, authentication, or authorization configured. Apache Pulsar can be accessed from remote server without any authorization. Please do check [Security Overview](security-overview.md) document to secure your deployment. +> * By default, there is no encryption, authentication, or authorization configured. Apache Pulsar can be accessed from remote server without any authorization. Please do check [Security Overview](security-overview) document to secure your deployment. > > * When you start a local standalone cluster, a `public/default` [namespace](concepts-messaging.md#namespaces) is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). @@ -189,39 +220,51 @@ Pulsar provides a CLI tool called [`pulsar-client`](reference-cli-tools.md#pulsa The following command consumes a message with the subscription name `first-subscription` to the `my-topic` topic: ```bash + $ bin/pulsar-client consume my-topic -s "first-subscription" + ``` If the message has been successfully consumed, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 09:56:55.566 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.MultiTopicsConsumerImpl - [TopicsConsumerFakeTopicNamee2df9] [first-subscription] Success subscribe new topic my-topic in topics consumer, partitions: 4, allTopicPartitionsNumber: 4 + ``` -> #### Tip -> -> As you have noticed that we do not explicitly create the `my-topic` topic, to which we consume the message. When you consume a message to a topic that does not yet exist, Pulsar creates that topic for you automatically. Producing a message to a topic that does not exist will automatically create that topic for you as well. +:::tip + +As you have noticed that we do not explicitly create the `my-topic` topic, to which we consume the message. When you consume a message to a topic that does not yet exist, Pulsar creates that topic for you automatically. Producing a message to a topic that does not exist will automatically create that topic for you as well. + +::: ### Produce a message The following command produces a message saying `hello-pulsar` to the `my-topic` topic: ```bash + $ bin/pulsar-client produce my-topic --messages "hello-pulsar" + ``` If the message has been successfully published to the topic, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 13:09:39.356 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 1 messages successfully produced + ``` ## Stop Pulsar standalone Press `Ctrl+C` to stop a local standalone Pulsar. -> #### Tip -> -> If the service runs as a background process using the `pulsar-daemon start standalone` command, then use the `pulsar-daemon stop standalone` command to stop the service. -> -> For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). +:::tip + +If the service runs as a background process using the `pulsar-daemon start standalone` command, then use the `pulsar-daemon stop standalone` command to stop the service. +For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). + +::: + diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-deploy.md b/site2/website-next/versioned_docs/version-2.7.3/helm-deploy.md index 3b0bb67867360..e64ca2de9b1cf 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-deploy.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-deploy.md @@ -1,7 +1,7 @@ --- id: helm-deploy title: Deploy Pulsar cluster using Helm -sidebar_label: Deployment +sidebar_label: "Deployment" original_id: helm-deploy --- @@ -21,25 +21,33 @@ In each section, collect the options that are combined to use with the `helm ins By default, the Pulsar Helm chart is installed to a namespace called `pulsar`. ```yaml + namespace: pulsar + ``` To install the Pulsar Helm chart into a different Kubernetes namespace, you can include this option in the `helm install` command. ```bash + --set namespace= + ``` By default, the Pulsar Helm chart doesn't create the namespace. ```yaml + namespaceCreate: false + ``` To use the Pulsar Helm chart to create the Kubernetes namespace automatically, you can include this option in the `helm install` command. ```bash + --set namespaceCreate=true + ``` ### Persistence @@ -47,22 +55,25 @@ To use the Pulsar Helm chart to create the Kubernetes namespace automatically, y By default, the Pulsar Helm chart creates Volume Claims with the expectation that a dynamic provisioner creates the underlying Persistent Volumes. ```yaml + volumes: persistence: true # configure the components to use local persistent volume # the local provisioner should be installed prior to enable local persistent volume local_storage: false + ``` To use local persistent volumes as the persistent storage for Helm release, you can install the [local storage provisioner](#install-local-storage-provisioner) and include the following option in the `helm install` command. ```bash + --set volumes.local_storage=true + ``` :::note - Before installing the production instance of Pulsar, ensure to plan the storage settings to avoid extra storage migration work. Because after initial installation, you must edit Kubernetes objects manually if you want to change storage settings. ::: @@ -70,7 +81,9 @@ Before installing the production instance of Pulsar, ensure to plan the storage The Pulsar Helm chart is designed for production use. To use the Pulsar Helm chart in a development environment (such as Minikube), you can disable persistence by including this option in your `helm install` command. ```bash + --set volumes.persistence=false + ``` ### Affinity @@ -78,14 +91,18 @@ The Pulsar Helm chart is designed for production use. To use the Pulsar Helm cha By default, `anti-affinity` is enabled to ensure pods of the same component can run on different nodes. ```yaml + affinity: anti_affinity: true + ``` To use the Pulsar Helm chart in a development environment (such as Minikue), you can disable `anti-affinity` by including this option in your `helm install` command. ```bash + --set affinity.anti_affinity=false + ``` ### Components @@ -95,6 +112,7 @@ The Pulsar Helm chart is designed for production usage. It deploys a production- You can customize the components to be deployed by turning on/off individual components. ```yaml + ## Components ## ## Control what components of Apache Pulsar to deploy for the cluster @@ -124,6 +142,7 @@ monitoring: prometheus: true # monitoring - grafana grafana: true + ``` ### Docker images @@ -131,6 +150,7 @@ monitoring: The Pulsar Helm chart is designed to enable controlled upgrades. So it can configure independent image versions for components. You can customize the images by setting individual component. ```yaml + ## Images ## ## Control what images to use for each component @@ -171,6 +191,7 @@ images: tag: v0.1.0 pullPolicy: IfNotPresent hasCommand: false + ``` ### TLS @@ -182,15 +203,19 @@ The Pulsar Helm chart can be configured to enable TLS (Transport Layer Security) To use the `cert-manager` to provision the TLS certificates, you have to install the [cert-manager](#install-cert-manager) before installing the Pulsar Helm chart. After successfully installing the cert-manager, you can set `certs.internal_issuer.enabled` to `true`. Therefore, the Pulsar Helm chart can use the `cert-manager` to generate `selfsigning` TLS certificates for the configured components. ```yaml + certs: internal_issuer: enabled: false component: internal-cert-issuer type: selfsigning + ``` + You can also customize the generated TLS certificates by configuring the fields as the following. ```yaml + tls: # common settings for generating certs common: @@ -203,6 +228,7 @@ tls: keySize: 4096 keyAlgorithm: rsa keyEncoding: pkcs8 + ``` #### Enable TLS @@ -210,13 +236,16 @@ tls: After installing the `cert-manager`, you can set `tls.enabled` to `true` to enable TLS encryption for the entire cluster. ```yaml + tls: enabled: false + ``` You can also configure whether to enable TLS encryption for individual component. ```yaml + tls: # settings for generating certs for proxy proxy: @@ -240,6 +269,7 @@ tls: # settings for generating certs for toolset toolset: cert_name: tls-toolset + ``` ### Authentication @@ -248,6 +278,7 @@ By default, authentication is disabled. You can set `auth.authentication.enabled Currently, the Pulsar Helm chart only supports JWT authentication provider. You can set `auth.authentication.provider` to `jwt` to use the JWT authentication provider. ```yaml + # Enable or disable broker authentication and authorization. auth: authentication: @@ -265,12 +296,15 @@ auth: proxy: "proxy-admin" # pulsar-admin client to broker/proxy communication client: "admin" + ``` To enable authentication, you can run [prepare helm release](#prepare-the-helm-release) to generate token secret keys and tokens for three super users specified in the `auth.superUsers` field. The generated token keys and super user tokens are uploaded and stored as Kubernetes secrets prefixed with `-token-`. You can use the following command to find those secrets. ```bash + kubectl get secrets -n + ``` ### Authorization @@ -278,15 +312,19 @@ kubectl get secrets -n By default, authorization is disabled. Authorization can be enabled only when authentication is enabled. ```yaml + auth: authorization: enabled: false + ``` To enable authorization, you can include this option in the `helm install` command. ```bash + --set auth.authorization.enabled=true + ``` ### CPU and RAM resource requirements @@ -304,9 +342,11 @@ To use local persistent volumes as the persistent storage, you need to install a One of the easiest way to get started is to use the local storage provisioner provided along with the Pulsar Helm chart. ``` + helm repo add streamnative https://charts.streamnative.io helm repo update helm install pulsar-storage-provisioner streamnative/local-storage-provisioner + ``` ### Install cert-manager @@ -318,9 +358,11 @@ For details about how to install the cert-manager, follow the [official instruct Alternatively, we provide a bash script [install-cert-manager.sh](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/cert-manager/install-cert-manager.sh) to install a cert-manager release to the namespace `cert-manager`. ```bash + git clone https://github.com/apache/pulsar-helm-chart cd pulsar-helm-chart ./scripts/cert-manager/install-cert-manager.sh + ``` ## Prepare Helm release @@ -328,18 +370,20 @@ cd pulsar-helm-chart Once you have install all the dependent charts and collected all of your configuration options, you can run [prepare_helm_release.sh](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/prepare_helm_release.sh) to prepare the Helm release. ```bash + git clone https://github.com/apache/pulsar-helm-chart cd pulsar-helm-chart ./scripts/pulsar/prepare_helm_release.sh -n -k + ``` The `prepare_helm_release` creates the following resources: - A Kubernetes namespace for installing the Pulsar release - JWT secret keys and tokens for three super users: `broker-admin`, `proxy-admin`, and `admin`. By default, it generates an asymmetric pubic/private key pair. You can choose to generate a symmetric secret key by specifying `--symmetric`. - - `proxy-admin` role is used for proxies to communicate to brokers. - - `broker-admin` role is used for inter-broker communications. - - `admin` role is used by the admin tools. + - `proxy-admin` role is used for proxies to communicate to brokers. + - `broker-admin` role is used for inter-broker communications. + - `admin` role is used by the admin tools. ## Deploy Pulsar cluster using Helm @@ -352,17 +396,18 @@ Once you have finished the following three things, you can install a Helm releas In this example, we name our Helm release `pulsar`. ```bash + helm repo add apache https://pulsar.apache.org/charts helm repo update helm install pulsar apache/pulsar \ --timeout 10m \ --set initialize=true \ --set [your configuration options] + ``` :::note - For the first deployment, add `--set initialize=true` option to initialize bookie and Pulsar cluster metadata. ::: @@ -386,5 +431,8 @@ The default values will create a `ClusterIP` for the following resources, which To find the IP addresses of those components, run the following command: ```bash + kubectl get service -n + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-install.md b/site2/website-next/versioned_docs/version-2.7.3/helm-install.md index addf960318250..7db5d915fe216 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-install.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-install.md @@ -1,7 +1,7 @@ --- id: helm-install title: Install Apache Pulsar using Helm -sidebar_label: Install +sidebar_label: "Install" original_id: helm-install --- @@ -25,7 +25,7 @@ Before deploying Pulsar, you need to prepare your environment. ### Tools -Install [`helm`](helm-tools.md) and [`kubectl`](helm-tools.md) on your computer. +Install [`helm`](helm-tools.md) and [`kubectl`](helm-tools) on your computer. ## Cloud cluster preparation @@ -41,8 +41,8 @@ To create and connect to the Kubernetes cluster, follow the instructions: ## Pulsar deployment -Once the environment is set up and configuration is generated, you can now proceed to the [deployment of Pulsar](helm-deploy.md). +Once the environment is set up and configuration is generated, you can now proceed to the [deployment of Pulsar](helm-deploy). ## Pulsar upgrade -To upgrade an existing Kubernetes installation, follow the [upgrade documentation](helm-upgrade.md). +To upgrade an existing Kubernetes installation, follow the [upgrade documentation](helm-upgrade). diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-overview.md b/site2/website-next/versioned_docs/version-2.7.3/helm-overview.md index 24c904163410c..1f9d2b066205f 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-overview.md @@ -1,7 +1,7 @@ --- id: helm-overview title: Apache Pulsar Helm Chart -sidebar_label: Overview +sidebar_label: "Overview" original_id: helm-overview --- @@ -18,47 +18,47 @@ The Apache Pulsar Helm chart is one of the most convenient ways to operate Pulsa This chart includes all the components for a complete experience, but each part can be configured to be installed separately. - Pulsar core components: - - ZooKeeper - - Bookies - - Brokers - - Function workers - - Proxies + - ZooKeeper + - Bookies + - Brokers + - Function workers + - Proxies - Control Center: - - Pulsar Manager - - Prometheus - - Grafana + - Pulsar Manager + - Prometheus + - Grafana It includes support for: - Security - - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) - - self-signed - - [Let's Encrypt](https://letsencrypt.org/) - - TLS Encryption - - Proxy - - Broker - - Toolset - - Bookie - - ZooKeeper - - Authentication - - JWT - - Authorization + - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) + - self-signed + - [Let's Encrypt](https://letsencrypt.org/) + - TLS Encryption + - Proxy + - Broker + - Toolset + - Bookie + - ZooKeeper + - Authentication + - JWT + - Authorization - Storage - - Non-persistence storage - - Persistence volume - - Local persistent volumes + - Non-persistence storage + - Persistence volume + - Local persistent volumes - Functions - - Kubernetes Runtime - - Process Runtime - - Thread Runtime + - Kubernetes Runtime + - Process Runtime + - Thread Runtime - Operations - - Independent image versions for all components, enabling controlled upgrades + - Independent image versions for all components, enabling controlled upgrades ## Pulsar Helm chart quick start -To get up and run with these charts as fast as possible, in a **non-production** use case, we provide a [quick start guide](getting-started-helm.md) for Proof of Concept (PoC) deployments. +To get up and run with these charts as fast as possible, in a **non-production** use case, we provide a [quick start guide](getting-started-helm) for Proof of Concept (PoC) deployments. -This guide walks the user through deploying these charts with default values and features, but *does not* meet production ready requirements. To deploy these charts into production under sustained load, follow the complete [Installation Guide](helm-install.md). +This guide walks the user through deploying these charts with default values and features, but *does not* meet production ready requirements. To deploy these charts into production under sustained load, follow the complete [Installation Guide](helm-install). ## Troubleshooting @@ -68,37 +68,41 @@ We have done our best to make these charts as seamless as possible. Occasionally The Apache Pulsar Helm chart contains all required dependencies. -If you deploy a PoC for testing, we strongly suggest you follow our [Quick Start Guide](getting-started-helm.md) for your first iteration. +If you deploy a PoC for testing, we strongly suggest you follow our [Quick Start Guide](getting-started-helm) for your first iteration. -1. [Preparation](helm-prepare.md) -2. [Deployment](helm-deploy.md) +1. [Preparation](helm-prepare) +2. [Deployment](helm-deploy) ## Upgrading Once the Pulsar Helm chart is installed, use the `helm upgrade` to complete configuration changes and chart updates. ```bash + helm repo add apache https://pulsar.apache.org/charts helm repo update helm get values > pulsar.yaml helm upgrade apache/pulsar -f pulsar.yaml + ``` -For more detailed information, see [Upgrading](helm-upgrade.md). +For more detailed information, see [Upgrading](helm-upgrade). ## Uninstallation To uninstall the Pulsar Helm chart, run the following command: ```bash + helm delete + ``` For the purposes of continuity, these charts have some Kubernetes objects that cannot be removed when performing `helm delete`. It is recommended to *consciously* remove these items, as they affect re-deployment. * PVCs for stateful data: *consciously* remove these items. - - ZooKeeper: This is your metadata. - - BookKeeper: This is your data. - - Prometheus: This is your metrics data, which can be safely removed. + - ZooKeeper: This is your metadata. + - BookKeeper: This is your data. + - Prometheus: This is your metrics data, which can be safely removed. * Secrets: if the secrets are generated by the [prepare release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/prepare_helm_release.sh), they contain secret keys and tokens. You can use the [cleanup release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/cleanup_helm_release.sh) to remove these secrets and tokens as needed. diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-prepare.md b/site2/website-next/versioned_docs/version-2.7.3/helm-prepare.md index 7856a23f0a338..705e35794a39f 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-prepare.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-prepare.md @@ -1,7 +1,7 @@ --- id: helm-prepare title: Prepare Kubernetes resources -sidebar_label: Prepare +sidebar_label: "Prepare" original_id: helm-prepare --- @@ -14,7 +14,7 @@ For a fully functional Pulsar cluster, you need a few resources before deploying - [Google Kubernetes Engine](#google-kubernetes-engine) - [Manual cluster creation](#manual-cluster-creation) - [Scripted cluster creation](#scripted-cluster-creation) - - [Create cluster with local SSDs](#create-cluster-with-local-ssds) + - [Create cluster with local SSDs](#create-cluster-with-local-ssds) - [Next Steps](#next-steps) ## Google Kubernetes Engine @@ -24,7 +24,7 @@ To get started easier, a script is provided to create the cluster automatically. - [Google Kubernetes Engine](#google-kubernetes-engine) - [Manual cluster creation](#manual-cluster-creation) - [Scripted cluster creation](#scripted-cluster-creation) - - [Create cluster with local SSDs](#create-cluster-with-local-ssds) + - [Create cluster with local SSDs](#create-cluster-with-local-ssds) - [Next Steps](#next-steps) ### Manual cluster creation @@ -68,13 +68,17 @@ The following table describes all variables. Run the script, by passing in your desired parameters. It can work with the default parameters except for `PROJECT` which is required: ```bash + PROJECT= scripts/pulsar/gke_bootstrap_script.sh up + ``` The script can also be used to clean up the created GKE resources. ```bash + PROJECT= scripts/pulsar/gke_bootstrap_script.sh down + ``` #### Create cluster with local SSDs @@ -82,8 +86,11 @@ PROJECT= scripts/pulsar/gke_bootstrap_script.sh down To install a Pulsar Helm chart using local persistent volumes, you need to create a GKE cluster with local SSDs. You can do so Specifying the `USE_LOCAL_SSD` to be `true` in the following command to create a Pulsar cluster with local SSDs. ``` + PROJECT= USE_LOCAL_SSD=true LOCAL_SSD_COUNT= scripts/pulsar/gke_bootstrap_script.sh up + ``` + ## Next Steps -Continue with the [installation of the chart](helm-deploy.md) once you have the cluster up and running. +Continue with the [installation of the chart](helm-deploy) once you have the cluster up and running. diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-tools.md b/site2/website-next/versioned_docs/version-2.7.3/helm-tools.md index 02ada4c546b6d..333205a2d8084 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-tools.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-tools.md @@ -1,7 +1,7 @@ --- id: helm-tools title: Required tools for deploying Pulsar Helm Chart -sidebar_label: Required Tools +sidebar_label: "Required Tools" original_id: helm-tools --- @@ -29,7 +29,7 @@ You can get Helm from the project's [releases page](https://github.com/helm/helm ### Next steps -Once kubectl and Helm are configured, you can configure your [Kubernetes cluster](helm-prepare.md). +Once kubectl and Helm are configured, you can configure your [Kubernetes cluster](helm-prepare). ## Additional information diff --git a/site2/website-next/versioned_docs/version-2.7.3/helm-upgrade.md b/site2/website-next/versioned_docs/version-2.7.3/helm-upgrade.md index acf15eeaefe1b..8b00b8ed37553 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/helm-upgrade.md +++ b/site2/website-next/versioned_docs/version-2.7.3/helm-upgrade.md @@ -1,7 +1,7 @@ --- id: helm-upgrade title: Upgrade Pulsar Helm release -sidebar_label: Upgrade +sidebar_label: "Upgrade" original_id: helm-upgrade --- @@ -15,10 +15,10 @@ We also recommend that you need to provide all values using the `helm upgrade -- :::note - You can retrieve your previous `--set` arguments cleanly, with `helm get values `. If you direct this into a file (`helm get values > pulsar.yml`), you can safely ::: + pass this file through `-f`. Thus `helm upgrade apache/pulsar -f pulsar.yaml`. This safely replaces the behavior of `--reuse-values`. ## Steps @@ -26,16 +26,24 @@ pass this file through `-f`. Thus `helm upgrade apache/pulsar -f To upgrade Apache Pulsar to a newer version, follow these steps: 1. Check the change log for the specific version you would like to upgrade to. -2. Go through [deployment documentation](helm-deploy.md) step by step. +2. Go through [deployment documentation](helm-deploy) step by step. 3. Extract your previous `--set` arguments with the following command. - ```bash - helm get values > pulsar.yaml - ``` + + ```bash + + helm get values > pulsar.yaml + + ``` + 4. Decide all the values you need to set. 5. Perform the upgrade, with all `--set` arguments extracted in step 4. - ```bash - helm upgrade apache/pulsar \ - --version \ - -f pulsar.yaml \ - --set ... - ``` + + ```bash + + helm upgrade apache/pulsar \ + --version \ + -f pulsar.yaml \ + --set ... + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-aerospike-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-aerospike-sink.md index 5b4eda521564e..7ff980521a489 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-aerospike-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-aerospike-sink.md @@ -1,7 +1,7 @@ --- id: io-aerospike-sink title: Aerospike sink connector -sidebar_label: Aerospike sink connector +sidebar_label: "Aerospike sink connector" original_id: io-aerospike-sink --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-canal-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-canal-source.md index 3921aab4c4c41..853b387164d65 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-canal-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-canal-source.md @@ -1,7 +1,7 @@ --- id: io-canal-source title: Canal source connector -sidebar_label: Canal source connector +sidebar_label: "Canal source connector" original_id: io-canal-source --- @@ -24,7 +24,7 @@ The configuration of Canal source connector has the following properties. |`destination`|true|None|Source destination that Canal source connector connects to. | `singleHostname` | false | None | Canal server address.| | `singlePort` | false | None | Canal server port.| -| `cluster` | true | false | Whether to enable cluster mode based on Canal server configuration or not.

  • true: **cluster** mode.
    If set to true, it talks to `zkServers` to figure out the actual database host.

  • false: **standalone** mode.
    If set to false, it connects to the database specified by `singleHostname` and `singlePort`.
  • | +| `cluster` | true | false | Whether to enable cluster mode based on Canal server configuration or not.

  • true: **cluster** mode.
    If set to true, it talks to `zkServers` to figure out the actual database host.

  • false: **standalone** mode.
    If set to false, it connects to the database specified by `singleHostname` and `singlePort`.
  • | | `zkServers` | true | None | Address and port of the Zookeeper that Canal source connector talks to figure out the actual database host.| | `batchSize` | false | 1000 | Batch size to fetch from Canal. | @@ -34,34 +34,38 @@ Before using the Canal connector, you can create a configuration file through on * JSON - ```json - { - "zkServers": "127.0.0.1:2181", - "batchSize": "5120", - "destination": "example", - "username": "", - "password": "", - "cluster": false, - "singleHostname": "127.0.0.1", - "singlePort": "11111", - } - ``` + ```json + + { + "zkServers": "127.0.0.1:2181", + "batchSize": "5120", + "destination": "example", + "username": "", + "password": "", + "cluster": false, + "singleHostname": "127.0.0.1", + "singlePort": "11111", + } + + ``` * YAML - You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. - - ```yaml - configs: - zkServers: "127.0.0.1:2181" - batchSize: 5120 - destination: "example" - username: "" - password: "" - cluster: false - singleHostname: "127.0.0.1" - singlePort: 11111 - ``` + You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. + + ```yaml + + configs: + zkServers: "127.0.0.1:2181" + batchSize: 5120 + destination: "example" + username: "" + password: "" + cluster: false + singleHostname: "127.0.0.1" + singlePort: 11111 + + ``` ## Usage @@ -69,139 +73,167 @@ Here is an example of storing MySQL data using the configuration file as above. 1. Start a MySQL server. - ```bash - $ docker pull mysql:5.7 - $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 - ``` + ```bash + + $ docker pull mysql:5.7 + $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 + + ``` 2. Create a configuration file `mysqld.cnf`. - ```bash - [mysqld] - pid-file = /var/run/mysqld/mysqld.pid - socket = /var/run/mysqld/mysqld.sock - datadir = /var/lib/mysql - #log-error = /var/log/mysql/error.log - # By default we only accept connections from localhost - #bind-address = 127.0.0.1 - # Disabling symbolic-links is recommended to prevent assorted security risks - symbolic-links=0 - log-bin=mysql-bin - binlog-format=ROW - server_id=1 - ``` + ```bash + + [mysqld] + pid-file = /var/run/mysqld/mysqld.pid + socket = /var/run/mysqld/mysqld.sock + datadir = /var/lib/mysql + #log-error = /var/log/mysql/error.log + # By default we only accept connections from localhost + #bind-address = 127.0.0.1 + # Disabling symbolic-links is recommended to prevent assorted security risks + symbolic-links=0 + log-bin=mysql-bin + binlog-format=ROW + server_id=1 + + ``` 3. Copy the configuration file `mysqld.cnf` to MySQL server. + + ```bash - ```bash - $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ - ``` + $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ + + ``` 4. Restart the MySQL server. + + ```bash - ```bash - $ docker restart pulsar-mysql - ``` + $ docker restart pulsar-mysql + + ``` 5. Create a test database in MySQL server. + + ```bash - ```bash - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' - ``` + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' + + ``` 6. Start a Canal server and connect to MySQL server. - ``` - $ docker pull canal/canal-server:v1.1.2 - $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 - ``` + ``` + + $ docker pull canal/canal-server:v1.1.2 + $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 + + ``` 7. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:2.3.0 - $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone - ``` + ```bash + + $ docker pull apachepulsar/pulsar:2.3.0 + $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone + + ``` 8. Modify the configuration file `canal-mysql-source-config.yaml`. - ```yaml - configs: - zkServers: "" - batchSize: "5120" - destination: "test" - username: "" - password: "" - cluster: false - singleHostname: "pulsar-canal-server" - singlePort: "11111" - ``` + ```yaml + + configs: + zkServers: "" + batchSize: "5120" + destination: "test" + username: "" + password: "" + cluster: false + singleHostname: "pulsar-canal-server" + singlePort: "11111" + + ``` 9. Create a consumer file `pulsar-client.py`. - ```python - import pulsar + ```python + + import pulsar - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', - subscription_name='my-sub') + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', + subscription_name='my-sub') - while True: - msg = consumer.receive() - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + while True: + msg = consumer.receive() + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) - client.close() - ``` + client.close() + + ``` 10. Copy the configuration file `canal-mysql-source-config.yaml` and the consumer file `pulsar-client.py` to Pulsar server. - ```bash - $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ - $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ - ``` + ```bash + + $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ + $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ + + ``` 11. Download a Canal connector and start it. - - ```bash - $ docker exec -it pulsar-standalone /bin/bash - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors - $ ./bin/pulsar-admin source localrun \ - --archive ./connectors/pulsar-io-canal-2.3.0.nar \ - --classname org.apache.pulsar.io.canal.CanalStringSource \ - --tenant public \ - --namespace default \ - --name canal \ - --destination-topic-name my-topic \ - --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ - --parallelism 1 - ``` + + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors + $ ./bin/pulsar-admin source localrun \ + --archive ./connectors/pulsar-io-canal-2.3.0.nar \ + --classname org.apache.pulsar.io.canal.CanalStringSource \ + --tenant public \ + --namespace default \ + --name canal \ + --destination-topic-name my-topic \ + --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ + --parallelism 1 + + ``` 12. Consume data from MySQL. - ```bash - $ docker exec -it pulsar-standalone /bin/bash - $ python pulsar-client.py - ``` + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ python pulsar-client.py + + ``` 13. Open another window to log in MySQL server. - ```bash - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal - ``` + ```bash + + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal + + ``` 14. Create a table, and insert, delete, and update data in MySQL server. - - ```bash - mysql> use test; - mysql> show tables; - mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, - `test_author` VARCHAR(40) NOT NULL, - `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; - mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); - mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; - mysql> DELETE FROM test_table WHERE test_title='c'; - ``` + + ```bash + + mysql> use test; + mysql> show tables; + mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, + `test_author` VARCHAR(40) NOT NULL, + `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; + mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); + mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; + mysql> DELETE FROM test_table WHERE test_title='c'; + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-cassandra-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-cassandra-sink.md index 5d05c94a1542d..c79917ca80eec 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-cassandra-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-cassandra-sink.md @@ -1,7 +1,7 @@ --- id: io-cassandra-sink title: Cassandra sink connector -sidebar_label: Cassandra sink connector +sidebar_label: "Cassandra sink connector" original_id: io-cassandra-sink --- @@ -31,27 +31,30 @@ Before using the Cassandra sink connector, you need to create a configuration fi * JSON - ```json - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - ``` + ```json + + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` * YAML - - ``` - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" - ``` + ``` + + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" + + ``` ## Usage diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-cdc-debezium.md b/site2/website-next/versioned_docs/version-2.7.3/io-cdc-debezium.md index 02aa3041d1251..bc32077f562ae 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-cdc-debezium.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-cdc-debezium.md @@ -1,7 +1,7 @@ --- id: io-cdc-debezium title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" original_id: io-cdc-debezium --- @@ -25,11 +25,11 @@ The configuration of Debezium source connector has the following properties. | `database.password` | true | null | The password for a database user that has the required privileges. | | `database.server.id` | true | null | The connector’s identifier that must be unique within a database cluster and similar to the database’s server-id configuration property. | | `database.server.name` | true | null | The logical name of a database server/cluster, which forms a namespace and it is used in all the names of Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro Connector is used. | -| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | +| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | | `key.converter` | true | null | The converter provided by Kafka Connect to convert record key. | | `value.converter` | true | null | The converter provided by Kafka Connect to convert record value. | | `database.history` | true | null | The name of the database history class. | -| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | +| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | | `database.history.pulsar.service.url` | true | null | Pulsar cluster service URL for history topic. | | `pulsar.service.url` | true | null | Pulsar cluster service URL for the offset topic used in Debezium. You can use the `bin/pulsar-admin --admin-url http://pulsar:8080 sources localrun --source-config-file configs/pg-pulsar-config.yaml` command to point to the target Pulsar cluster. | | `offset.storage.topic` | true | null | Record the last committed offsets that the connector successfully completes. | @@ -51,61 +51,65 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "pulsar.service.url": "pulsar://127.0.0.1:6650", - "offset.storage.topic": "offset-topic" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - ``` + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -113,72 +117,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar`. + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - ``` + * Use the **YAML** configuration file as shown previously. - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -190,45 +208,49 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "postgres", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "schema.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "postgres", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "schema.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for pg, docker image: debezium/example-postgress:0.8 - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "postgres" - database.dbname: "postgres" - database.server.name: "dbserver1" - schema.whitelist: "inventory" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.8 + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "postgres" + database.dbname: "postgres" + database.server.name: "dbserver1" + schema.whitelist: "inventory" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -237,129 +259,150 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-postgres:0.8 - $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 - ``` + ```bash + + $ docker pull debezium/example-postgres:0.8 + $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - ``` + + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a PostgreSQL client in docker. + + ```bash - ```bash - $ docker exec -it pulsar-postgresql /bin/bash - ``` + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to change the data of the table _products_. - ``` - psql -U postgres postgres - postgres=# \c postgres; - You are now connected to database "postgres" as user "postgres". - postgres=# SET search_path TO inventory; - SET - postgres=# select * from products; - id | name | description | weight - -----+--------------------+---------------------------------------------------------+-------- - 102 | car battery | 12V car battery | 8.1 - 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 - 104 | hammer | 12oz carpenter's hammer | 0.75 - 105 | hammer | 14oz carpenter's hammer | 0.875 - 106 | hammer | 16oz carpenter's hammer | 1 - 107 | rocks | box of assorted rocks | 5.3 - 108 | jacket | water resistent black wind breaker | 0.1 - 109 | spare tire | 24 inch spare tire | 22.2 - 101 | 1111111111 | Small 2-wheel scooter | 3.14 - (9 rows) - - postgres=# UPDATE products SET name='1111111111' WHERE id=107; - UPDATE 1 - ``` + ``` + + psql -U postgres postgres + postgres=# \c postgres; + You are now connected to database "postgres" as user "postgres". + postgres=# SET search_path TO inventory; + SET + postgres=# select * from products; + id | name | description | weight + -----+--------------------+---------------------------------------------------------+-------- + 102 | car battery | 12V car battery | 8.1 + 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 + 104 | hammer | 12oz carpenter's hammer | 0.75 + 105 | hammer | 14oz carpenter's hammer | 0.875 + 106 | hammer | 16oz carpenter's hammer | 1 + 107 | rocks | box of assorted rocks | 5.3 + 108 | jacket | water resistent black wind breaker | 0.1 + 109 | spare tire | 24 inch spare tire | 22.2 + 101 | 1111111111 | Small 2-wheel scooter | 3.14 + (9 rows) + + postgres=# UPDATE products SET name='1111111111' WHERE id=107; + UPDATE 1 + + ``` + + In the terminal window of subscribing topic, you can receive the following messages. + + ```bash + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar" - parallelism: 1 - - configs: + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - ## config for pg, docker image: debezium/example-postgress:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -368,78 +411,106 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 - ``` - Use the following commands to initialize the data. + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` + + Use the following commands to initialize the data. + + ``` bash - ``` bash - ./usr/local/bin/init-inventory.sh - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` + ./usr/local/bin/init-inventory.sh + + ``` + + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ``` + + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + ``` + + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml - ``` + + ``` + + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MongoDB client in docker. + + ``` + - ```bash - $ docker exec -it pulsar-mongodb /bin/bash - ``` + $ docker exec -it pulsar-mongodb /bin/bash + + ``` 6. A MongoDB client pops out. + + ``` + - ```bash - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) - ``` + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` + + In the terminal window of subscribing topic, you can receive the following messages. + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. - ``` + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + + ``` + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -449,15 +520,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -470,10 +541,14 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupJavaInstance(JavaInstanceRunnable.java:200) at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` + +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-cdc.md b/site2/website-next/versioned_docs/version-2.7.3/io-cdc.md index cbce7fc16d961..9ce8f7fc39101 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-cdc.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-cdc.md @@ -1,7 +1,7 @@ --- id: io-cdc title: CDC connector -sidebar_label: CDC connector +sidebar_label: "CDC connector" original_id: io-cdc --- @@ -11,19 +11,14 @@ import TabItem from '@theme/TabItem'; CDC source connectors capture log changes of databases (such as MySQL, MongoDB, and PostgreSQL) into Pulsar. -:::note - -CDC source connectors are built on top of [Canal](https://github.com/alibaba/canal) and [Debezium](https://debezium.io/) and store all data into Pulsar cluster in a persistent, replicated, and partitioned way. - -::: - +> CDC source connectors are built on top of [Canal](https://github.com/alibaba/canal) and [Debezium](https://debezium.io/) and store all data into Pulsar cluster in a persistent, replicated, and partitioned way. Currently, Pulsar has the following CDC connectors. Name|Java Class |---|--- -[Canal source connector](io-canal-source.md)|[org.apache.pulsar.io.canal.CanalStringSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalStringSource.java) -[Debezium source connector](io-cdc-debezium.md)|
  • [org.apache.pulsar.io.debezium.DebeziumSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java)
  • [org.apache.pulsar.io.debezium.mysql.DebeziumMysqlSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/java/org/apache/pulsar/io/debezium/mysql/DebeziumMysqlSource.java)
  • [org.apache.pulsar.io.debezium.postgres.DebeziumPostgresSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/java/org/apache/pulsar/io/debezium/postgres/DebeziumPostgresSource.java)
  • +[Canal source connector](io-canal-source)|[org.apache.pulsar.io.canal.CanalStringSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalStringSource.java) +[Debezium source connector](io-cdc-debezium)|
  • [org.apache.pulsar.io.debezium.DebeziumSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java)
  • [org.apache.pulsar.io.debezium.mysql.DebeziumMysqlSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/java/org/apache/pulsar/io/debezium/mysql/DebeziumMysqlSource.java)
  • [org.apache.pulsar.io.debezium.postgres.DebeziumPostgresSource.java](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/java/org/apache/pulsar/io/debezium/postgres/DebeziumPostgresSource.java)
  • For more information about Canal and Debezium, see the information below. diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-cli.md b/site2/website-next/versioned_docs/version-2.7.3/io-cli.md index 7c66aa306fb9f..886affed7bfcb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-cli.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-cli.md @@ -1,7 +1,7 @@ --- id: io-cli title: Connector Admin CLI -sidebar_label: CLI +sidebar_label: "CLI" original_id: io-cli --- @@ -16,7 +16,9 @@ The `pulsar-admin` tool helps you manage Pulsar connectors. An interface for managing Pulsar IO sources (ingress data into Pulsar). ```bash + $ pulsar-admin sources subcommands + ``` Subcommands are: @@ -53,7 +55,9 @@ Submit a Pulsar IO source connector to run in a Pulsar cluster. #### Usage ```bash + $ pulsar-admin sources create options + ``` #### Options @@ -85,7 +89,9 @@ Update a already submitted Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources update options + ``` #### Options @@ -118,7 +124,9 @@ Delete a Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources delete options + ``` #### Option @@ -136,7 +144,9 @@ Get the information about a Pulsar IO source connector. #### Usage ```bash + $ pulsar-admin sources get options + ``` #### Options @@ -154,7 +164,9 @@ Check the current status of a Pulsar Source. #### Usage ```bash + $ pulsar-admin sources status options + ``` #### Options @@ -173,7 +185,9 @@ List all running Pulsar IO source connectors. #### Usage ```bash + $ pulsar-admin sources list options + ``` #### Options @@ -191,7 +205,9 @@ Stop a source instance. #### Usage ```bash + $ pulsar-admin sources stop options + ``` #### Options @@ -210,7 +226,9 @@ Start a source instance. #### Usage ```bash + $ pulsar-admin sources start options + ``` #### Options @@ -230,7 +248,9 @@ Restart a source instance. #### Usage ```bash + $ pulsar-admin sources restart options + ``` #### Options @@ -249,7 +269,9 @@ Run a Pulsar IO source connector locally rather than deploying it to the Pulsar #### Usage ```bash + $ pulsar-admin sources localrun options + ``` #### Options @@ -288,7 +310,9 @@ Get the list of Pulsar IO connector sources supported by Pulsar cluster. #### Usage ```bash + $ pulsar-admin sources available-sources + ``` ### `reload` @@ -298,7 +322,9 @@ Reload the available built-in connectors. #### Usage ```bash + $ pulsar-admin sources reload + ``` ## `sinks` @@ -306,7 +332,9 @@ $ pulsar-admin sources reload An interface for managing Pulsar IO sinks (egress data from Pulsar). ```bash + $ pulsar-admin sinks subcommands + ``` Subcommands are: @@ -343,7 +371,9 @@ Submit a Pulsar IO sink connector to run in a Pulsar cluster. #### Usage ```bash + $ pulsar-admin sinks create options + ``` #### Options @@ -379,7 +409,9 @@ Update a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks update options + ``` #### Options @@ -416,7 +448,9 @@ Delete a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks delete options + ``` #### Option @@ -434,7 +468,9 @@ Get the information about a Pulsar IO sink connector. #### Usage ```bash + $ pulsar-admin sinks get options + ``` #### Options @@ -452,7 +488,9 @@ Check the current status of a Pulsar sink. #### Usage ```bash + $ pulsar-admin sinks status options + ``` #### Options @@ -472,7 +510,9 @@ List all running Pulsar IO sink connectors. #### Usage ```bash + $ pulsar-admin sinks list options + ``` #### Options @@ -490,7 +530,9 @@ Stop a sink instance. #### Usage ```bash + $ pulsar-admin sinks stop options + ``` #### Options @@ -509,7 +551,9 @@ Start a sink instance. #### Usage ```bash + $ pulsar-admin sinks start options + ``` #### Options @@ -529,7 +573,9 @@ Restart a sink instance. #### Usage ```bash + $ pulsar-admin sinks restart options + ``` #### Options @@ -549,7 +595,9 @@ Run a Pulsar IO sink connector locally rather than deploying it to the Pulsar cl #### Usage ```bash + $ pulsar-admin sinks localrun options + ``` #### Options @@ -595,7 +643,9 @@ Get the list of Pulsar IO connector sinks supported by Pulsar cluster. #### Usage ```bash + $ pulsar-admin sinks available-sinks + ``` ### `reload` @@ -605,6 +655,8 @@ Reload the available built-in connectors. #### Usage ```bash + $ pulsar-admin sinks reload + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-connectors.md b/site2/website-next/versioned_docs/version-2.7.3/io-connectors.md index 47ee183c6367b..3e0924a7f3c91 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-connectors.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-connectors.md @@ -1,7 +1,7 @@ --- id: io-connectors title: Built-in connector -sidebar_label: Built-in connector +sidebar_label: "Built-in connector" original_id: io-connectors --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-debezium-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-debezium-source.md index 29ca3ef35c27f..692d1d64e9ead 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-debezium-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-debezium-source.md @@ -1,7 +1,7 @@ --- id: io-debezium-source title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" original_id: io-debezium-source --- @@ -25,11 +25,11 @@ The configuration of Debezium source connector has the following properties. | `database.password` | true | null | The password for a database user that has the required privileges. | | `database.server.id` | true | null | The connector’s identifier that must be unique within a database cluster and similar to the database’s server-id configuration property. | | `database.server.name` | true | null | The logical name of a database server/cluster, which forms a namespace and it is used in all the names of Kafka topics to which the connector writes, the Kafka Connect schema names, and the namespaces of the corresponding Avro schema when the Avro Connector is used. | -| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | +| `database.whitelist` | false | null | A list of all databases hosted by this server which is monitored by the connector.

    This is optional, and there are other properties for listing databases and tables to include or exclude from monitoring. | | `key.converter` | true | null | The converter provided by Kafka Connect to convert record key. | | `value.converter` | true | null | The converter provided by Kafka Connect to convert record value. | | `database.history` | true | null | The name of the database history class. | -| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | +| `database.history.pulsar.topic` | true | null | The name of the database history topic where the connector writes and recovers DDL statements.

    **Note: this topic is for internal use only and should not be used by consumers.** | | `database.history.pulsar.service.url` | true | null | Pulsar cluster service URL for history topic. | | `pulsar.service.url` | true | null | Pulsar cluster service URL. | | `offset.storage.topic` | true | null | Record the last committed offsets that the connector successfully completes. | @@ -72,61 +72,65 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "pulsar.service.url": "pulsar://127.0.0.1:6650", - "offset.storage.topic": "offset-topic" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - ``` + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -134,72 +138,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar`. + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-{{pulsar:version}}.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - ``` + * Use the **YAML** configuration file as shown previously. - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -211,50 +229,54 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "changeme", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "plugin.name": "pgoutput", - "schema.whitelist": "public", - "table.whitelist": "public.users", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "changeme", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "plugin.name": "pgoutput", + "schema.whitelist": "public", + "table.whitelist": "public.users", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar" - parallelism: 1 - - configs: - - ## config for postgres version 10+, official docker image: postgres:<10+> - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "changeme" - database.dbname: "postgres" - database.server.name: "dbserver1" - plugin.name: "pgoutput" - schema.whitelist: "public" - table.whitelist: "public.users" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` - + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. + + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for postgres version 10+, official docker image: postgres:<10+> + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "changeme" + database.dbname: "postgres" + database.server.name: "dbserver1" + plugin.name: "pgoutput" + schema.whitelist: "public" + table.whitelist: "public.users" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` + Notice that `pgoutput` is a standard plugin of Postgres introduced in version 10 - [see Postgres architecture docu](https://www.postgresql.org/docs/10/logical-replication-architecture.html). You don't need to install anything, just make sure the WAL level is set to `logical` (see docker command below and [Postgres docu](https://www.postgresql.org/docs/current/runtime-config-wal.html)). ### Usage @@ -264,137 +286,158 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=changeme \ - postgres:13.3 -c wal_level=logical - ``` + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=changeme \ + postgres:13.3 -c wal_level=logical + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-{{pulsar:version}}.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - ``` + + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-users_ for the _public.users_ table. - ``` - $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 - ``` + ``` + + $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 + + ``` 5. Start a PostgreSQL client in docker. + + ```bash - ```bash - $ docker exec -it pulsar-postgresql /bin/bash - ``` + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to create sample data in the table _users_. - ``` - psql -U postgres -h localhost -p 5432 - Password for user postgres: - - CREATE TABLE users( - id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), - hash_firstname TEXT NOT NULL, - hash_lastname TEXT NOT NULL, - gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) - ); - - INSERT INTO users(hash_firstname, hash_lastname, gender) - SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); - - postgres=# select * from users; - - id | hash_firstname | hash_lastname | gender - -------+----------------------------------+----------------------------------+-------- - 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female - 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male - 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female - 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female + ``` + + psql -U postgres -h localhost -p 5432 + Password for user postgres: + + CREATE TABLE users( + id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), + hash_firstname TEXT NOT NULL, + hash_lastname TEXT NOT NULL, + gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) + ); + + INSERT INTO users(hash_firstname, hash_lastname, gender) + SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); + + postgres=# select * from users; + + id | hash_firstname | hash_lastname | gender + -------+----------------------------------+----------------------------------+-------- + 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female + 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male + 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female + 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female - - postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; - UPDATE 1 - ``` + + postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; + UPDATE 1 + + ``` + + In the terminal window of subscribing topic, you can receive the following messages. + + ```bash + + ----- got message ----- + {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} + ...many more + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} - ...many more - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - - ```yaml - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar" - parallelism: 1 + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - configs: - - ## config for pg, docker image: debezium/example-mongodb:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-mongodb:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -403,78 +446,106 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 - ``` - Use the following commands to initialize the data. + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` + + Use the following commands to initialize the data. + + ``` bash - ``` bash - ./usr/local/bin/init-inventory.sh - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` + ./usr/local/bin/init-inventory.sh + + ``` + + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` + ``` + + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-{{pulsar:version}}.nar`. - - ```bash - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-{{pulsar:version}}.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' - ``` - + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + ``` + + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` + * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml - ``` + + ``` + + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - ``` + ``` + + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MongoDB client in docker. + + ``` + - ```bash - $ docker exec -it pulsar-mongodb /bin/bash - ``` + $ docker exec -it pulsar-mongodb /bin/bash + + ``` 6. A MongoDB client pops out. + + ``` + - ```bash - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) - ``` + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` - In the terminal window of subscribing topic, you can receive the following messages. - - ```bash - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. - ``` + In the terminal window of subscribing topic, you can receive the following messages. + + ``` + + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + ``` + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -484,15 +555,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -505,10 +576,14 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.setupJavaInstance(JavaInstanceRunnable.java:200) at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` + +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-debug.md b/site2/website-next/versioned_docs/version-2.7.3/io-debug.md index fb4b95b45bef4..f815e862cae42 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-debug.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-debug.md @@ -1,7 +1,7 @@ --- id: io-debug title: How to debug Pulsar connectors -sidebar_label: Debug +sidebar_label: "Debug" original_id: io-debug --- @@ -13,45 +13,63 @@ To better demonstrate how to debug Pulsar connectors, here takes a Mongo sink co **Deploy a Mongo sink environment** 1. Start a Mongo service. - ```bash - docker pull mongo:4 - docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 - ``` -2. Create a DB and a collection. - ```bash - docker exec -it pulsar-mongo /bin/bash - mongo - :::note - use pulsar - db.createCollection('messages') - exit + ```bash + + docker pull mongo:4 + docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 + + ``` + +2. Create a DB and a collection. - ::: + ```bash + + docker exec -it pulsar-mongo /bin/bash + mongo + > use pulsar + > db.createCollection('messages') + > exit + + ``` - ``` 3. Start Pulsar standalone. - ```bash - docker pull apachepulsar/pulsar:2.4.0 - docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone - ``` + + ```bash + + docker pull apachepulsar/pulsar:2.4.0 + docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + + ``` + 4. Configure the Mongo sink with the `mongo-sink-config.yaml` file. - ```bash - configs: - mongoUri: "mongodb://pulsar-mongo:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - ``` - ```bash - docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ - ``` + + ```bash + + configs: + mongoUri: "mongodb://pulsar-mongo:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + + ``` + + ```bash + + docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ + + ``` + 5. Download the Mongo sink nar package. - ```bash - docker exec -it pulsar-mongo-standalone /bin/bash - curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar - ``` + + ```bash + + docker exec -it pulsar-mongo-standalone /bin/bash + curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar + + ``` + ## Debug in localrun mode Start the Mongo sink in localrun mode using the `localrun` command. :::tip @@ -61,6 +79,7 @@ For more information about the `localrun` command, see [`localrun`](reference-co ::: ```bash + ./bin/pulsar-admin sinks localrun \ --archive pulsar-io-mongo-2.4.0.nar \ --tenant public --namespace default \ @@ -68,122 +87,146 @@ For more information about the `localrun` command, see [`localrun`](reference-co --name pulsar-mongo-sink \ --sink-config-file mongo-sink-config.yaml \ --parallelism 1 + ``` + ### Use connector log Use one of the following methods to get a connector log in localrun mode: * After executing the `localrun` command, the **log is automatically printed on the console**. * The log is located at: + + ```bash - ```bash - logs/functions/tenant/namespace/function-name/function-name-instance-id.log - ``` - - **Example** - - The path of the Mongo sink connector is: - ```bash - logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log - ``` + logs/functions/tenant/namespace/function-name/function-name-instance-id.log + + ``` + + **Example** + + The path of the Mongo sink connector is: + + ```bash + + logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log + + ``` + To clearly explain the log information, here breaks down the large block of information into small blocks and add descriptions for each block. * This piece of log information shows the storage path of the nar package after decompression. - ``` - 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, - ``` - :::tip - If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. + ``` + + 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, + + ``` + + :::tip - ::: + If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. + + ::: * This piece of log information illustrates the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, resources, and so on, which can be used to **check whether the Mongo sink connector is configured correctly or not**. - ```bash - 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" - namespace: "default" - name: "pulsar-mongo-sink" - className: "org.apache.pulsar.functions.api.utils.IdentityFunction" - autoAck: true - parallelism: 1 - source { - typeClassName: "[B" - inputSpecs { - key: "test-mongo" - value { - } - } - cleanupSubscription: true - } - sink { - className: "org.apache.pulsar.io.mongodb.MongoSink" - configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" - typeClassName: "[B" - } - resources { - cpu: 1.0 - ram: 1073741824 - disk: 10737418240 - } - componentType: SINK - , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) - ``` + + ```bash + + 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" + namespace: "default" + name: "pulsar-mongo-sink" + className: "org.apache.pulsar.functions.api.utils.IdentityFunction" + autoAck: true + parallelism: 1 + source { + typeClassName: "[B" + inputSpecs { + key: "test-mongo" + value { + } + } + cleanupSubscription: true + } + sink { + className: "org.apache.pulsar.io.mongodb.MongoSink" + configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" + typeClassName: "[B" + } + resources { + cpu: 1.0 + ram: 1073741824 + disk: 10737418240 + } + componentType: SINK + , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) + + ``` + * This piece of log information demonstrates the status of the connections to Mongo and configuration information. - ```bash - 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 - 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} - ``` + + ```bash + + 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 + 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} + + ``` + * This piece of log information explains the configuration of consumers and clients, including the topic name, subscription name, subscription type, and so on. - ```bash - 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { - "topicNames" : [ "test-mongo" ], - "topicsPattern" : null, - "subscriptionName" : "public/default/pulsar-mongo-sink", - "subscriptionType" : "Shared", - "receiverQueueSize" : 1000, - "acknowledgementsGroupTimeMicros" : 100000, - "negativeAckRedeliveryDelayMicros" : 60000000, - "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, - "consumerName" : null, - "ackTimeoutMillis" : 0, - "tickDurationMillis" : 1000, - "priorityLevel" : 0, - "cryptoFailureAction" : "CONSUME", - "properties" : { - "application" : "pulsar-sink", - "id" : "public/default/pulsar-mongo-sink", - "instance_id" : "0" - }, - "readCompacted" : false, - "subscriptionInitialPosition" : "Latest", - "patternAutoDiscoveryPeriod" : 1, - "regexSubscriptionMode" : "PersistentOnly", - "deadLetterPolicy" : null, - "autoUpdatePartitions" : true, - "replicateSubscriptionState" : false, - "resetIncludeHead" : false - } - 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { - "serviceUrl" : "pulsar://localhost:6650", - "authPluginClassName" : null, - "authParams" : null, - "operationTimeoutMs" : 30000, - "statsIntervalSeconds" : 60, - "numIoThreads" : 1, - "numListenerThreads" : 1, - "connectionsPerBroker" : 1, - "useTcpNoDelay" : true, - "useTls" : false, - "tlsTrustCertsFilePath" : null, - "tlsAllowInsecureConnection" : false, - "tlsHostnameVerificationEnable" : false, - "concurrentLookupRequest" : 5000, - "maxLookupRequest" : 50000, - "maxNumberOfRejectedRequestPerConnection" : 50, - "keepAliveIntervalSeconds" : 30, - "connectionTimeoutMs" : 10000, - "requestTimeoutMs" : 60000, - "defaultBackoffIntervalNanos" : 100000000, - "maxBackoffIntervalNanos" : 30000000000 - } - ``` + + ```bash + + 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { + "topicNames" : [ "test-mongo" ], + "topicsPattern" : null, + "subscriptionName" : "public/default/pulsar-mongo-sink", + "subscriptionType" : "Shared", + "receiverQueueSize" : 1000, + "acknowledgementsGroupTimeMicros" : 100000, + "negativeAckRedeliveryDelayMicros" : 60000000, + "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, + "consumerName" : null, + "ackTimeoutMillis" : 0, + "tickDurationMillis" : 1000, + "priorityLevel" : 0, + "cryptoFailureAction" : "CONSUME", + "properties" : { + "application" : "pulsar-sink", + "id" : "public/default/pulsar-mongo-sink", + "instance_id" : "0" + }, + "readCompacted" : false, + "subscriptionInitialPosition" : "Latest", + "patternAutoDiscoveryPeriod" : 1, + "regexSubscriptionMode" : "PersistentOnly", + "deadLetterPolicy" : null, + "autoUpdatePartitions" : true, + "replicateSubscriptionState" : false, + "resetIncludeHead" : false + } + 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { + "serviceUrl" : "pulsar://localhost:6650", + "authPluginClassName" : null, + "authParams" : null, + "operationTimeoutMs" : 30000, + "statsIntervalSeconds" : 60, + "numIoThreads" : 1, + "numListenerThreads" : 1, + "connectionsPerBroker" : 1, + "useTcpNoDelay" : true, + "useTls" : false, + "tlsTrustCertsFilePath" : null, + "tlsAllowInsecureConnection" : false, + "tlsHostnameVerificationEnable" : false, + "concurrentLookupRequest" : 5000, + "maxLookupRequest" : 50000, + "maxNumberOfRejectedRequestPerConnection" : 50, + "keepAliveIntervalSeconds" : 30, + "connectionTimeoutMs" : 10000, + "requestTimeoutMs" : 60000, + "defaultBackoffIntervalNanos" : 100000000, + "maxBackoffIntervalNanos" : 30000000000 + } + + ``` + ## Debug in cluster mode You can use the following methods to debug a connector in cluster mode: * [Use connector log](#use-connector-log) @@ -198,7 +241,9 @@ Pulsar admin CLI helps you debug Pulsar connectors with the following subcommand * [`topics stats`](#topics-stats) **Create a Mongo sink** + ```bash + ./bin/pulsar-admin sinks create \ --archive pulsar-io-mongo-2.4.0.nar \ --tenant public \ @@ -207,10 +252,14 @@ Pulsar admin CLI helps you debug Pulsar connectors with the following subcommand --name pulsar-mongo-sink \ --sink-config-file mongo-sink-config.yaml \ --parallelism 1 + ``` + ### `get` Use the `get` command to get the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, and so on. + ```bash + ./bin/pulsar-admin sinks get --tenant public --namespace default --name pulsar-mongo-sink { "tenant": "public", @@ -234,7 +283,9 @@ Use the `get` command to get the basic information about the Mongo sink connecto "retainOrdering": false, "autoAck": true } + ``` + :::tip For more information about the `get` command, see [`get`](reference-connector-admin.md/#get-1). @@ -243,7 +294,9 @@ For more information about the `get` command, see [`get`](reference-connector-ad ### `status` Use the `status` command to get the current status about the Mongo sink connector, such as the number of instance, the number of running instance, instanceId, workerId and so on. + ```bash + ./bin/pulsar-admin sinks status --tenant public \ --namespace default \ @@ -268,18 +321,21 @@ Use the `status` command to get the current status about the Mongo sink connecto } } ] } + ``` + :::tip For more information about the `status` command, see [`status`](reference-connector-admin.md/#stauts-1). - If there are multiple connectors running on a worker, `workerId` can locate the worker on which the specified connector is running. ::: ### `topics stats` Use the `topics stats` command to get the stats for a topic and its connected producer and consumer, such as whether the topic has received messages or not, whether there is a backlog of messages or not, the available permits and other key information. All rates are computed over a 1-minute window and are relative to the last completed 1-minute period. + ```bash + ./bin/pulsar-admin topics stats test-mongo { "msgRateIn" : 0.0, @@ -323,7 +379,9 @@ Use the `topics stats` command to get the stats for a topic and its connected pr "replication" : { }, "deduplicationStatus" : "Disabled" } + ``` + :::tip For more information about the `topic stats` command, see [`topic stats`](http://pulsar.apache.org/docs/en/pulsar-admin/#stats-1). diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-develop.md b/site2/website-next/versioned_docs/version-2.7.3/io-develop.md index 07319cee85f73..d7531a0d30d67 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-develop.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-develop.md @@ -1,7 +1,7 @@ --- id: io-develop title: How to develop Pulsar connectors -sidebar_label: Develop +sidebar_label: "Develop" original_id: io-develop --- @@ -12,15 +12,15 @@ import TabItem from '@theme/TabItem'; This guide describes how to develop Pulsar connectors to move data between Pulsar and other systems. -Pulsar connectors are special [Pulsar Functions](functions-overview.md), so creating +Pulsar connectors are special [Pulsar Functions](functions-overview), so creating a Pulsar connector is similar to creating a Pulsar function. Pulsar connectors come in two types: | Type | Description | Example |---|---|--- -{@inject: github:Source:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java}|Import data from another system to Pulsar.|[RabbitMQ source connector](io-rabbitmq.md) imports the messages of a RabbitMQ queue to a Pulsar topic. -{@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java}|Export data from Pulsar to another system.|[Kinesis sink connector](io-kinesis.md) exports the messages of a Pulsar topic to a Kinesis stream. +{@inject: github:Source:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java}|Import data from another system to Pulsar.|[RabbitMQ source connector](io-rabbitmq) imports the messages of a RabbitMQ queue to a Pulsar topic. +{@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java}|Export data from Pulsar to another system.|[Kinesis sink connector](io-kinesis) exports the messages of a Pulsar topic to a Kinesis stream. ## Develop @@ -33,62 +33,66 @@ interface, which means you need to implement the {@inject: github:open:/pulsar-i 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java - /** - * Open connector with configuration - * - * @param config initialization config - * @param sourceContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SourceContext sourceContext) throws Exception; - ``` + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sourceContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SourceContext sourceContext) throws Exception; + + ``` - This method is called when the source connector is initialized. + This method is called when the source connector is initialized. - In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. - - For example, a Kafka connector can create a Kafka client in this `open` method. + In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. + + For example, a Kafka connector can create a Kafka client in this `open` method. - Besides, Pulsar runtime also provides a `SourceContext` for the - connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. + Besides, Pulsar runtime also provides a `SourceContext` for the + connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. 2. Implement the {@inject: github:read:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java - /** - * Reads the next message from source. - * If source does not have any new messages, this call should block. - * @return next message from source. The return result should never be null - * @throws Exception - */ - Record read() throws Exception; - ``` - - If nothing to return, the implementation should be blocking rather than returning `null`. - - The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: - - |Variable|Required|Description - |---|---|--- - `TopicName`|No|Pulsar topic name from which the record is originated from. - `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| - `Value`|Yes|Actual data of the record. - `EventTime`|No|Event time of the record from the source. - `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `Properties` |No| If the record carries user-defined properties, it returns those properties. - `DestinationTopic`|No|Topic to which message should be written. - `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: - - Method|Description - |---|--- - `ack` |Acknowledge that the record is fully processed. - `fail`|Indicate that the record fails to be processed. + ```java + + /** + * Reads the next message from source. + * If source does not have any new messages, this call should block. + * @return next message from source. The return result should never be null + * @throws Exception + */ + Record read() throws Exception; + + ``` + + If nothing to return, the implementation should be blocking rather than returning `null`. + + The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: + + |Variable|Required|Description + |---|---|--- + `TopicName`|No|Pulsar topic name from which the record is originated from. + `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| + `Value`|Yes|Actual data of the record. + `EventTime`|No|Event time of the record from the source. + `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `Properties` |No| If the record carries user-defined properties, it returns those properties. + `DestinationTopic`|No|Topic to which message should be written. + `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: + + Method|Description + |---|--- + `ack` |Acknowledge that the record is fully processed. + `fail`|Indicate that the record fails to be processed. :::tip @@ -96,40 +100,43 @@ For more information about **how to create a source connector**, see {@inject: g ::: - ### Sink Developing a sink connector **is similar to** developing a source connector, that is, you need to implement the {@inject: github:Sink:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} interface, which means implementing the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method and the {@inject: github:write:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - /** - * Open connector with configuration - * - * @param config initialization config - * @param sinkContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SinkContext sinkContext) throws Exception; - ``` + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sinkContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SinkContext sinkContext) throws Exception; + + ``` 2. Implement the {@inject: github:write:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - /** - * Write a message to Sink - * @param record record to write to sink - * @throws Exception - */ - void write(Record record) throws Exception; - ``` + ```java + + /** + * Write a message to Sink + * @param record record to write to sink + * @throws Exception + */ + void write(Record record) throws Exception; + + ``` - During the implementation, you can decide how to write the `Value` and - the `Key` to the actual source, and leverage all the provided information such as - `PartitionId` and `RecordSequence` to achieve different processing guarantees. + During the implementation, you can decide how to write the `Value` and + the `Key` to the actual source, and leverage all the provided information such as + `PartitionId` and `RecordSequence` to achieve different processing guarantees. - You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). + You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). ## Test @@ -149,8 +156,7 @@ You can create unit tests for your connector. Once you have written sufficient unit tests, you can add separate integration tests to verify end-to-end functionality. -Pulsar uses -[testcontainers](https://www.testcontainers.org/) **for all integration tests**. +Pulsar uses [testcontainers](https://www.testcontainers.org/) **for all integration tests**. :::tip @@ -158,11 +164,10 @@ For more information about **how to create integration tests for Pulsar connecto ::: - ## Package Once you've developed and tested your connector, you need to package it so that it can be submitted -to a [Pulsar Functions](functions-overview.md) cluster. +to a [Pulsar Functions](functions-overview) cluster. There are two methods to work with Pulsar Functions' runtime, that is, [NAR](#nar) and [uber JAR](#uber-jar). @@ -175,12 +180,8 @@ If you plan to package and distribute your connector for others to use, you are license and copyright your own code properly. Remember to add the license and copyright to all libraries your code uses and to your distribution. -:::note - -If you use the [NAR](#nar) method, the NAR plugin - -::: - +> +> If you use the [NAR](#nar) method, the NAR plugin automatically creates a `DEPENDENCIES` file in the generated NAR package, including the proper licensing and copyrights of all libraries of your connector. @@ -191,20 +192,18 @@ a bit of Java ClassLoader isolation. :::tip -For more information about **how NAR works**, see -[here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). +For more information about **how NAR works**, see [here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). ::: - Pulsar uses the same mechanism for packaging **all** [built-in connectors](io-connectors). -The easiest approach to package a Pulsar connector is to create a NAR package using -[nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin). +The easiest approach to package a Pulsar connector is to create a NAR package using [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin). Include this [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apache.nifi/nifi-nar-maven-plugin) in your maven project for your connector as below. ```xml + org.apache.nifi @@ -212,15 +211,18 @@ Include this [nifi-nar-maven-plugin](https://mvnrepository.com/artifact/org.apac 1.2.0 + ``` You must also create a `resources/META-INF/services/pulsar-io.yaml` file with the following contents: ```yaml + name: connector name description: connector description sourceClass: fully qualified class name (only if source connector) sinkClass: fully qualified class name (only if sink connector) + ``` For Gradle users, there is a [Gradle Nar plugin available on the Gradle Plugin Portal](https://plugins.gradle.org/plugin/io.github.lhotari.gradle-nar-plugin). @@ -231,7 +233,6 @@ For more information about an **how to use NAR for Pulsar connectors**, see {@in ::: - ### Uber JAR An alternative approach is to create an **uber JAR** that contains all of the connector's JAR files @@ -240,6 +241,7 @@ and other resource files. No directory internal structure is necessary. You can use [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-plugin/examples/includes-excludes.html) to create a uber JAR as below: ```xml + org.apache.maven.plugins maven-shade-plugin @@ -260,4 +262,6 @@ You can use [maven-shade-plugin](https://maven.apache.org/plugins/maven-shade-pl + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-dynamodb-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-dynamodb-source.md index 5c9aa2c4e6671..4a93683eb4a5b 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-dynamodb-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-dynamodb-source.md @@ -1,7 +1,7 @@ --- id: io-dynamodb-source title: AWS DynamoDB source connector -sidebar_label: AWS DynamoDB source connector +sidebar_label: "AWS DynamoDB source connector" original_id: io-dynamodb-source --- @@ -24,19 +24,19 @@ The configuration of the DynamoDB source connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • +`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • `startAtTime`|Date|false|" " (empty string)|If set to `AT_TIMESTAMP`, it specifies the point in time to start consumption. -`applicationName`|String|false|Pulsar IO connector|The name of the KCL application. Must be unique, as it is used to define the table name for the dynamo table used for state tracking.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. +`applicationName`|String|false|Pulsar IO connector|The name of the KCL application. Must be unique, as it is used to define the table name for the dynamo table used for state tracking.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. `checkpointInterval`|long|false|60000|The frequency of the KCL checkpoint in milliseconds. `backoffTime`|long|false|3000|The amount of time to delay between requests when the connector encounters a throttling exception from AWS Kinesis in milliseconds. `numRetries`|int|false|3|The number of re-attempts when the connector encounters an exception while trying to set a checkpoint. -`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. +`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. `dynamoEndpoint`|String|false|" " (empty string)|The Dynamo end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `cloudwatchEndpoint`|String|false|" " (empty string)|The Cloudwatch end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `awsEndpoint`|String|false|" " (empty string)|The DynamoDB Streams end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsDynamodbStreamArn`|String|true|" " (empty string)|The DynamoDB stream arn. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Example @@ -45,36 +45,40 @@ Before using the DynamoDB source connector, you need to create a configuration f * JSON - ```json - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-elasticsearch-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-elasticsearch-sink.md index 53dd0bc7d697b..a2624bdca129d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-elasticsearch-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-elasticsearch-sink.md @@ -1,7 +1,7 @@ --- id: io-elasticsearch-sink title: ElasticSearch sink connector -sidebar_label: ElasticSearch sink connector +sidebar_label: "ElasticSearch sink connector" original_id: io-elasticsearch-sink --- @@ -37,106 +37,141 @@ Before using the ElasticSearch sink connector, you need to create a configuratio * JSON - ```json - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "username": "scooby", - "password": "doobie" - } - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - username: "scooby" - password: "doobie" - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + username: "scooby" + password: "doobie" + + ``` #### For Elasticsearch Before 6.2 * JSON - ```json - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "typeName": "doc", - "username": "scooby", - "password": "doobie" - } - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "typeName": "doc", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - typeName: "doc" - username: "scooby" - password: "doobie" - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + typeName: "doc" + username: "scooby" + password: "doobie" + + ``` ### Usage 1. Start a single node Elasticsearch cluster. - ```bash - $ docker run -p 9200:9200 -p 9300:9300 \ - -e "discovery.type=single-node" \ - docker.elastic.co/elasticsearch/elasticsearch:7.5.1 - ``` + ```bash + + $ docker run -p 9200:9200 -p 9300:9300 \ + -e "discovery.type=single-node" \ + docker.elastic.co/elasticsearch/elasticsearch:7.5.1 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - $ bin/pulsar standalone - ``` - Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar`. + + ```bash + + $ bin/pulsar standalone + + ``` + + Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-@pulsar:version@.nar`. 3. Start the Pulsar Elasticsearch connector in local run mode using one of the following methods. - * Use the **JSON** configuration as shown previously. - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ - --inputs elasticsearch_test - ``` - * Use the **YAML** configuration file as shown previously. - - ```bash - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-{{pulsar:version}}.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config-file elasticsearch-sink.yml \ - --inputs elasticsearch_test - ``` + * Use the **JSON** configuration as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ + --inputs elasticsearch_test + + ``` + + * Use the **YAML** configuration file as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config-file elasticsearch-sink.yml \ + --inputs elasticsearch_test + + ``` 4. Publish records to the topic. - ```bash - $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" - ``` + ```bash + + $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" + + ``` 5. Check documents in Elasticsearch. - - * refresh the index - ```bash - $ curl -s http://localhost:9200/my_index/_refresh - ``` - * search documents - ```bash - $ curl -s http://localhost:9200/my_index/_search - ``` - You can see the record that published earlier has been successfully written into Elasticsearch. - ```json - {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} - ``` + + * refresh the index + + ```bash + + $ curl -s http://localhost:9200/my_index/_refresh + + ``` + + + * search documents + + ```bash + + $ curl -s http://localhost:9200/my_index/_search + + ``` + + You can see the record that published earlier has been successfully written into Elasticsearch. + + ```json + + {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-file-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-file-source.md index f1f73ade2fe02..4e34dfd9a1f02 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-file-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-file-source.md @@ -1,7 +1,7 @@ --- id: io-file-source title: File source connector -sidebar_label: File source connector +sidebar_label: "File source connector" original_id: io-file-source --- @@ -38,40 +38,44 @@ Before using the File source connector, you need to create a configuration file * JSON - ```json - { - "inputDirectory": "/Users/david", - "recurse": true, - "keepFile": true, - "fileFilter": "[^\\.].*", - "pathFilter": "*", - "minimumFileAge": 0, - "maximumFileAge": 9999999999, - "minimumSize": 1, - "maximumSize": 5000000, - "ignoreHiddenFiles": true, - "pollingInterval": 5000, - "numWorkers": 1 - } - ``` + ```json + + { + "inputDirectory": "/Users/david", + "recurse": true, + "keepFile": true, + "fileFilter": "[^\\.].*", + "pathFilter": "*", + "minimumFileAge": 0, + "maximumFileAge": 9999999999, + "minimumSize": 1, + "maximumSize": 5000000, + "ignoreHiddenFiles": true, + "pollingInterval": 5000, + "numWorkers": 1 + } + + ``` * YAML - ```yaml - configs: - inputDirectory: "/Users/david" - recurse: true - keepFile: true - fileFilter: "[^\\.].*" - pathFilter: "*" - minimumFileAge: 0 - maximumFileAge: 9999999999 - minimumSize: 1 - maximumSize: 5000000 - ignoreHiddenFiles: true - pollingInterval: 5000 - numWorkers: 1 - ``` + ```yaml + + configs: + inputDirectory: "/Users/david" + recurse: true + keepFile: true + fileFilter: "[^\\.].*" + pathFilter: "*" + minimumFileAge: 0 + maximumFileAge: 9999999999 + minimumSize: 1 + maximumSize: 5000000 + ignoreHiddenFiles: true + pollingInterval: 5000 + numWorkers: 1 + + ``` ## Usage @@ -79,64 +83,81 @@ Here is an example of using the File source connecter. 1. Pull a Pulsar image. - ```bash - $ docker pull apachepulsar/pulsar:{version} - ``` + ```bash + + $ docker pull apachepulsar/pulsar:{version} + + ``` 2. Start Pulsar standalone. + + ```bash - ```bash - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 3. Create a configuration file _file-connector.yaml_. - ```yaml - configs: - inputDirectory: "/opt" - ``` + ```yaml + + configs: + inputDirectory: "/opt" + + ``` 4. Copy the configuration file _file-connector.yaml_ to the container. - ```bash - $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ - ``` + ```bash + + $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ + + ``` 5. Download the File source connector. - ```bash - $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar - ``` + ```bash + + $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar + + ``` 6. Start the File source connector. - ```bash - $ docker exec -it pulsar-standalone /bin/bash + ```bash + + $ docker exec -it pulsar-standalone /bin/bash - $ ./bin/pulsar-admin sources localrun \ - --archive /pulsar/pulsar-io-file-{version}.nar \ - --name file-test \ - --destination-topic-name pulsar-file-test \ - --source-config-file /pulsar/file-connector.yaml - ``` + $ ./bin/pulsar-admin sources localrun \ + --archive /pulsar/pulsar-io-file-{version}.nar \ + --name file-test \ + --destination-topic-name pulsar-file-test \ + --source-config-file /pulsar/file-connector.yaml + + ``` 7. Start a consumer. - ```bash - ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test - ``` + ```bash + + ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test + + ``` 8. Write the message to the file _test.txt_. + + ```bash + + echo "hello world!" > /opt/test.txt - ```bash - echo "hello world!" > /opt/test.txt - ``` + ``` - The following information appears on the consumer terminal window. + The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello world! - ``` + ```bash + + ----- got message ----- + hello world! + + ``` - diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-flume-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-flume-sink.md index 086dfd4df1b66..ded05d80726f1 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-flume-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-flume-sink.md @@ -1,7 +1,7 @@ --- id: io-flume-sink title: Flume sink connector -sidebar_label: Flume sink connector +sidebar_label: "Flume sink connector" original_id: io-flume-sink --- @@ -29,33 +29,32 @@ The configuration of the Flume sink connector has the following properties. Before using the Flume sink connector, you need to create a configuration file through one of the following methods. -:::note - -For more information about the `sink.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/sink.conf). - -::: - +> For more information about the `sink.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/sink.conf). * JSON - ```json - { - "name": "a1", - "confFile": "sink.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - ``` + ```json + + { + "name": "a1", + "confFile": "sink.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - configs: - name: a1 - confFile: sink.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - ``` + ```yaml + + configs: + name: a1 + confFile: sink.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-flume-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-flume-source.md index 888d0bac9c666..42f35e7ea499d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-flume-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-flume-source.md @@ -1,7 +1,7 @@ --- id: io-flume-source title: Flume source connector -sidebar_label: Flume source connector +sidebar_label: "Flume source connector" original_id: io-flume-source --- @@ -29,33 +29,32 @@ The configuration of the Flume source connector has the following properties. Before using the Flume source connector, you need to create a configuration file through one of the following methods. -:::note - -For more information about the `source.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/source.conf). - -::: - +> For more information about the `source.conf` in the example below, see [here](https://github.com/apache/pulsar/blob/master/pulsar-io/flume/src/main/resources/flume/source.conf). * JSON - ```json - { - "name": "a1", - "confFile": "source.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - ``` + ```json + + { + "name": "a1", + "confFile": "source.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - configs: - name: a1 - confFile: source.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - ``` + ```yaml + + configs: + name: a1 + confFile: source.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-hbase-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-hbase-sink.md index 471ca0eeb509f..8d39551a05ddb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-hbase-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-hbase-sink.md @@ -1,7 +1,7 @@ --- id: io-hbase-sink title: HBase sink connector -sidebar_label: HBase sink connector +sidebar_label: "HBase sink connector" original_id: io-hbase-sink --- @@ -37,32 +37,34 @@ Before using the HBase sink connector, you need to create a configuration file t * JSON - ```json - { - "hbaseConfigResources": "hbase-site.xml", - "zookeeperQuorum": "localhost", - "zookeeperClientPort": "2181", - "zookeeperZnodeParent": "/hbase", - "tableName": "pulsar_hbase", - "rowKeyName": "rowKey", - "familyName": "info", - "qualifierNames": [ 'name', 'address', 'age'] - } - ``` - + ```json + + { + "hbaseConfigResources": "hbase-site.xml", + "zookeeperQuorum": "localhost", + "zookeeperClientPort": "2181", + "zookeeperZnodeParent": "/hbase", + "tableName": "pulsar_hbase", + "rowKeyName": "rowKey", + "familyName": "info", + "qualifierNames": [ 'name', 'address', 'age'] + } + + ``` * YAML - ```yaml - configs: - hbaseConfigResources: "hbase-site.xml" - zookeeperQuorum: "localhost" - zookeeperClientPort: "2181" - zookeeperZnodeParent: "/hbase" - tableName: "pulsar_hbase" - rowKeyName: "rowKey" - familyName: "info" - qualifierNames: [ 'name', 'address', 'age'] - ``` + ```yaml + + configs: + hbaseConfigResources: "hbase-site.xml" + zookeeperQuorum: "localhost" + zookeeperClientPort: "2181" + zookeeperZnodeParent: "/hbase" + tableName: "pulsar_hbase" + rowKeyName: "rowKey" + familyName: "info" + qualifierNames: [ 'name', 'address', 'age'] + + ``` - diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-hdfs2-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-hdfs2-sink.md index 1804170aa8390..10369997588d6 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-hdfs2-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-hdfs2-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs2-sink title: HDFS2 sink connector -sidebar_label: HDFS2 sink connector +sidebar_label: "HDFS2 sink connector" original_id: io-hdfs2-sink --- @@ -20,18 +20,18 @@ The configuration of the HDFS2 sink connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | +| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | | `directory` | String | true | None|The HDFS directory where files read from or written to. | -| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | -| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | +| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | +| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | | `kerberosUserPrincipal` |String| false| None|The principal account of Kerberos user used for authentication. | | `keytab` | String|false|None| The full pathname of the Kerberos keytab file used for authentication. | -| `filenamePrefix` |String| true, if `compression` is set to `None`. | None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | -| `fileExtension` | String| true | None | The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | -| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | +| `filenamePrefix` |String| true, if `compression` is set to `None`. | None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | +| `fileExtension` | String| true | None | The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | +| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | | `syncInterval` | long| false |0| The interval between calls to flush data to HDFS disk in milliseconds. | -| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. -| `subdirectoryPattern` | String | false | None | A subdirectory associated with the created time of the sink.
    The pattern is the formatted pattern of `directory`'s subdirectory.

    See [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) for pattern's syntax. | +| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. +| `subdirectoryPattern` | String | false | None | A subdirectory associated with the created time of the sink.
    The pattern is the formatted pattern of `directory`'s subdirectory.

    See [DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) for pattern's syntax. | ### Example @@ -39,25 +39,30 @@ Before using the HDFS2 sink connector, you need to create a configuration file t * JSON - ```json - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "fileExtension": ".log", - "compression": "SNAPPY", - "subdirectoryPattern": "yyyy-MM-dd" - } - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "fileExtension": ".log", + "compression": "SNAPPY", + "subdirectoryPattern": "yyyy-MM-dd" + } + + ``` * YAML - ```yaml - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - fileExtension: ".log" - compression: "SNAPPY" - subdirectoryPattern: "yyyy-MM-dd" - ``` + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + fileExtension: ".log" + compression: "SNAPPY" + subdirectoryPattern: "yyyy-MM-dd" + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-hdfs3-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-hdfs3-sink.md index ff7457bf2010b..6bd77ee58d277 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-hdfs3-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-hdfs3-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs3-sink title: HDFS3 sink connector -sidebar_label: HDFS3 sink connector +sidebar_label: "HDFS3 sink connector" original_id: io-hdfs3-sink --- @@ -20,17 +20,17 @@ The configuration of the HDFS3 sink connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | +| `hdfsConfigResources` | String|true| None | A file or a comma-separated list containing the Hadoop file system configuration.

    **Example**
    'core-site.xml'
    'hdfs-site.xml' | | `directory` | String | true | None|The HDFS directory where files read from or written to. | -| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | -| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | +| `encoding` | String |false |None |The character encoding for the files.

    **Example**
    UTF-8
    ASCII | +| `compression` | Compression |false |None |The compression code used to compress or de-compress the files on HDFS.

    Below are the available options:
  • BZIP2
  • DEFLATE
  • GZIP
  • LZ4
  • SNAPPY
  • | | `kerberosUserPrincipal` |String| false| None|The principal account of Kerberos user used for authentication. | | `keytab` | String|false|None| The full pathname of the Kerberos keytab file used for authentication. | -| `filenamePrefix` |String| false |None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | -| `fileExtension` | String| false | None| The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | -| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | +| `filenamePrefix` |String| false |None |The prefix of the files created inside the HDFS directory.

    **Example**
    The value of topicA result in files named topicA-. | +| `fileExtension` | String| false | None| The extension added to the files written to HDFS.

    **Example**
    '.txt'
    '.seq' | +| `separator` | char|false |None |The character used to separate records in a text file.

    If no value is provided, the contents from all records are concatenated together in one continuous byte array. | | `syncInterval` | long| false |0| The interval between calls to flush data to HDFS disk in milliseconds. | -| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. +| `maxPendingRecords` |int| false|Integer.MAX_VALUE | The maximum number of records that hold in memory before acking.

    Setting this property to 1 makes every record send to disk before the record is acked.

    Setting this property to a higher value allows buffering records before flushing them to disk. ### Example @@ -38,21 +38,26 @@ Before using the HDFS3 sink connector, you need to create a configuration file t * JSON - ```json - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "compression": "SNAPPY" - } - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "compression": "SNAPPY" + } + + ``` * YAML - ```yaml - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - compression: "SNAPPY" - ``` + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + compression: "SNAPPY" + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-influxdb-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-influxdb-sink.md index 2f2f8811076c4..d2dbeedd47895 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-influxdb-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-influxdb-sink.md @@ -1,7 +1,7 @@ --- id: io-influxdb-sink title: InfluxDB sink connector -sidebar_label: InfluxDB sink connector +sidebar_label: "InfluxDB sink connector" original_id: io-influxdb-sink --- @@ -50,63 +50,67 @@ The configuration of the InfluxDB sink connector has the following properties. Before using the InfluxDB sink connector, you need to create a configuration file through one of the following methods. #### InfluxDBv2 * JSON - ```json - { - "influxdbUrl": "http://localhost:9999", - "organization": "example-org", - "bucket": "example-bucket", - "token": "xxxx", - "precision": "ns", - "logLevel": "NONE", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - ``` + + ```json + + { + "influxdbUrl": "http://localhost:9999", + "organization": "example-org", + "bucket": "example-bucket", + "token": "xxxx", + "precision": "ns", + "logLevel": "NONE", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` + * YAML - ```yaml - { - influxdbUrl: "http://localhost:9999" - organization: "example-org" - bucket: "example-bucket" - token: "xxxx" - precision: "ns" - logLevel: "NONE" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } - ``` + ```yaml + configs: + influxdbUrl: "http://localhost:9999" + organization: "example-org" + bucket: "example-bucket" + token: "xxxx" + precision: "ns" + logLevel: "NONE" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + ``` #### InfluxDBv1 * JSON - ```json - { - "influxdbUrl": "http://localhost:8086", - "database": "test_db", - "consistencyLevel": "ONE", - "logLevel": "NONE", - "retentionPolicy": "autogen", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - ``` + ```json + + { + "influxdbUrl": "http://localhost:8086", + "database": "test_db", + "consistencyLevel": "ONE", + "logLevel": "NONE", + "retentionPolicy": "autogen", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` * YAML - ```yaml - { - influxdbUrl: "http://localhost:8086" - database: "test_db" - consistencyLevel: "ONE" - logLevel: "NONE" - retentionPolicy: "autogen" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } - ``` + ```yaml + configs: + influxdbUrl: "http://localhost:8086" + database: "test_db" + consistencyLevel: "ONE" + logLevel: "NONE" + retentionPolicy: "autogen" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-jdbc-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-jdbc-sink.md index 2d899ee19a89c..e992e724e534d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-jdbc-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-jdbc-sink.md @@ -1,7 +1,7 @@ --- id: io-jdbc-sink title: JDBC sink connector -sidebar_label: JDBC sink connector +sidebar_label: "JDBC sink connector" original_id: io-jdbc-sink --- @@ -12,12 +12,7 @@ import TabItem from '@theme/TabItem'; The JDBC sink connectors allow pulling messages from Pulsar topics and persists the messages to ClickHouse, MariaDB, PostgreSQL, and SQLite. -:::note - -Currently, INSERT, DELETE and UPDATE operations are supported. - -::: - +> Currently, INSERT, DELETE and UPDATE operations are supported. ## Configuration @@ -40,57 +35,65 @@ The configuration of all JDBC sink connectors has the following properties. * JSON - ```json - { - "userName": "clickhouse", - "password": "password", - "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", - "tableName": "pulsar_clickhouse_jdbc_sink" - } - ``` + ```json + + { + "userName": "clickhouse", + "password": "password", + "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", + "tableName": "pulsar_clickhouse_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-clickhouse-sink" - topicName: "persistent://public/default/jdbc-clickhouse-topic" - sinkType: "jdbc-clickhouse" - configs: - userName: "clickhouse" - password: "password" - jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" - tableName: "pulsar_clickhouse_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-clickhouse-sink" + topicName: "persistent://public/default/jdbc-clickhouse-topic" + sinkType: "jdbc-clickhouse" + configs: + userName: "clickhouse" + password: "password" + jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" + tableName: "pulsar_clickhouse_jdbc_sink" + + ``` ### Example for MariaDB * JSON - ```json - { - "userName": "mariadb", - "password": "password", - "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", - "tableName": "pulsar_mariadb_jdbc_sink" - } - ``` + ```json + + { + "userName": "mariadb", + "password": "password", + "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", + "tableName": "pulsar_mariadb_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-mariadb-sink" - topicName: "persistent://public/default/jdbc-mariadb-topic" - sinkType: "jdbc-mariadb" - configs: - userName: "mariadb" - password: "password" - jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" - tableName: "pulsar_mariadb_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-mariadb-sink" + topicName: "persistent://public/default/jdbc-mariadb-topic" + sinkType: "jdbc-mariadb" + configs: + userName: "mariadb" + password: "password" + jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" + tableName: "pulsar_mariadb_jdbc_sink" + + ``` ### Example for PostgreSQL @@ -98,29 +101,33 @@ Before using the JDBC PostgreSQL sink connector, you need to create a configurat * JSON - ```json - { - "userName": "postgres", - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "tableName": "pulsar_postgres_jdbc_sink" - } - ``` + ```json + + { + "userName": "postgres", + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "tableName": "pulsar_postgres_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-postgres-sink" - topicName: "persistent://public/default/jdbc-postgres-topic" - sinkType: "jdbc-postgres" - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-postgres-sink" + topicName: "persistent://public/default/jdbc-postgres-topic" + sinkType: "jdbc-postgres" + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" + tableName: "pulsar_postgres_jdbc_sink" + + ``` For more information on **how to use this JDBC sink connector**, see [connect Pulsar to PostgreSQL](io-quickstart.md#connect-pulsar-to-postgresql). @@ -128,22 +135,27 @@ For more information on **how to use this JDBC sink connector**, see [connect Pu * JSON - ```json - { - "jdbcUrl": "jdbc:sqlite:db.sqlite", - "tableName": "pulsar_sqlite_jdbc_sink" - } - ``` + ```json + + { + "jdbcUrl": "jdbc:sqlite:db.sqlite", + "tableName": "pulsar_sqlite_jdbc_sink" + } + + ``` * YAML - ```yaml - tenant: "public" - namespace: "default" - name: "jdbc-sqlite-sink" - topicName: "persistent://public/default/jdbc-sqlite-topic" - sinkType: "jdbc-sqlite" - configs: - jdbcUrl: "jdbc:sqlite:db.sqlite" - tableName: "pulsar_sqlite_jdbc_sink" - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-sqlite-sink" + topicName: "persistent://public/default/jdbc-sqlite-topic" + sinkType: "jdbc-sqlite" + configs: + jdbcUrl: "jdbc:sqlite:db.sqlite" + tableName: "pulsar_sqlite_jdbc_sink" + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-kafka-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-kafka-sink.md index bf2d77fd13b7b..743f78514e068 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-kafka-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-kafka-sink.md @@ -1,7 +1,7 @@ --- id: io-kafka-sink title: Kafka sink connector -sidebar_label: Kafka sink connector +sidebar_label: "Kafka sink connector" original_id: io-kafka-sink --- @@ -23,13 +23,13 @@ The configuration of the Kafka sink connector has the following parameters. | Name | Type| Required | Default | Description |------|----------|---------|-------------|-------------| | `bootstrapServers` |String| true | " " (empty string) | A comma-separated list of host and port pairs for establishing the initial connection to the Kafka cluster. | -|`acks`|String|true|" " (empty string) |The number of acknowledgments that the producer requires the leader to receive before a request completes.
    This controls the durability of the sent records. +|`acks`|String|true|" " (empty string) |The number of acknowledgments that the producer requires the leader to receive before a request completes.
    This controls the durability of the sent records. |`batchsize`|long|false|16384L|The batch size that a Kafka producer attempts to batch records together before sending them to brokers. |`maxRequestSize`|long|false|1048576L|The maximum size of a Kafka request in bytes. |`topic`|String|true|" " (empty string) |The Kafka topic which receives messages from Pulsar. | `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringSerializer | The serializer class for Kafka producers to serialize keys. -| `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArraySerializer | The serializer class for Kafka producers to serialize values.

    The serializer is set by a specific implementation of [`KafkaAbstractSink`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java). -|`producerConfigProperties`|Map|false|" " (empty string)|The producer configuration properties to be passed to producers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. +| `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArraySerializer | The serializer class for Kafka producers to serialize values.

    The serializer is set by a specific implementation of [`KafkaAbstractSink`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java). +|`producerConfigProperties`|Map|false|" " (empty string)|The producer configuration properties to be passed to producers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. ### Example @@ -38,36 +38,39 @@ Before using the Kafka sink connector, you need to create a configuration file t * JSON - ```json - { - "bootstrapServers": "localhost:6667", - "topic": "test", - "acks": "1", - "batchSize": "16384", - "maxRequestSize": "1048576", - "producerConfigProperties": - { - "client.id": "test-pulsar-producer", - "security.protocol": "SASL_PLAINTEXT", - "sasl.mechanism": "GSSAPI", - "sasl.kerberos.service.name": "kafka", - "acks": "all" - } - } + ```json + + { + "bootstrapServers": "localhost:6667", + "topic": "test", + "acks": "1", + "batchSize": "16384", + "maxRequestSize": "1048576", + "producerConfigProperties": + { + "client.id": "test-pulsar-producer", + "security.protocol": "SASL_PLAINTEXT", + "sasl.mechanism": "GSSAPI", + "sasl.kerberos.service.name": "kafka", + "acks": "all" + } + } * YAML - ```yaml - configs: - bootstrapServers: "localhost:6667" - topic: "test" - acks: "1" - batchSize: "16384" - maxRequestSize: "1048576" - producerConfigProperties: - client.id: "test-pulsar-producer" - security.protocol: "SASL_PLAINTEXT" - sasl.mechanism: "GSSAPI" - sasl.kerberos.service.name: "kafka" - acks: "all" - ``` + ``` + +yaml + configs: + bootstrapServers: "localhost:6667" + topic: "test" + acks: "1" + batchSize: "16384" + maxRequestSize: "1048576" + producerConfigProperties: + client.id: "test-pulsar-producer" + security.protocol: "SASL_PLAINTEXT" + sasl.mechanism: "GSSAPI" + sasl.kerberos.service.name: "kafka" + acks: "all" + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-kafka-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-kafka-source.md index 95c62ffff6f87..e353aea21c4b7 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-kafka-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-kafka-source.md @@ -1,7 +1,7 @@ --- id: io-kafka-source title: Kafka source connector -sidebar_label: Kafka source connector +sidebar_label: "Kafka source connector" original_id: io-kafka-source --- @@ -25,13 +25,13 @@ The configuration of the Kafka source connector has the following properties. | `bootstrapServers` |String| true | " " (empty string) | A comma-separated list of host and port pairs for establishing the initial connection to the Kafka cluster. | | `groupId` |String| true | " " (empty string) | A unique string that identifies the group of consumer processes to which this consumer belongs. | | `fetchMinBytes` | long|false | 1 | The minimum byte expected for each fetch response. | -| `autoCommitEnabled` | boolean |false | true | If set to true, the consumer's offset is periodically committed in the background.

    This committed offset is used when the process fails as the position from which a new consumer begins. | +| `autoCommitEnabled` | boolean |false | true | If set to true, the consumer's offset is periodically committed in the background.

    This committed offset is used when the process fails as the position from which a new consumer begins. | | `autoCommitIntervalMs` | long|false | 5000 | The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if `autoCommitEnabled` is set to true. | -| `heartbeatIntervalMs` | long| false | 3000 | The interval between heartbeats to the consumer when using Kafka's group management facilities.

    **Note: `heartbeatIntervalMs` must be smaller than `sessionTimeoutMs`**.| +| `heartbeatIntervalMs` | long| false | 3000 | The interval between heartbeats to the consumer when using Kafka's group management facilities.

    **Note: `heartbeatIntervalMs` must be smaller than `sessionTimeoutMs`**.| | `sessionTimeoutMs` | long|false | 30000 | The timeout used to detect consumer failures when using Kafka's group management facility. | | `topic` | String|true | " " (empty string)| The Kafka topic which sends messages to Pulsar. | -| `consumerConfigProperties` | Map| false | " " (empty string) | The consumer configuration properties to be passed to consumers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. | -| `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringDeserializer | The deserializer class for Kafka consumers to deserialize keys.
    The deserializer is set by a specific implementation of [`KafkaAbstractSource`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java). +| `consumerConfigProperties` | Map| false | " " (empty string) | The consumer configuration properties to be passed to consumers.

    **Note: other properties specified in the connector configuration file take precedence over this configuration**. | +| `keyDeserializationClass` | String|false | org.apache.kafka.common.serialization.StringDeserializer | The deserializer class for Kafka consumers to deserialize keys.
    The deserializer is set by a specific implementation of [`KafkaAbstractSource`](https://github.com/apache/pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java). | `valueDeserializationClass` | String|false | org.apache.kafka.common.serialization.ByteArrayDeserializer | The deserializer class for Kafka consumers to deserialize values. @@ -41,26 +41,30 @@ Before using the Kafka source connector, you need to create a configuration file * JSON - ```json - { - "bootstrapServers": "pulsar-kafka:9092", - "groupId": "test-pulsar-io", - "topic": "my-topic", - "sessionTimeoutMs": "10000", - "autoCommitEnabled": false - } - ``` + ```json + + { + "bootstrapServers": "pulsar-kafka:9092", + "groupId": "test-pulsar-io", + "topic": "my-topic", + "sessionTimeoutMs": "10000", + "autoCommitEnabled": false + } + + ``` * YAML - ```yaml - configs: - bootstrapServers: "pulsar-kafka:9092" - groupId: "test-pulsar-io" - topic: "my-topic" - sessionTimeoutMs: "10000" - autoCommitEnabled: false - ``` + ```yaml + + configs: + bootstrapServers: "pulsar-kafka:9092" + groupId: "test-pulsar-io" + topic: "my-topic" + sessionTimeoutMs: "10000" + autoCommitEnabled: false + + ``` ## Usage @@ -68,108 +72,130 @@ Here is an example of using the Kafka source connecter with the configuration fi 1. Download a Kafka client and a Kafka connector. - ```bash - $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar + ```bash + + $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar - ``` + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar + + ``` 2. Create a network. - + ```bash + $ docker network create kafka-pulsar + ``` 3. Pull a ZooKeeper image and start ZooKeeper. - + ```bash + $ docker pull wurstmeister/zookeeper $ docker run -d -it -p 2181:2181 --name pulsar-kafka-zookeeper --network kafka-pulsar wurstmeister/zookeeper + ``` 4. Pull a Kafka image and start Kafka. - + ```bash + $ docker pull wurstmeister/kafka:2.11-1.0.2 $ docker run -d -it --network kafka-pulsar -p 6667:6667 -p 9092:9092 -e KAFKA_ADVERTISED_HOST_NAME=pulsar-kafka -e KAFKA_ZOOKEEPER_CONNECT=pulsar-kafka-zookeeper:2181 --name pulsar-kafka wurstmeister/kafka:2.11-1.0.2 + ``` 5. Pull a Pulsar image and start Pulsar standalone. - + ```bash + $ docker pull apachepulsar/pulsar:2.4.0 $ docker run -d -it --network kafka-pulsar -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-kafka-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + ``` 6. Create a producer file _kafka-producer.py_. - + ```python + from kafka import KafkaProducer producer = KafkaProducer(bootstrap_servers='pulsar-kafka:9092') future = producer.send('my-topic', b'hello world') future.get() + ``` 7. Create a consumer file _pulsar-client.py_. - ```python - import pulsar + ```python + + import pulsar - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', subscription_name='my-aa') + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', subscription_name='my-aa') - while True: - msg = consumer.receive() - print msg - print dir(msg) - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + while True: + msg = consumer.receive() + print msg + print dir(msg) + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) - client.close() - ``` + client.close() + + ``` 8. Copy the following files to Pulsar. + + ```bash + + $ docker cp pulsar-io-kafka-2.4.0.nar pulsar-kafka-standalone:/pulsar + $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf + $ docker cp kafka-clients-0.10.2.1.jar pulsar-kafka-standalone:/pulsar/lib + $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ + $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ - ```bash - $ docker cp pulsar-io-kafka-2.4.0.nar pulsar-kafka-standalone:/pulsar - $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf - $ docker cp kafka-clients-0.10.2.1.jar pulsar-kafka-standalone:/pulsar/lib - $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ - $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ - ``` + ``` 9. Open a new terminal window and start the Kafka source connector in local run mode. - ```bash - $ docker exec -it pulsar-kafka-standalone /bin/bash - - $ ./bin/pulsar-admin source localrun \ - --archive ./pulsar-io-kafka-2.4.0.nar \ - --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ - --tenant public \ - --namespace default \ - --name kafka \ - --destination-topic-name my-topic \ - --source-config-file ./conf/kafkaSourceConfig.yaml \ - --parallelism 1 - ``` + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash + + $ ./bin/pulsar-admin source localrun \ + --archive ./pulsar-io-kafka-2.4.0.nar \ + --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ + --tenant public \ + --namespace default \ + --name kafka \ + --destination-topic-name my-topic \ + --source-config-file ./conf/kafkaSourceConfig.yaml \ + --parallelism 1 + + ``` 10. Open a new terminal window and run the consumer. - ```bash - $ docker exec -it pulsar-kafka-standalone /bin/bash + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash - $ pip install kafka-python + $ pip install kafka-python - $ python3 kafka-producer.py - ``` + $ python3 kafka-producer.py + + ``` - The following information appears on the consumer terminal window. + The following information appears on the consumer terminal window. - ```bash - Received message: 'hello world' - ``` + ```bash + + Received message: 'hello world' + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-sink.md index 977492d872864..483c861db284d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-sink.md @@ -1,7 +1,7 @@ --- id: io-kinesis-sink title: Kinesis sink connector -sidebar_label: Kinesis sink connector +sidebar_label: "Kinesis sink connector" original_id: io-kinesis-sink --- @@ -19,12 +19,12 @@ The configuration of the Kinesis sink connector has the following property. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`messageFormat`|MessageFormat|true|ONLY_RAW_PAYLOAD|Message format in which Kinesis sink converts Pulsar messages and publishes to Kinesis streams.

    Below are the available options:

  • `ONLY_RAW_PAYLOAD`: Kinesis sink directly publishes Pulsar message payload as a message into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_JSON`: Kinesis sink creates a JSON payload with Pulsar message payload, properties and encryptionCtx, and publishes JSON payload into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_FB`: Kinesis sink creates a flatbuffer serialized payload with Pulsar message payload, properties and encryptionCtx, and publishes flatbuffer payload into the configured Kinesis stream.
  • +`messageFormat`|MessageFormat|true|ONLY_RAW_PAYLOAD|Message format in which Kinesis sink converts Pulsar messages and publishes to Kinesis streams.

    Below are the available options:

  • `ONLY_RAW_PAYLOAD`: Kinesis sink directly publishes Pulsar message payload as a message into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_JSON`: Kinesis sink creates a JSON payload with Pulsar message payload, properties and encryptionCtx, and publishes JSON payload into the configured Kinesis stream.

  • `FULL_MESSAGE_IN_FB`: Kinesis sink creates a flatbuffer serialized payload with Pulsar message payload, properties and encryptionCtx, and publishes flatbuffer payload into the configured Kinesis stream.
  • `retainOrdering`|boolean|false|false|Whether Pulsar connectors to retain ordering when moving messages from Pulsar to Kinesis or not. `awsEndpoint`|String|false|" " (empty string)|The Kinesis end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsKinesisStreamName`|String|true|" " (empty string)|The Kinesis stream name. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    It is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If it is empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`. +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    It is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If it is empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`. `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Built-in plugins @@ -33,19 +33,21 @@ The following are built-in `AwsCredentialProviderPlugin` plugins: * `org.apache.pulsar.io.aws.AwsDefaultProviderChainPlugin` - This plugin takes no configuration, it uses the default AWS provider chain. - - For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). + This plugin takes no configuration, it uses the default AWS provider chain. + + For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). * `org.apache.pulsar.io.aws.STSAssumeRoleProviderPlugin` - This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. + This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. - This configuration takes the form of a small json document like: + This configuration takes the form of a small json document like: - ```json - {"roleArn": "arn...", "roleSessionName": "name"} - ``` + ```json + + {"roleArn": "arn...", "roleSessionName": "name"} + + ``` ### Example @@ -53,25 +55,30 @@ Before using the Kinesis sink connector, you need to create a configuration file * JSON - ```json - { - "awsEndpoint": "some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "messageFormat": "ONLY_RAW_PAYLOAD", - "retainOrdering": "true" - } - ``` + ```json + + { + "awsEndpoint": "some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "messageFormat": "ONLY_RAW_PAYLOAD", + "retainOrdering": "true" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - messageFormat: "ONLY_RAW_PAYLOAD" - retainOrdering: "true" - ``` + ```yaml + + configs: + awsEndpoint: "some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + messageFormat: "ONLY_RAW_PAYLOAD" + retainOrdering: "true" + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-source.md index 77850cecf65e0..3e593a31161ad 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-kinesis-source.md @@ -1,7 +1,7 @@ --- id: io-kinesis-source title: Kinesis source connector -sidebar_label: Kinesis source connector +sidebar_label: "Kinesis source connector" original_id: io-kinesis-source --- @@ -13,12 +13,7 @@ The Kinesis source connector pulls data from Amazon Kinesis and persists data in This connector uses the [Kinesis Consumer Library](https://github.com/awslabs/amazon-kinesis-client) (KCL) to do the actual consuming of messages. The KCL uses DynamoDB to track state for consumers. -:::note - -: currently, the Kinesis source connector only supports raw messages. If you use KMS encrypted messages, the encrypted messages are sent to downstream. This connector will support decrypting messages in the future release. - -::: - +> Note: currently, the Kinesis source connector only supports raw messages. If you use KMS encrypted messages, the encrypted messages are sent to downstream. This connector will support decrypting messages in the future release. ## Configuration @@ -29,20 +24,20 @@ The configuration of the Kinesis source connector has the following properties. | Name | Type|Required | Default | Description |------|----------|----------|---------|-------------| -`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • +`initialPositionInStream`|InitialPositionInStream|false|LATEST|The position where the connector starts from.

    Below are the available options:

  • `AT_TIMESTAMP`: start from the record at or after the specified timestamp.

  • `LATEST`: start after the most recent data record.

  • `TRIM_HORIZON`: start from the oldest available data record.
  • `startAtTime`|Date|false|" " (empty string)|If set to `AT_TIMESTAMP`, it specifies the point in time to start consumption. -`applicationName`|String|false|Pulsar IO connector|The name of the Amazon Kinesis application.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. +`applicationName`|String|false|Pulsar IO connector|The name of the Amazon Kinesis application.

    By default, the application name is included in the user agent string used to make AWS requests. This can assist with troubleshooting, for example, distinguish requests made by separate connector instances. `checkpointInterval`|long|false|60000|The frequency of the Kinesis stream checkpoint in milliseconds. `backoffTime`|long|false|3000|The amount of time to delay between requests when the connector encounters a throttling exception from AWS Kinesis in milliseconds. `numRetries`|int|false|3|The number of re-attempts when the connector encounters an exception while trying to set a checkpoint. -`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. +`receiveQueueSize`|int|false|1000|The maximum number of AWS records that can be buffered inside the connector.

    Once the `receiveQueueSize` is reached, the connector does not consume any messages from Kinesis until some messages in the queue are successfully consumed. `dynamoEndpoint`|String|false|" " (empty string)|The Dynamo end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `cloudwatchEndpoint`|String|false|" " (empty string)|The Cloudwatch end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). `useEnhancedFanOut`|boolean|false|true|If set to true, it uses Kinesis enhanced fan-out.

    If set to false, it uses polling. `awsEndpoint`|String|false|" " (empty string)|The Kinesis end-point URL, which can be found at [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). -`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 +`awsRegion`|String|false|" " (empty string)|The AWS region.

    **Example**
    us-west-1, us-west-2 `awsKinesisStreamName`|String|true|" " (empty string)|The Kinesis stream name. -`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • +`awsCredentialPluginName`|String|false|" " (empty string)|The fully-qualified class name of implementation of {@inject: github:AwsCredentialProviderPlugin:/pulsar-io/aws/src/main/java/org/apache/pulsar/io/aws/AwsCredentialProviderPlugin.java}.

    `awsCredentialProviderPlugin` has the following built-in plugs:

  • `org.apache.pulsar.io.kinesis.AwsDefaultProviderChainPlugin`:
    this plugin uses the default AWS provider chain.
    For more information, see [using the default credential provider chain](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default).

  • `org.apache.pulsar.io.kinesis.STSAssumeRoleProviderPlugin`:
    this plugin takes a configuration via the `awsCredentialPluginParam` that describes a role to assume when running the KCL.
    **JSON configuration example**
    `{"roleArn": "arn...", "roleSessionName": "name"}`

    `awsCredentialPluginName` is a factory class which creates an AWSCredentialsProvider that is used by Kinesis sink.

    If `awsCredentialPluginName` set to empty, the Kinesis sink creates a default AWSCredentialsProvider which accepts json-map of credentials in `awsCredentialPluginParam`.
  • `awsCredentialPluginParam`|String |false|" " (empty string)|The JSON parameter to initialize `awsCredentialsProviderPlugin`. ### Example @@ -51,36 +46,40 @@ Before using the Kinesis source connector, you need to create a configuration fi * JSON - ```json - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-mongo-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-mongo-sink.md index f76f5ea50cb83..b370464c18f4b 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-mongo-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-mongo-sink.md @@ -1,7 +1,7 @@ --- id: io-mongo-sink title: MongoDB sink connector -sidebar_label: MongoDB sink connector +sidebar_label: "MongoDB sink connector" original_id: io-mongo-sink --- @@ -32,25 +32,30 @@ The configuration of the MongoDB sink connector has the following properties. Before using the Mongo sink connector, you need to create a configuration file through one of the following methods. * JSON + + ```json + + { + "mongoUri": "mongodb://localhost:27017", + "database": "pulsar", + "collection": "messages", + "batchSize": "2", + "batchTimeMs": "500" + } - ```json - { - "mongoUri": "mongodb://localhost:27017", - "database": "pulsar", - "collection": "messages", - "batchSize": "2", - "batchTimeMs": "500" - } - ``` + ``` * YAML + + ```yaml - ```yaml - { - mongoUri: "mongodb://localhost:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - } - ``` + { + mongoUri: "mongodb://localhost:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + } + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-netty-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-netty-source.md index 1e2ba61cfc4f4..d41265fa4fa28 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-netty-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-netty-source.md @@ -1,7 +1,7 @@ --- id: io-netty-source title: Netty source connector -sidebar_label: Netty source connector +sidebar_label: "Netty source connector" original_id: io-netty-source --- @@ -34,25 +34,28 @@ Before using the Netty source connector, you need to create a configuration file * JSON - ```json - { - "type": "tcp", - "host": "127.0.0.1", - "port": "10911", - "numberOfThreads": "1" - } - ``` + ```json + + { + "type": "tcp", + "host": "127.0.0.1", + "port": "10911", + "numberOfThreads": "1" + } + + ``` * YAML - ```yaml - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` - + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` ## Usage @@ -62,148 +65,181 @@ The following examples show how to use the Netty source connector with TCP and H 1. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:{version} + ```bash + + $ docker pull apachepulsar/pulsar:{version} - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 2. Create a configuration file _netty-source-config.yaml_. + + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 - ```yaml - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` + ``` 3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. - ```bash - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ - ``` + ```bash + + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar - ``` - -5. Start the Netty source connector. + ```bash + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + + ``` + +5. Start the Netty source connector. + ```bash + $ ./bin/pulsar-admin sources localrun \ - --archive pulsar-io-{{pulsar:version}}.nar \ + --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ --namespace default \ --name netty \ --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 + ``` 6. Consume data. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ apt-get update - - $ apt-get -y install telnet - - $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 - Trying 127.0.0.1... - Connected to 127.0.0.1. - Escape character is '^]'. - hello - world - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ apt-get update + + $ apt-get -y install telnet + + $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 + Trying 127.0.0.1... + Connected to 127.0.0.1. + Escape character is '^]'. + hello + world + + ``` 8. The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello + ```bash + + ----- got message ----- + hello - ----- got message ----- - world - ``` + ----- got message ----- + world + + ``` ### HTTP 1. Start Pulsar standalone. - ```bash - $ docker pull apachepulsar/pulsar:{version} + ```bash + + $ docker pull apachepulsar/pulsar:{version} - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - ``` + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 2. Create a configuration file _netty-source-config.yaml_. + + ```yaml - ```yaml - configs: - type: "http" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - ``` + configs: + type: "http" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` 3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. + + ```bash - ```bash - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ - ``` + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar - ``` - -5. Start the Netty source connector. + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + ``` + +5. Start the Netty source connector. + ```bash + $ ./bin/pulsar-admin sources localrun \ - --archive pulsar-io-{{pulsar:version}}.nar \ + --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ --namespace default \ --name netty \ --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 + ``` 6. Consume data. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ + + ``` 8. The following information appears on the consumer terminal window. - ```bash - ----- got message ----- - hello, world! - ``` + ```bash + + ----- got message ----- + hello, world! + + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-nsq-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-nsq-source.md index b5305c70a1aa6..0bf16463b160b 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-nsq-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-nsq-source.md @@ -1,7 +1,7 @@ --- id: io-nsq-source title: NSQ source connector -sidebar_label: NSQ source connector +sidebar_label: "NSQ source connector" original_id: io-nsq-source --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-overview.md b/site2/website-next/versioned_docs/version-2.7.3/io-overview.md index 143ef7789caf5..3a55ff2aef2cc 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-overview.md @@ -1,7 +1,7 @@ --- id: io-overview title: Pulsar connector overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: io-overview --- @@ -25,12 +25,7 @@ This diagram illustrates the relationship between source, Pulsar, and sink: ### Source -:::note - -Sources **feed data from external systems into Pulsar**. - -::: - +> Sources **feed data from external systems into Pulsar**. Common sources include other messaging systems and firehose-style data pipeline APIs. @@ -38,12 +33,7 @@ For the complete list of Pulsar built-in source connectors, see [source connecto ### Sink -:::note - -Sinks **feed data from Pulsar into external systems**. - -::: - +> Sinks **feed data from Pulsar into external systems**. Common sinks include other messaging systems and SQL and NoSQL databases. @@ -53,12 +43,7 @@ For the complete list of Pulsar built-in sink connectors, see [sink connector](i Processing guarantees are used to handle errors when writing messages to Pulsar topics. -:::note - -Pulsar connectors and Functions use the **same** processing guarantees as below. - -::: - +> Pulsar connectors and Functions use the **same** processing guarantees as below. Delivery semantic | Description :------------------|:------- @@ -66,12 +51,7 @@ Delivery semantic | Description `at-least-once` | Each message sent to a connector is to be **processed once** or **more than once**. `effectively-once` | Each message sent to a connector has **one output associated** with it. -:::note - -Processing guarantees for connectors not just rely on Pulsar guarantee but also **relate to external systems**, that is, **the implementation of source and sink**. - -::: - +> Processing guarantees for connectors not just rely on Pulsar guarantee but also **relate to external systems**, that is, **the implementation of source and sink**. * Source: Pulsar ensures that writing messages to Pulsar topics respects to the processing guarantees. It is within Pulsar's control. @@ -87,12 +67,7 @@ When creating a connector, you can set the processing guarantee with the followi * EFFECTIVELY_ONCE -:::note - -If `--processing-guarantees` is not specified when creating a connector, the default semantic is `ATLEAST_ONCE`. - -::: - +> If `--processing-guarantees` is not specified when creating a connector, the default semantic is `ATLEAST_ONCE`. Here takes **Admin CLI** as an example. For more information about **REST API** or **JAVA Admin API**, see [here](io-use.md#create). @@ -112,9 +87,11 @@ Here takes **Admin CLI** as an example. For more information about **REST API** ```bash + $ bin/pulsar-admin sources create \ --processing-guarantees ATMOST_ONCE \ # Other source configs + ``` For more information about the options of `pulsar-admin sources create`, see [here](reference-connector-admin.md#create). @@ -123,9 +100,11 @@ For more information about the options of `pulsar-admin sources create`, see [he ```bash + $ bin/pulsar-admin sinks create \ --processing-guarantees EFFECTIVELY_ONCE \ # Other sink configs + ``` For more information about the options of `pulsar-admin sinks create`, see [here](reference-connector-admin.md#create-1). @@ -162,9 +141,11 @@ Here takes **Admin CLI** as an example. For more information about **REST API** ```bash + $ bin/pulsar-admin sources update \ --processing-guarantees EFFECTIVELY_ONCE \ # Other source configs + ``` For more information about the options of `pulsar-admin sources update`, see [here](reference-connector-admin.md#update). @@ -173,9 +154,11 @@ For more information about the options of `pulsar-admin sources update`, see [he ```bash + $ bin/pulsar-admin sinks update \ --processing-guarantees ATMOST_ONCE \ # Other sink configs + ``` For more information about the options of `pulsar-admin sinks update`, see [here](reference-connector-admin.md#update-1). @@ -187,7 +170,7 @@ For more information about the options of `pulsar-admin sinks update`, see [here ## Work with connector -You can manage Pulsar connectors (for example, create, update, start, stop, restart, reload, delete and perform other operations on connectors) via the [Connector Admin CLI](reference-connector-admin.md) with [sources](reference-connector-admin.md#sources) and [sinks](reference-connector-admin.md#sinks) subcommands. +You can manage Pulsar connectors (for example, create, update, start, stop, restart, reload, delete and perform other operations on connectors) via the [Connector Admin CLI](reference-connector-admin) with [sources](reference-connector-admin.md#sources) and [sinks](reference-connector-admin.md#sinks) subcommands. -Connectors (sources and sinks) and Functions are components of instances, and they all run on Functions workers. When managing a source, sink or function via [Connector Admin CLI](reference-connector-admin.md) or [Functions Admin CLI](functions-cli.md), an instance is started on a worker. For more information, see [Functions worker](functions-worker.md#run-functions-worker-separately). +Connectors (sources and sinks) and Functions are components of instances, and they all run on Functions workers. When managing a source, sink or function via [Connector Admin CLI](reference-connector-admin.md) or [Functions Admin CLI](functions-cli), an instance is started on a worker. For more information, see [Functions worker](functions-worker.md#run-functions-worker-separately). diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-quickstart.md b/site2/website-next/versioned_docs/version-2.7.3/io-quickstart.md index 18e903c7bd083..67ca9710a10a8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-quickstart.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-quickstart.md @@ -1,7 +1,7 @@ --- id: io-quickstart title: How to connect Pulsar to database -sidebar_label: Get started +sidebar_label: "Get started" original_id: io-quickstart --- @@ -11,7 +11,7 @@ import TabItem from '@theme/TabItem'; This tutorial provides a hands-on look at how you can move data out of Pulsar without writing a single line of code. -It is helpful to review the [concepts](io-overview.md) for Pulsar I/O with running the steps in this guide to gain a deeper understanding. +It is helpful to review the [concepts](io-overview) for Pulsar I/O with running the steps in this guide to gain a deeper understanding. At the end of this tutorial, you are able to: @@ -21,14 +21,12 @@ At the end of this tutorial, you are able to: :::tip -* These instructions assume you are running Pulsar in [standalone mode](getting-started-standalone.md). However, all +* These instructions assume you are running Pulsar in [standalone mode](getting-started-standalone). However, all the commands used in this tutorial can be used in a multi-nodes Pulsar cluster without any changes. - * All the instructions are assumed to run at the root directory of a Pulsar binary distribution. ::: - ## Install Pulsar and built-in connector Before connecting Pulsar to a database, you need to install Pulsar and the desired built-in connector. @@ -39,56 +37,74 @@ For more information about **how to install a standalone Pulsar and built-in con 1. Start Pulsar locally. - ```bash - bin/pulsar standalone - ``` + ```bash + + bin/pulsar standalone + + ``` - All the components of a Pulsar service are start in order. - - You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. + All the components of a Pulsar service are start in order. + + You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. 2. Check Pulsar binary protocol port. - ```bash - telnet localhost 6650 - ``` + ```bash + + telnet localhost 6650 + + ``` 3. Check Pulsar Function cluster. - ```bash - curl -s http://localhost:8080/admin/v2/worker/cluster - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/worker/cluster + + ``` - **Example output** - ```json - [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] - ``` + **Example output** + + ```json + + [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] + + ``` 4. Make sure a public tenant and a default namespace exist. - ```bash - curl -s http://localhost:8080/admin/v2/namespaces/public - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/namespaces/public + + ``` + + **Example output** - **Example output** - ```json - ["public/default","public/functions"] - ``` + ```json + + ["public/default","public/functions"] + + ``` 5. All built-in connectors should be listed as available. - ```bash - curl -s http://localhost:8080/admin/v2/functions/connectors - ``` + ```bash + + curl -s http://localhost:8080/admin/v2/functions/connectors + + ``` - **Example output** + **Example output** - ```json - [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] - ``` + ```json + + [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] + + ``` - If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, - or you can navigate to the `logs` directory under the Pulsar directory to view the logs. + If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, + or you can navigate to the `logs` directory under the Pulsar directory to view the logs. ## Connect Pulsar to Cassandra @@ -97,80 +113,93 @@ This section demonstrates how to connect Pulsar to Cassandra. :::tip * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - -* The Cassandra sink connector reads messages from Pulsar topics and writes the messages into Cassandra tables. For more information, see [Cassandra sink connector](io-cassandra-sink.md). +* The Cassandra sink connector reads messages from Pulsar topics and writes the messages into Cassandra tables. For more information, see [Cassandra sink connector](io-cassandra-sink). ::: - ### Setup a Cassandra cluster This example uses `cassandra` Docker image to start a single-node Cassandra cluster in Docker. 1. Start a Cassandra cluster. - ```bash - docker run -d --rm --name=cassandra -p 9042:9042 cassandra - ``` - - :::note + ```bash + + docker run -d --rm --name=cassandra -p 9042:9042 cassandra + + ``` - Before moving to the next steps, make sure the Cassandra cluster is running. + :::note - ::: + Before moving to the next steps, make sure the Cassandra cluster is running. + ::: 2. Make sure the Docker process is running. - ```bash - docker ps - ``` + ```bash + + docker ps + + ``` 3. Check the Cassandra logs to make sure the Cassandra process is running as expected. - ```bash - docker logs cassandra - ``` + ```bash + + docker logs cassandra + + ``` 4. Check the status of the Cassandra cluster. - ```bash - docker exec cassandra nodetool status - ``` + ```bash + + docker exec cassandra nodetool status + + ``` - **Example output** + **Example output** - ``` - Datacenter: datacenter1 - ======================= - Status=Up/Down - |/ State=Normal/Leaving/Joining/Moving - -- Address Load Tokens Owns (effective) Host ID Rack - UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 - ``` + ``` + + Datacenter: datacenter1 + ======================= + Status=Up/Down + |/ State=Normal/Leaving/Joining/Moving + -- Address Load Tokens Owns (effective) Host ID Rack + UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 + + ``` 5. Use `cqlsh` to connect to the Cassandra cluster. - ```bash - $ docker exec -ti cassandra cqlsh localhost - Connected to Test Cluster at localhost:9042. - [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] - Use HELP for help. - cqlsh> - ``` + ```bash + + $ docker exec -ti cassandra cqlsh localhost + Connected to Test Cluster at localhost:9042. + [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] + Use HELP for help. + cqlsh> + + ``` 6. Create a keyspace `pulsar_test_keyspace`. - ```bash - cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; - ``` + ```bash + + cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; + + ``` 7. Create a table `pulsar_test_table`. - ```bash - cqlsh> USE pulsar_test_keyspace; - cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); - ``` + ```bash + + cqlsh> USE pulsar_test_keyspace; + cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); + + ``` ### Configure a Cassandra sink @@ -186,44 +215,45 @@ You can create a configuration file through one of the following methods. * JSON - ```json - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - ``` + ```json + + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` * YAML + + ```yaml - ```yaml - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" - ``` + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" -For more information, see [Cassandra sink connector](io-cassandra-sink.md). + ``` + +For more information, see [Cassandra sink connector](io-cassandra-sink). ### Create a Cassandra sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to create a sink connector and perform other operations on them. Run the following command to create a Cassandra sink connector with sink type _cassandra_ and the config file _examples/cassandra-sink.yml_ created previously. #### Note -:::note - -The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file. - -::: +> The `sink-type` parameter of the currently built-in connectors is determined by the setting of the `name` parameter specified in the pulsar-io.yaml file. ```bash + bin/pulsar-admin sinks create \ --tenant public \ --namespace default \ @@ -231,6 +261,7 @@ bin/pulsar-admin sinks create \ --sink-type cassandra \ --sink-config-file examples/cassandra-sink.yml \ --inputs test_cassandra + ``` Once the command is executed, Pulsar creates the sink connector _cassandra-test-sink_. @@ -240,160 +271,180 @@ as a Pulsar Function and writes the messages produced in the topic _test_cassand ### Inspect a Cassandra sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to monitor a connector and perform other operations on it. * Get the information of a Cassandra sink. ```bash + bin/pulsar-admin sinks get \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json + { - "tenant": "public", - "namespace": "default", - "name": "cassandra-test-sink", - "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", - "inputSpecs": { - "test_cassandra": { - "isRegexPattern": false - } - }, - "configs": { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true, - "archive": "builtin://cassandra" + "tenant": "public", + "namespace": "default", + "name": "cassandra-test-sink", + "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", + "inputSpecs": { + "test_cassandra": { + "isRegexPattern": false + } + }, + "configs": { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true, + "archive": "builtin://cassandra" } + ``` * Check the status of a Cassandra sink. ```bash + bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] } + ``` ### Verify a Cassandra sink 1. Produce some messages to the input topic of the Cassandra sink _test_cassandra_. - ```bash - for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done - ``` + ```bash + + for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done + + ``` 2. Inspect the status of the Cassandra sink _test_cassandra_. - ```bash - bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink - ``` - - You can see 10 messages are processed by the Cassandra sink _test_cassandra_. - - **Example output** - - ```json - { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 10, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 10, - "lastReceivedTime" : 1551685489136, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] - } - ``` + ```bash + + bin/pulsar-admin sinks status \ + --tenant public \ + --namespace default \ + --name cassandra-test-sink + + ``` + + You can see 10 messages are processed by the Cassandra sink _test_cassandra_. + + **Example output** + + ```json + + { + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 10, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 10, + "lastReceivedTime" : 1551685489136, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] + } + + ``` 3. Use `cqlsh` to connect to the Cassandra cluster. ```bash + docker exec -ti cassandra cqlsh localhost + ``` 4. Check the data of the Cassandra table _pulsar_test_table_. ```bash + cqlsh> use pulsar_test_keyspace; cqlsh:pulsar_test_keyspace> select * from pulsar_test_table; key | col --------+-------- - key-5 | key-5 - key-0 | key-0 - key-9 | key-9 - key-2 | key-2 - key-1 | key-1 - key-3 | key-3 - key-6 | key-6 - key-7 | key-7 - key-4 | key-4 - key-8 | key-8 + key-5 | key-5 + key-0 | key-0 + key-9 | key-9 + key-2 | key-2 + key-1 | key-1 + key-3 | key-3 + key-6 | key-6 + key-7 | key-7 + key-4 | key-4 + key-8 | key-8 + ``` ### Delete a Cassandra Sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to delete a connector and perform other operations on it. ```bash + bin/pulsar-admin sinks delete \ --tenant public \ --namespace default \ --name cassandra-test-sink + ``` ## Connect Pulsar to PostgreSQL @@ -403,18 +454,12 @@ This section demonstrates how to connect Pulsar to PostgreSQL. :::tip * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - * The JDBC sink connector pulls messages from Pulsar topics ::: and persists the messages to ClickHouse, MariaDB, PostgreSQL, or SQlite. -:::note - -For more information, see [JDBC sink connector](io-jdbc-sink.md). - -::: - +>For more information, see [JDBC sink connector](io-jdbc-sink). ### Setup a PostgreSQL cluster @@ -423,73 +468,84 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS 1. Pull the PostgreSQL 12 image from Docker. - ```bash - $ docker pull postgres:12 - ``` + ```bash + + $ docker pull postgres:12 + + ``` 2. Start PostgreSQL. - ```bash - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=password \ - -e POSTGRES_USER=postgres \ - postgres:12 - ``` - - #### Tip - - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=password \ + -e POSTGRES_USER=postgres \ + postgres:12 + + ``` - :::tip + #### Tip + + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + :::tip - ::: + For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + ::: 3. Check if PostgreSQL has been started successfully. - ```bash - $ docker logs -f pulsar-postgres - ``` + ```bash + + $ docker logs -f pulsar-postgres + + ``` - PostgreSQL has been started successfully if the following message appears. + PostgreSQL has been started successfully if the following message appears. - ```text - 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 - 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" - 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC - 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections - ``` + ```text + + 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 + 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" + 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC + 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections + + ``` 4. Access to PostgreSQL. - ```bash - $ docker exec -it pulsar-postgres /bin/bash - ``` + ```bash + + $ docker exec -it pulsar-postgres /bin/bash + + ``` 5. Create a PostgreSQL table _pulsar_postgres_jdbc_sink_. - ```bash - $ psql -U postgres postgres - - postgres=# create table if not exists pulsar_postgres_jdbc_sink - ( - id serial PRIMARY KEY, - name VARCHAR(255) NOT NULL - ); - ``` + ```bash + + $ psql -U postgres postgres + + postgres=# create table if not exists pulsar_postgres_jdbc_sink + ( + id serial PRIMARY KEY, + name VARCHAR(255) NOT NULL + ); + + ``` ### Configure a JDBC sink @@ -499,74 +555,84 @@ In this section, you need to configure a JDBC sink connector. 1. Add a configuration file. - To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. - - For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. + To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. + + For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. - Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. + Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - ```yaml - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" - ``` + ```yaml + + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" + tableName: "pulsar_postgres_jdbc_sink" + + ``` 2. Create a schema. - Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - - ```json - { - "type": "AVRO", - "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", - "properties": {} - } - ``` - - :::tip + Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - For more information about AVRO, see [Apache Avro](https://avro.apache.org/docs/1.9.1/). + ```json + + { + "type": "AVRO", + "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", + "properties": {} + } + + ``` - ::: + :::tip + For more information about AVRO, see [Apache Avro](https://avro.apache.org/docs/1.9.1/). + ::: 3. Upload a schema to a topic. - This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. + This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. - ```bash - $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema - ``` + ```bash + + $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema + + ``` 4. Check if the schema has been uploaded successfully. - ```bash - $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic - ``` + ```bash + + $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic + + ``` - The schema has been uploaded successfully if the following message appears. + The schema has been uploaded successfully if the following message appears. - ```json - {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} - ``` + ```json + + {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} + + ``` ### Create a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to create a sink connector and perform other operations on it. This example creates a sink connector and specifies the desired information. ```bash + $ bin/pulsar-admin sinks create \ ---archive ./connectors/pulsar-io-jdbc-postgres-{{pulsar:version}}.nar \ +--archive ./connectors/pulsar-io-jdbc-postgres-@pulsar:version@.nar \ --inputs pulsar-postgres-jdbc-sink-topic \ --name pulsar-postgres-jdbc-sink \ --sink-config-file ./connectors/pulsar-postgres-jdbc-sink.yaml \ --parallelism 1 + ``` Once the command is executed, Pulsar creates a sink connector _pulsar-postgres-jdbc-sink_. @@ -577,7 +643,7 @@ This sink connector runs as a Pulsar Function and writes the messages produced i Flag | Description | This example ---|---|---| - `--archive` | The path to the archive file for the sink. | _pulsar-io-jdbc-postgres-{{pulsar:version}}.nar_ | + `--archive` | The path to the archive file for the sink. | _pulsar-io-jdbc-postgres-@pulsar:version@.nar_ | `--inputs` | The input topic(s) of the sink.

    Multiple topics can be specified as a comma-separated list.|| `--name` | The name of the sink. | _pulsar-postgres-jdbc-sink_ | `--sink-config-file` | The path to a YAML config file specifying the configuration of the sink. | _pulsar-postgres-jdbc-sink.yaml_ | @@ -589,24 +655,27 @@ For more information about `pulsar-admin sinks create options`, see [here](io-cl ::: - The sink has been created successfully if the following message appears. ```bash + "Created successfully" + ``` ### Inspect a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to monitor a connector and perform other operations on it. * List all running JDBC sink(s). ```bash + $ bin/pulsar-admin sinks list \ --tenant public \ --namespace default + ``` :::tip @@ -615,22 +684,25 @@ to monitor a connector and perform other operations on it. ::: - The result shows that only the _postgres-jdbc-sink_ sink is running. ```json + [ "pulsar-postgres-jdbc-sink" ] + ``` * Get the information of a JDBC sink. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -639,40 +711,43 @@ to monitor a connector and perform other operations on it. ::: - The result shows the information of the sink connector, including tenant, namespace, topic and so on. ```json + { - "tenant": "public", - "namespace": "default", - "name": "pulsar-postgres-jdbc-sink", - "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", - "inputSpecs": { - "pulsar-postgres-jdbc-sink-topic": { - "isRegexPattern": false - } - }, - "configs": { - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "userName": "postgres", - "tableName": "pulsar_postgres_jdbc_sink" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true + "tenant": "public", + "namespace": "default", + "name": "pulsar-postgres-jdbc-sink", + "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", + "inputSpecs": { + "pulsar-postgres-jdbc-sink-topic": { + "isRegexPattern": false + } + }, + "configs": { + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "userName": "postgres", + "tableName": "pulsar_postgres_jdbc_sink" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true } + ``` * Get the status of a JDBC sink ```bash + $ bin/pulsar-admin sinks status \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -681,42 +756,45 @@ to monitor a connector and perform other operations on it. ::: - The result shows the current status of sink connector, including the number of instance, running status, worker ID and so on. ```json + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-192.168.2.52-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-192.168.2.52-8080" + } + } ] } + ``` ### Stop a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to stop a connector and perform other operations on it. ```bash + $ bin/pulsar-admin sinks stop \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -725,23 +803,26 @@ For more information about `pulsar-admin sinks stop options`, see [here](io-cli. ::: - The sink instance has been stopped successfully if the following message disappears. ```bash + "Stopped successfully" + ``` ### Restart a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to restart a connector and perform other operations on it. ```bash + $ bin/pulsar-admin sinks restart \ --tenant public \ --namespace default \ ---name pulsar-postgres-jdbc-sink +--name pulsar-postgres-jdbc-sink + ``` :::tip @@ -750,35 +831,35 @@ For more information about `pulsar-admin sinks restart options`, see [here](io-c ::: - The sink instance has been started successfully if the following message disappears. ```bash + "Started successfully" + ``` :::tip * Optionally, you can run a standalone sink connector using `pulsar-admin sinks localrun options`. - Note that `pulsar-admin sinks localrun options` **runs a sink connector locally**, while `pulsar-admin sinks start options` **starts a sink connector in a cluster**. - * For more information about `pulsar-admin sinks localrun options`, see [here](io-cli.md#localrun-1). ::: - ### Update a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to update a connector and perform other operations on it. This example updates the parallelism of the _pulsar-postgres-jdbc-sink_ sink connector to 2. ```bash + $ bin/pulsar-admin sinks update \ --name pulsar-postgres-jdbc-sink \ --parallelism 2 + ``` :::tip @@ -787,25 +868,29 @@ For more information about `pulsar-admin sinks update options`, see [here](io-cl ::: - The sink connector has been updated successfully if the following message disappears. ```bash + "Updated successfully" + ``` This example double-checks the information. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` The result shows that the parallelism is 2. ```json + { "tenant": "public", "namespace": "default", @@ -827,20 +912,23 @@ The result shows that the parallelism is 2. "retainOrdering": false, "autoAck": true } + ``` ### Delete a JDBC sink -You can use the [Connector Admin CLI](io-cli.md) +You can use the [Connector Admin CLI](io-cli) to delete a connector and perform other operations on it. This example deletes the _pulsar-postgres-jdbc-sink_ sink connector. ```bash + $ bin/pulsar-admin sinks delete \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` :::tip @@ -849,26 +937,32 @@ For more information about `pulsar-admin sinks delete options`, see [here](io-cl ::: - The sink connector has been deleted successfully if the following message appears. ```text + "Deleted successfully" + ``` This example double-checks the status of the sink connector. ```bash + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink + ``` The result shows that the sink connector does not exist. ```text + HTTP 404 Not Found Reason: Sink pulsar-postgres-jdbc-sink doesn't exist + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-sink.md index e27911b8b2162..48f1c75678559 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-sink.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-sink title: RabbitMQ sink connector -sidebar_label: RabbitMQ sink connector +sidebar_label: "RabbitMQ sink connector" original_id: io-rabbitmq-sink --- @@ -44,42 +44,46 @@ Before using the RabbitMQ sink connector, you need to create a configuration fil * JSON - ```json - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "exchangeName": "test-exchange", - "routingKey": "test-key" - } - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "exchangeName": "test-exchange", + "routingKey": "test-key" + } + + ``` * YAML - ```yaml - configs: - host: "localhost" - port: 5672 - virtualHost: "/", - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - exchangeName: "test-exchange" - routingKey: "test-key" - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/", + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + exchangeName: "test-exchange" + routingKey: "test-key" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-source.md index 51a17841c59a0..0a10cfaab1b1a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-rabbitmq-source.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-source title: RabbitMQ source connector -sidebar_label: RabbitMQ source connector +sidebar_label: "RabbitMQ source connector" original_id: io-rabbitmq-source --- @@ -42,44 +42,48 @@ Before using the RabbitMQ source connector, you need to create a configuration f * JSON - ```json - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "prefetchCount": "0", - "prefetchGlobal": "false", - "passive": "false" - } - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "prefetchCount": "0", + "prefetchGlobal": "false", + "passive": "false" + } + + ``` * YAML - ```yaml - configs: - host: "localhost" - port: 5672 - virtualHost: "/" - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - prefetchCount: 0 - prefetchGlobal: "false" - passive: "false" - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/" + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + prefetchCount: 0 + prefetchGlobal: "false" + passive: "false" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-redis-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-redis-sink.md index 76924d0eb30c0..49eb119802736 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-redis-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-redis-sink.md @@ -1,7 +1,7 @@ --- id: io-redis-sink title: Redis sink connector -sidebar_label: Redis sink connector +sidebar_label: "Redis sink connector" original_id: io-redis-sink --- @@ -44,31 +44,35 @@ Before using the Redis sink connector, you need to create a configuration file t * JSON - ```json - { - "redisHosts": "localhost:6379", - "redisPassword": "fake@123", - "redisDatabase": "1", - "clientMode": "Standalone", - "operationTimeout": "2000", - "batchSize": "100", - "batchTimeMs": "1000", - "connectTimeout": "3000" - } - ``` + ```json + + { + "redisHosts": "localhost:6379", + "redisPassword": "fake@123", + "redisDatabase": "1", + "clientMode": "Standalone", + "operationTimeout": "2000", + "batchSize": "100", + "batchTimeMs": "1000", + "connectTimeout": "3000" + } + + ``` * YAML - ```yaml - { - redisHosts: "localhost:6379" - redisPassword: "fake@123" - redisDatabase: 1 - clientMode: "Standalone" - operationTimeout: 2000 - batchSize: 100 - batchTimeMs: 1000 - connectTimeout: 3000 - } - ``` + ```yaml + + { + redisHosts: "localhost:6379" + redisPassword: "fake@123" + redisDatabase: 1 + clientMode: "Standalone" + operationTimeout: 2000 + batchSize: 100 + batchTimeMs: 1000 + connectTimeout: 3000 + } + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-solr-sink.md b/site2/website-next/versioned_docs/version-2.7.3/io-solr-sink.md index e4dc6db1f673f..d7b31ad97cde8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-solr-sink.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-solr-sink.md @@ -1,7 +1,7 @@ --- id: io-solr-sink title: Solr sink connector -sidebar_label: Solr sink connector +sidebar_label: "Solr sink connector" original_id: io-solr-sink --- @@ -39,27 +39,31 @@ Before using the Solr sink connector, you need to create a configuration file th * JSON - ```json - { - "solrUrl": "localhost:2181,localhost:2182/chroot", - "solrMode": "SolrCloud", - "solrCollection": "techproducts", - "solrCommitWithinMs": 100, - "username": "fakeuser", - "password": "fake@123" - } - ``` + ```json + + { + "solrUrl": "localhost:2181,localhost:2182/chroot", + "solrMode": "SolrCloud", + "solrCollection": "techproducts", + "solrCommitWithinMs": 100, + "username": "fakeuser", + "password": "fake@123" + } + + ``` * YAML - ```yaml - { - solrUrl: "localhost:2181,localhost:2182/chroot" - solrMode: "SolrCloud" - solrCollection: "techproducts" - solrCommitWithinMs: 100 - username: "fakeuser" - password: "fake@123" - } - ``` + ```yaml + + { + solrUrl: "localhost:2181,localhost:2182/chroot" + solrMode: "SolrCloud" + solrCollection: "techproducts" + solrCommitWithinMs: 100 + username: "fakeuser" + password: "fake@123" + } + + ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-twitter-source.md b/site2/website-next/versioned_docs/version-2.7.3/io-twitter-source.md index 313e63386144f..101602e246fb8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-twitter-source.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-twitter-source.md @@ -1,7 +1,7 @@ --- id: io-twitter-source title: Twitter Firehose source connector -sidebar_label: Twitter Firehose source connector +sidebar_label: "Twitter Firehose source connector" original_id: io-twitter-source --- @@ -29,9 +29,4 @@ The configuration of the Twitter Firehose source connector has the following pro | `clientHosts` |String| false | Constants.STREAM_HOST | The twitter firehose hosts to which client connects. | | `clientBufferSize` | int|false | 50000 | The buffer size for buffering tweets fetched from twitter firehose. | -:::note - -For more information about OAuth credentials, see [Twitter developers portal](https://developer.twitter.com/en.html). - -::: - +> For more information about OAuth credentials, see [Twitter developers portal](https://developer.twitter.com/en.html). diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-twitter.md b/site2/website-next/versioned_docs/version-2.7.3/io-twitter.md index 11f583b329e3f..53f949863702d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-twitter.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-twitter.md @@ -1,7 +1,7 @@ --- id: io-twitter title: Twitter Firehose Connector -sidebar_label: Twitter Firehose Connector +sidebar_label: "Twitter Firehose Connector" original_id: io-twitter --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/io-use.md b/site2/website-next/versioned_docs/version-2.7.3/io-use.md index 0123de232f316..43b7a30cadc33 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/io-use.md +++ b/site2/website-next/versioned_docs/version-2.7.3/io-use.md @@ -1,7 +1,7 @@ --- id: io-use title: How to use Pulsar connectors -sidebar_label: Use +sidebar_label: "Use" original_id: io-use --- @@ -13,7 +13,7 @@ This guide describes how to use Pulsar connectors. ## Install a connector -Pulsar bundles several [builtin connectors](io-connectors.md) used to move data in and out of commonly used systems (such as database and messaging system). Optionally, you can create and use your desired non-builtin connectors. +Pulsar bundles several [builtin connectors](io-connectors) used to move data in and out of commonly used systems (such as database and messaging system). Optionally, you can create and use your desired non-builtin connectors. :::note @@ -21,7 +21,6 @@ When using a non-builtin connector, you need to specify the path of a archive fi ::: - To set up a builtin connector, follow the instructions [here](getting-started-standalone.md#installing-builtin-connectors). @@ -44,11 +43,13 @@ To configure a default folder for builtin connectors, set the `connectorsDirecto Set the `./connectors` folder as the default storage location for builtin connectors. ``` + ######################## # Connectors ######################## connectorsDirectory: ./connectors + ``` ### Configure a connector with a YAML file @@ -68,6 +69,7 @@ Below is a YAML configuration file of a Cassandra sink, which tells Pulsar: * How to map Pulsar messages into Cassandra table key and columns ```shell + tenant: public namespace: default name: cassandra-test-sink @@ -79,6 +81,7 @@ configs: columnFamily: "pulsar_test_table" keyname: "key" columnName: "col" + ``` **Example 2** @@ -86,12 +89,14 @@ configs: Below is a YAML configuration file of a Kafka source. ```shell + configs: bootstrapServers: "pulsar-kafka:9092" groupId: "test-pulsar-io" topic: "my-topic" sessionTimeoutMs: "10000" autoCommitEnabled: "false" + ``` **Example 3** @@ -99,11 +104,13 @@ configs: Below is a YAML configuration file of a PostgreSQL JDBC sink. ```shell + configs: userName: "postgres" password: "password" jdbcUrl: "jdbc:postgresql://localhost:5432/test_jdbc" tableName: "test_jdbc" + ``` ## Get available connectors @@ -123,7 +130,9 @@ If you add or delete a nar file in a connector folder, reload the available buil Use the `reload` subcommand. ```shell + $ pulsar-admin sources reload + ``` For more information, see [`here`](io-cli.md#reload). @@ -133,7 +142,9 @@ For more information, see [`here`](io-cli.md#reload). Use the `reload` subcommand. ```shell + $ pulsar-admin sinks reload + ``` For more information, see [`here`](io-cli.md#reload-1). @@ -147,7 +158,9 @@ After reloading connectors (optional), you can get a list of available connector Use the `available-sources` subcommand. ```shell + $ pulsar-admin sources available-sources + ``` #### Sink @@ -155,7 +168,9 @@ $ pulsar-admin sources available-sources Use the `available-sinks` subcommand. ```shell + $ pulsar-admin sinks available-sinks + ``` ## Run a connector @@ -198,7 +213,9 @@ Create a source connector. Use the `create` subcommand. ``` + $ pulsar-admin sources create options + ``` For more information, see [here](io-cli.md#create). @@ -213,56 +230,60 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/source * Create a source connector with a **local file**. - ```java - void createSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void createSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sourceConfig` | The source configuration object + |Name|Description + |---|--- + `sourceConfig` | The source configuration object **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). + For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). * Create a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void createSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void createSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sourceConfig` | The source configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sourceConfig` | The source configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-). + For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-).
    @@ -294,7 +315,9 @@ Create a sink connector. Use the `create` subcommand. ``` + $ pulsar-admin sinks create options + ``` For more information, see [here](io-cli.md#create-1). @@ -308,57 +331,61 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/ * Create a sink connector with a **local file**. + + ```java - ```java - void createSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - ``` + void createSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sinkConfig` | The sink configuration object + |Name|Description + |---|--- + `sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). + For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). * Create a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void createSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void createSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sinkConfig` | The sink configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sinkConfig` | The sink configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). + For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). @@ -390,7 +417,9 @@ Start a source connector. Use the `start` subcommand. ``` + $ pulsar-admin sources start options + ``` For more information, see [here](io-cli.md#start). @@ -400,11 +429,11 @@ For more information, see [here](io-cli.md#start). * Start **all** source connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} * Start a **specified** source connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@}
    @@ -432,7 +461,9 @@ Start a sink connector. Use the `start` subcommand. ``` + $ pulsar-admin sinks start options + ``` For more information, see [here](io-cli.md#start-1). @@ -442,11 +473,11 @@ For more information, see [here](io-cli.md#start-1). * Start **all** sink connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} * Start a **specified** sink connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@}
    @@ -474,7 +505,9 @@ Run a source connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sources localrun options + ``` For more information, see [here](io-cli.md#localrun). @@ -501,7 +534,9 @@ Run a sink connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sinks localrun options + ``` For more information, see [here](io-cli.md#localrun-1). @@ -550,7 +585,9 @@ Get the information of a source connector. Use the `get` subcommand. ``` + $ pulsar-admin sources get options + ``` For more information, see [here](io-cli.md#get). @@ -564,10 +601,12 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sources/ ```java + SourceConfig getSource(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Example** @@ -575,6 +614,7 @@ SourceConfig getSource(String tenant, This is a sourceConfig. ```java + { "tenant": "tenantName", "namespace": "namespaceName", @@ -590,11 +630,13 @@ This is a sourceConfig. "disk": 10737418240 } } + ``` This is a sourceConfig example. ``` + { "tenant": "public", "namespace": "default", @@ -624,6 +666,7 @@ This is a sourceConfig example. "disk": 10737418240 } } + ``` **Exception** @@ -666,7 +709,9 @@ Get the information of a sink connector. Use the `get` subcommand. ``` + $ pulsar-admin sinks get options + ``` For more information, see [here](io-cli.md#get-1). @@ -680,10 +725,12 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sinks/:t ```java + SinkConfig getSink(String tenant, String namespace, String sink) throws PulsarAdminException + ``` **Example** @@ -691,6 +738,7 @@ SinkConfig getSink(String tenant, This is a sinkConfig. ```json + { "tenant": "tenantName", "namespace": "namespaceName", @@ -707,11 +755,13 @@ This is a sinkConfig. "retainOrdering": false, "autoAck": true } + ``` This is a sinkConfig example. ```json + { "tenant": "public", "namespace": "default", @@ -733,6 +783,7 @@ This is a sinkConfig example. "retainOrdering": false, "autoAck": true } + ``` **Parameter description** @@ -779,7 +830,9 @@ Get the list of all running source connectors. Use the `list` subcommand. ``` + $ pulsar-admin sources list options + ``` For more information, see [here](io-cli.md#list). @@ -793,15 +846,19 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sources/ ```java + List listSources(String tenant, String namespace) throws PulsarAdminException + ``` **Response example** ```java + ["f1", "f2", "f3"] + ``` **Exception** @@ -843,7 +900,9 @@ Get the list of all running sink connectors. Use the `list` subcommand. ``` + $ pulsar-admin sinks list options + ``` For more information, see [here](io-cli.md#list-1). @@ -857,15 +916,19 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v3/sinks/:t ```java + List listSinks(String tenant, String namespace) throws PulsarAdminException + ``` **Response example** ```java + ["f1", "f2", "f3"] + ``` **Exception** @@ -911,7 +974,9 @@ Get the current status of a source connector. Use the `status` subcommand. ``` + $ pulsar-admin sources status options + ``` For more information, see [here](io-cli.md#status). @@ -932,55 +997,59 @@ For more information, see [here](io-cli.md#status). * Get the current status of **all** source connectors. - ```java - SourceStatus getSourceStatus(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + SourceStatus getSourceStatus(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, - String namespace, - String source, - int id) - throws PulsarAdminException - ``` + ```java + + SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, + String namespace, + String source, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Source instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Source instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1012,7 +1081,9 @@ Get the current status of a Pulsar sink connector. Use the `status` subcommand. ``` + $ pulsar-admin sinks status options + ``` For more information, see [here](io-cli.md#status-1). @@ -1033,55 +1104,59 @@ For more information, see [here](io-cli.md#status-1). * Get the current status of **all** sink connectors. - ```java - SinkStatus getSinkStatus(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + SinkStatus getSinkStatus(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, - String namespace, - String sink, - int id) - throws PulsarAdminException - ``` + ```java + + SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, + String namespace, + String sink, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Sink instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Sink instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1119,7 +1194,9 @@ Update a running Pulsar source connector. Use the `update` subcommand. ``` + $ pulsar-admin sources update options + ``` For more information, see [here](io-cli.md#update). @@ -1134,58 +1211,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sources/ * Update a running source connector with a **local file**. - ```java - void updateSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void updateSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sourceConfig` | The source configuration object + | Name | Description + |---|--- + |`sourceConfig` | The source configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). + For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). * Update a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void updateSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void updateSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sourceConfig` | The source configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sourceConfig` | The source configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSourceWithUrl-SourceConfig-java.lang.String-). @@ -1219,7 +1300,9 @@ Update a running Pulsar sink connector. Use the `update` subcommand. ``` + $ pulsar-admin sinks update options + ``` For more information, see [here](io-cli.md#update-1). @@ -1234,58 +1317,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sinks/:t * Update a running sink connector with a **local file**. - ```java - void updateSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void updateSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sinkConfig` | The sink configuration object + | Name | Description + |---|--- + |`sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). + For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). * Update a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - void updateSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - ``` + ```java + + void updateSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sinkConfig` | The sink configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sinkConfig` | The sink configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - |`PulsarAdminException.NotFoundException` | Cluster doesn't exist - |`PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + |`PulsarAdminException.NotFoundException` | Cluster doesn't exist + |`PulsarAdminException` | Unexpected error For more information, see [`updateSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSinkWithUrl-SinkConfig-java.lang.String-). @@ -1325,7 +1412,9 @@ Stop a source connector. Use the `stop` subcommand. ``` + $ pulsar-admin sources stop options + ``` For more information, see [here](io-cli.md#stop). @@ -1346,55 +1435,59 @@ For more information, see [here](io-cli.md#stop). * Stop **all** source connectors. - ```java - void stopSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** source connector. - ```java - void stopSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1426,7 +1519,9 @@ Stop a sink connector. Use the `stop` subcommand. ``` + $ pulsar-admin sinks stop options + ``` For more information, see [here](io-cli.md#stop-1). @@ -1447,55 +1542,59 @@ For more information, see [here](io-cli.md#stop-1). * Stop **all** sink connectors. - ```java - void stopSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** sink connector. - ```java - void stopSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-). @@ -1533,7 +1632,9 @@ Restart a source connector. Use the `restart` subcommand. ``` + $ pulsar-admin sources restart options + ``` For more information, see [here](io-cli.md#restart). @@ -1554,55 +1655,59 @@ For more information, see [here](io-cli.md#restart). * Restart **all** source connectors. - ```java - void restartSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** source connector. - ```java - void restartSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1634,7 +1739,9 @@ Restart a sink connector. Use the `restart` subcommand. ``` + $ pulsar-admin sinks restart options + ``` For more information, see [here](io-cli.md#restart-1). @@ -1655,55 +1762,59 @@ For more information, see [here](io-cli.md#restart-1). * Restart all Pulsar sink connectors. - ```java - void restartSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Sink name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Sink name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** sink connector. - ```java - void restartSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Sink instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Sink instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1741,7 +1852,9 @@ Delete a source connector. Use the `delete` subcommand. ``` + $ pulsar-admin sources delete options + ``` For more information, see [here](io-cli.md#delete). @@ -1759,10 +1872,12 @@ Send a `DELETE` request to this endpoint: {@inject: endpoint|DELETE|/admin/v3/so Delete a source connector. ```java + void deleteSource(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Parameter** @@ -1814,7 +1929,9 @@ Delete a sink connector. Use the `delete` subcommand. ``` + $ pulsar-admin sinks delete options + ``` For more information, see [here](io-cli.md#delete-1). @@ -1832,10 +1949,12 @@ Send a `DELETE` request to this endpoint: {@inject: endpoint|DELETE|/admin/v3/si Delete a Pulsar sink connector. ```java + void deleteSink(String tenant, String namespace, String source) throws PulsarAdminException + ``` **Parameter** diff --git a/site2/website-next/versioned_docs/version-2.7.3/performance-pulsar-perf.md b/site2/website-next/versioned_docs/version-2.7.3/performance-pulsar-perf.md index 8196e2c49e5ac..d41c4cf203923 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/performance-pulsar-perf.md +++ b/site2/website-next/versioned_docs/version-2.7.3/performance-pulsar-perf.md @@ -1,7 +1,7 @@ --- id: performance-pulsar-perf title: Pulsar Perf -sidebar_label: Pulsar Perf +sidebar_label: "Pulsar Perf" original_id: performance-pulsar-perf --- @@ -20,6 +20,7 @@ The Pulsar Perf is a built-in performance test tool for Apache Pulsar. You can u This example shows how the Pulsar Perf produces messages with default options. For all configuration options available for the `pulsar-perf produce` command, see [configuration options](#configuration-options-for-pulsar-perf-produce). ``` + bin/pulsar-perf produce my-topic ``` @@ -29,6 +30,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 19:53:31.459 [pulsar-perf-producer-exec-1-1] INFO org.apache.pulsar.testclient.PerformanceProducer - Created 1 producers 19:53:31.482 [pulsar-timer-5-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 19:53:40.861 [main] INFO org.apache.pulsar.testclient.PerformanceProducer - Throughput produced: 93.7 msg/s --- 0.7 Mbit/s --- failure 0.0 msg/s --- Latency: mean: 3.575 ms - med: 3.460 - 95pct: 4.790 - 99pct: 5.308 - 99.9pct: 5.834 - 99.99pct: 6.609 - Max: 6.609 @@ -92,6 +94,7 @@ The following table lists configuration options available for the `pulsar-perf p This example shows how the Pulsar Perf consumes messages with default options. ``` + bin/pulsar-perf consume my-topic ``` @@ -101,6 +104,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 20:35:37.071 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Start receiving from 1 consumers on 1 topics 20:35:41.150 [pulsar-client-io-1-9] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 20:35:47.092 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Throughput received: 59.572 msg/s -- 0.465 Mbit/s --- Latency: mean: 11.298 ms - med: 10 - 95pct: 15 - 99pct: 98 - 99.9pct: 137 - 99.99pct: 152 - Max: 152 @@ -160,6 +164,7 @@ By default, the Pulsar Perf uses `conf/client.conf` as the default configuration You can use the following commands to change the configuration file and the Log4j configuration file. ``` + export PULSAR_CLIENT_CONF= export PULSAR_LOG_CONF= @@ -168,6 +173,7 @@ export PULSAR_LOG_CONF= In addition, you can use the following command to configure the JVM configuration through environment variables: ``` + export PULSAR_EXTRA_OPTS='-Xms4g -Xmx4g -XX:MaxDirectMemorySize=4g' ``` @@ -180,32 +186,38 @@ To check test results through the HdrHistogram Plotter, follow these steps: 1. Clone the HdrHistogram repository from GitHub to the local. - ``` - git clone https://github.com/HdrHistogram/HdrHistogram.git - - ``` + ``` + + git clone https://github.com/HdrHistogram/HdrHistogram.git + + ``` 2. Switch to the HdrHistogram folder. - ``` - cd HdrHistogram + ``` + + cd HdrHistogram + + ``` - ``` 3. Install the HdrHistogram Plotter. - ``` - mvn clean install -DskipTests + ``` + + mvn clean install -DskipTests + + ``` - ``` 4. Transform the file generated by the Pulsar Perf. - ``` - ./HistogramLogProcessor -i -o - - ``` + ``` + + ./HistogramLogProcessor -i -o + + ``` 5. You will get two output files. Upload the output file with the filename extension of .hgrm to the [HdrHistogram Plotter](https://hdrhistogram.github.io/HdrHistogram/plotFiles.html). 6. Check the test result through the Graphical User Interface of the HdrHistogram Plotter, as shown blow. - ![](/assets/perf-produce.png) + ![](/assets/perf-produce.png) diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-cli-tools.md b/site2/website-next/versioned_docs/version-2.7.3/reference-cli-tools.md index 081def9ec6117..d2d23d831d703 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-cli-tools.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-cli-tools.md @@ -1,7 +1,7 @@ --- id: reference-cli-tools title: Pulsar command-line tools -sidebar_label: Pulsar CLI tools +sidebar_label: "Pulsar CLI tools" original_id: reference-cli-tools --- @@ -22,9 +22,12 @@ All Pulsar command-line tools can be run from the `bin` directory of your [insta > ### Getting help > You can get help for any CLI tool, command, or subcommand using the `--help` flag, or `-h` for short. Here's an example: + > ```shell -> +> > $ bin/pulsar broker --help +> +> > ``` ## `pulsar` @@ -40,6 +43,7 @@ Usage: $ pulsar command ``` + Commands: * `bookie` * `broker` @@ -149,6 +153,7 @@ Usage $ pulsar compact-topic options ``` + Options |Flag|Description|Default| @@ -526,6 +531,7 @@ Run a consumer Usage ``` + $ pulsar-perf consume options ``` @@ -908,6 +914,7 @@ Usage $ broker-tool command ``` + Commands * `load-report` * `help` diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-configuration.md b/site2/website-next/versioned_docs/version-2.7.3/reference-configuration.md index 0c13c83319d65..da68b439fefb1 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-configuration.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-configuration.md @@ -1,7 +1,7 @@ --- id: reference-configuration title: Pulsar configuration -sidebar_label: Pulsar configuration +sidebar_label: "Pulsar configuration" original_id: reference-configuration --- @@ -203,7 +203,8 @@ Pulsar brokers are responsible for handling incoming messages from producers, di |tlsTrustCertsFilePath| Path for the trusted TLS certificate file. This cert is used to verify that any certs presented by connecting clients are signed by a certificate authority. If this verification fails, then the certs are untrusted and the connections are dropped. || |tlsAllowInsecureConnection| Accept untrusted TLS certificate from client. If it is set to `true`, a client with a cert which cannot be verified with the 'tlsTrustCertsFilePath' cert will be allowed to connect to the server, though the cert will not be used for client authentication. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.2```, ```TLSv1.1```, ```TLSv1``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + |tlsEnabledWithKeyStore| Enable TLS with KeyStore type configuration in broker |false| |tlsProvider| TLS Provider for KeyStore type || |tlsKeyStoreType| LS KeyStore type configuration in broker: JKS, PKCS12 |JKS| @@ -728,7 +729,8 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tlsHostnameVerificationEnabled| Whether the hostname is validated when the proxy creates a TLS connection with brokers |false| |tlsRequireTrustedClientCertOnConnect| Whether client certificates are required for TLS. Connections are rejected if the client certificate isn’t trusted. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.2```, ```TLSv1.1```, ```TLSv1``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + | httpReverseProxyConfigs | HTTP directs to redirect to non-pulsar services | | | httpOutputBufferSize | HTTP output buffer size. The amount of data that will be buffered for HTTP requests before it is flushed to the channel. A larger buffer size may result in higher HTTP throughput though it may take longer for the client to see data. If using HTTP streaming via the reverse proxy, this should be set to the minimum value (1) so that clients see the data as soon as possible. | 32768 | | httpNumThreads | Number of threads to use for HTTP requests processing| 2 * Runtime.getRuntime().availableProcessors() | diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-connector-admin.md b/site2/website-next/versioned_docs/version-2.7.3/reference-connector-admin.md index c6cc98f9271d6..b995e7bb231ca 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-connector-admin.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-connector-admin.md @@ -1,7 +1,7 @@ --- id: reference-connector-admin title: Connector Admin CLI -sidebar_label: Connector Admin CLI +sidebar_label: "Connector Admin CLI" original_id: reference-connector-admin --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-metrics.md b/site2/website-next/versioned_docs/version-2.7.3/reference-metrics.md index 1a91435eed466..89c327534c8a3 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-metrics.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-metrics.md @@ -1,7 +1,7 @@ --- id: reference-metrics title: Pulsar Metrics -sidebar_label: Pulsar Metrics +sidebar_label: "Pulsar Metrics" original_id: reference-metrics --- @@ -24,12 +24,13 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. + - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -97,19 +98,20 @@ All the metrics exposed by a broker are labelled with `cluster=${pulsar_cluster} The following metrics are available for broker: * [Namespace metrics](#namespace-metrics) - * [Replication metrics](#replication-metrics) + * [Replication metrics](#replication-metrics) * [Topic metrics](#topic-metrics) - * [Replication metrics](#replication-metrics-1) + * [Replication metrics](#replication-metrics-1) * [ManagedLedgerCache metrics](#managedledgercache-metrics) * [ManagedLedger metrics](#managedledger-metrics) * [LoadBalancing metrics](#loadbalancing-metrics) - * [BundleUnloading metrics](#bundleunloading-metrics) - * [BundleSplit metrics](#bundlesplit-metrics) + * [BundleUnloading metrics](#bundleunloading-metrics) + * [BundleSplit metrics](#bundlesplit-metrics) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) * [Token metrics](#token-metrics) -* [Authentication metrics](#authentication-metrics) +* [Authentication metrics](#authentication-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -261,7 +263,7 @@ All the cursor acknowledgment state metrics are labelled with the following labe Name |Type |Description |---|---|--- -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to a ledger.| +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:")|Gauge|The number of acknowledgment states that is persistent to a ledger.| brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to the ledger.| brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to ZooKeeper. brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to ZooKeeper. @@ -271,20 +273,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -294,7 +296,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -387,6 +389,35 @@ All the authentication metrics are labelled with the following labels: | pulsar_authentication_success_count| Counter | The number of successful authentication operations. | | pulsar_authentication_failures_count | Counter | The number of failing authentication operations. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-pulsar-admin.md b/site2/website-next/versioned_docs/version-2.7.3/reference-pulsar-admin.md index 317213a437db6..e5afbed6a80bb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-pulsar-admin.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-pulsar-admin.md @@ -1,7 +1,7 @@ --- id: pulsar-admin title: Pulsar admin CLI -sidebar_label: Pulsar Admin CLI +sidebar_label: "Pulsar Admin CLI" original_id: pulsar-admin --- @@ -573,6 +573,7 @@ Create a Pulsar Function in cluster mode (i.e. deploy it on a Pulsar cluster) Usage ``` + $ pulsar-admin functions create options ``` @@ -2127,6 +2128,7 @@ Usage $ pulsar-admin namespaces get-max-subscriptions-per-topic tenant/namespace ``` + ### `remove-max-subscriptions-per-topic` Remove the maximum subscription per topic for a namespace. @@ -2347,6 +2349,7 @@ Run compaction on the specified topic (persistent topics only) Usage ``` + $ pulsar-admin topics compact persistent://tenant/namespace/topic ``` @@ -2407,13 +2410,9 @@ Create a partitioned topic. A partitioned topic must be created before producers :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2459,13 +2458,9 @@ Creates a non-partitioned topic. A non-partitioned topic must explicitly be crea :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2522,6 +2517,7 @@ Get the list of topics under a namespace Usage ``` + $ pulsar-admin topics list tenant/cluster/namespace ``` @@ -2942,6 +2938,7 @@ Usage $ pulsar-admin topics remove-persistence tenant/namespace/topic ``` + ### `get-message-ttl` Get the message TTL for a topic. @@ -2976,7 +2973,7 @@ Usage ```bash -$ pulsar-admin topics remove-message-ttl tenant/namespace/topic +$ pulsar-admin topics remove-message-ttl tenant/namespace/topic ``` @@ -3018,6 +3015,7 @@ Usage $ pulsar-admin topics remove-deduplication tenant/namespace/topic ``` + ### `set-max-subscriptions` Set the maximum number of subscriptions for a topic. @@ -3300,7 +3298,7 @@ Usage ```bash -$ pulsar-admin topics set-compaction-threshold tenant/namespace/topic options +$ pulsar-admin topics set-compaction-threshold tenant/namespace/topic options ``` @@ -3551,6 +3549,7 @@ Operations related to Schemas associated with Pulsar topics. Usage ``` + $ pulsar-admin schemas subcommand ``` diff --git a/site2/website-next/versioned_docs/version-2.7.3/reference-terminology.md b/site2/website-next/versioned_docs/version-2.7.3/reference-terminology.md index 3eb54b8116b3d..cce3a3f6dbf93 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/reference-terminology.md +++ b/site2/website-next/versioned_docs/version-2.7.3/reference-terminology.md @@ -1,7 +1,7 @@ --- id: reference-terminology title: Pulsar Terminology -sidebar_label: Terminology +sidebar_label: "Terminology" original_id: reference-terminology --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/schema-evolution-compatibility.md b/site2/website-next/versioned_docs/version-2.7.3/schema-evolution-compatibility.md index f6c9b4cdf7140..6e08e164a05ae 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/schema-evolution-compatibility.md +++ b/site2/website-next/versioned_docs/version-2.7.3/schema-evolution-compatibility.md @@ -1,13 +1,14 @@ --- id: schema-evolution-compatibility title: Schema evolution and compatibility -sidebar_label: Schema evolution and compatibility +sidebar_label: "Schema evolution and compatibility" original_id: schema-evolution-compatibility --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + Normally, schemas do not stay the same over a long period of time. Instead, they undergo evolutions to satisfy new needs. This chapter examines how Pulsar schema evolves and what Pulsar schema compatibility check strategies are. @@ -36,9 +37,9 @@ For more information, see [Schema compatibility check strategy](#schema-compatib 1. When a producer/consumer/reader connects to a broker, the broker deploys the schema compatibility checker configured by `schemaRegistryCompatibilityCheckers` to enforce schema compatibility check. - The schema compatibility checker is one instance per schema type. - - Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. + The schema compatibility checker is one instance per schema type. + + Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. 2. The producer/consumer/reader sends its client `SchemaInfo` to the broker. @@ -54,328 +55,47 @@ Pulsar has 8 schema compatibility check strategies, which are summarized in the Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is the oldest and V3 is the latest: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Changes allowed - - - -Check against which schema - - - -Upgrade first - -
    -`ALWAYS_COMPATIBLE` - -Disable schema compatibility check. - -All changes are allowed - -All previous versions - -Any order -
    -`ALWAYS_INCOMPATIBLE` - -Disable schema evolution. - -All changes are disabled - -None - -None -
    -`BACKWARD` - -Consumers using the schema V3 can process data written by producers using the schema V3 or V2. - -* Add optional fields -* Delete fields - -Latest version - -Consumers -
    -`BACKWARD_TRANSITIVE` - -Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. - -* Add optional fields -* Delete fields - -All previous versions - -Consumers -
    -`FORWARD` - -Consumers using the schema V3 or V2 can process data written by producers using the schema V3. - -* Add fields -* Delete optional fields - -Latest version - -Producers -
    -`FORWARD_TRANSITIVE` - -Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. - -* Add fields -* Delete optional fields - -All previous versions - -Producers -
    -`FULL` - -Backward and forward compatible between the schema V3 and V2. - -* Modify optional fields - -Latest version - -Any order -
    -`FULL_TRANSITIVE` - -Backward and forward compatible among the schema V3, V2, and V1. - -* Modify optional fields - -All previous versions - -Any order -
    +| Compatibility check strategy | Definition | Changes allowed | Check against which schema | Upgrade first | +| --- | --- | --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | All changes are allowed | All previous versions | Any order | +| `ALWAYS_INCOMPATIBLE` | Disable schema evolution. | All changes are disabled | None | None | +| `BACKWARD` | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. |
  • Add optional fields
  • Delete fields
  • | Latest version | Consumers | +| `BACKWARD_TRANSITIVE` | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. |
  • Add optional fields
  • Delete fields
  • | All previous versions | Consumers | +| `FORWARD` | Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | Latest version | Producers | +| `FORWARD_TRANSITIVE` | Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | All previous versions | Producers | +| `FULL` | Backward and forward compatible between the schema V3 and V2. |
  • Modify optional fields
  • | Latest version | Any order | +| `FULL_TRANSITIVE` | Backward and forward compatible among the schema V3, V2, and V1. |
  • Modify optional fields
  • | All previous versions | Any order | ### ALWAYS_COMPATIBLE and ALWAYS_INCOMPATIBLE - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Note - -
    -`ALWAYS_COMPATIBLE` - -Disable schema compatibility check. - -None -
    -`ALWAYS_INCOMPATIBLE` - -Disable schema evolution, that is, any schema change is rejected. - -* For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. -* For Avro and JSON, the default schema compatibility check strategy is `FULL`. -
    +| Compatibility check strategy | Definition | Note | +| --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | None | +| `ALWAYS_INCOMPATIBLE` | Disable schema evolution, that is, any schema change is rejected. |
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • | #### Example * Example 1 - In some situations, an application needs to store events of several different types in the same Pulsar topic. + In some situations, an application needs to store events of several different types in the same Pulsar topic. - In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. + In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. - For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. + For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. - Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. + Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. * Example 2 - Sometimes we also make incompatible changes. + Sometimes we also make incompatible changes. - For example, you are modifying a field type from `string` to `int`. + For example, you are modifying a field type from `string` to `int`. - In this case, you need to: + In this case, you need to: - * Upgrade all producers and consumers to the new schema versions at the same time. + * Upgrade all producers and consumers to the new schema versions at the same time. - * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. + * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. ### BACKWARD and BACKWARD_TRANSITIVE @@ -390,15 +110,15 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t * Example 1 - Remove a field. - - A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. + Remove a field. + + A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. * Example 2 - You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. + You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. - Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. + Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. ### FORWARD and FORWARD_TRANSITIVE @@ -429,87 +149,10 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is the oldest and V3 is the latest: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Definition - - - -Description - - - -Note - -
    -`FULL` - -Schemas are both backward and forward compatible, which means: -Consumers using the last schema can process data written by producers using the new schema. -AND -Consumers using the new schema can process data written by producers using the last schema. - -Consumers using the schema V3 can process data written by producers using the schema V3 or V2. -AND -Consumers using the schema V3 or V2 can process data written by producers using the schema V3. - -* For Avro and JSON, the default schema compatibility check strategy is `FULL`. -* For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. -
    -`FULL_TRANSITIVE` - -The new schema is backward and forward compatible with all previously registered schemas. - -Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. -AND -Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. - -None -
    +| Compatibility check strategy | Definition | Description | Note | +| --- | --- | --- | --- | +| `FULL` | Schemas are both backward and forward compatible, which means: Consumers using the last schema can process data written by producers using the new schema. AND Consumers using the new schema can process data written by producers using the last schema. | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. AND Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • | +| `FULL_TRANSITIVE` | The new schema is backward and forward compatible with all previously registered schemas. | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. AND Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. | None | #### Example @@ -525,130 +168,23 @@ When a producer tries to connect to a topic (suppose ignore the schema auto crea * Check if the schema carried by the producer exists in the schema registry or not. - * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. - - * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. - + * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. + + * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. + ### Consumer When a consumer tries to connect to a topic, a broker checks if a carried schema is compatible with a registered schema based on the configured schema compatibility check strategy. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Check logic - -
    -`ALWAYS_COMPATIBLE` - -All pass -
    -`ALWAYS_INCOMPATIBLE` - -No pass -
    -`BACKWARD` - -Can read the last schema -
    -`BACKWARD_TRANSITIVE` - -Can read all schemas -
    -`FORWARD` - -Can read the last schema -
    -`FORWARD_TRANSITIVE` - -Can read the last schema -
    -`FULL` - -Can read the last schema -
    -`FULL_TRANSITIVE` - -Can read all schemas -
    +| Compatibility check strategy | Check logic | +| --- | --- | +| `ALWAYS_COMPATIBLE` | All pass | +| `ALWAYS_INCOMPATIBLE` | No pass | +| `BACKWARD` | Can read the last schema | +| `BACKWARD_TRANSITIVE` | Can read all schemas | +| `FORWARD` | Can read the last schema | +| `FORWARD_TRANSITIVE` | Can read the last schema | +| `FULL` | Can read the last schema | +| `FULL_TRANSITIVE` | Can read all schemas | ## Order of upgrading clients @@ -656,120 +192,13 @@ The order of upgrading client applications is determined by the compatibility ch For example, the producers using schemas to write data to Pulsar and the consumers using schemas to read data from Pulsar. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Compatibility check strategy - - - -Upgrade first - - - -Description - -
    -`ALWAYS_COMPATIBLE` - -Any order - -The compatibility check is disabled. -Consequently, you can upgrade the producers and consumers in **any order**. -
    -`ALWAYS_INCOMPATIBLE` - -None - -The schema evolution is disabled. -
    -* `BACKWARD` -* `BACKWARD_TRANSITIVE` - -Consumers - -There is no guarantee that consumers using the old schema can read data produced using the new schema. -Consequently, **upgrade all consumers first**, and then start producing new data. -
    -* `FORWARD` -* `FORWARD_TRANSITIVE` - -Producers - -There is no guarantee that consumers using the new schema can read data produced using the old schema. -Consequently, **upgrade all producers first** to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers. -
    -* `FULL` -* `FULL_TRANSITIVE` - -Any order - -There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. -Consequently, you can upgrade the producers and consumers in **any order**. -
    +| Compatibility check strategy | Upgrade first | Description | +| --- | --- | --- | +| `ALWAYS_COMPATIBLE` | Any order | The compatibility check is disabled. Consequently, you can upgrade the producers and consumers in **any order**. | +| `ALWAYS_INCOMPATIBLE` | None | The schema evolution is disabled. | +|
  • `BACKWARD`
  • `BACKWARD_TRANSITIVE`
  • | Consumers | There is no guarantee that consumers using the old schema can read data produced using the new schema. Consequently, **upgrade all consumers first**, and then start producing new data. | +|
  • `FORWARD`
  • `FORWARD_TRANSITIVE`
  • | Producers | There is no guarantee that consumers using the new schema can read data produced using the old schema. Consequently, **upgrade all producers first**
  • to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers.
  • | +|
  • `FULL`
  • `FULL_TRANSITIVE`
  • | Any order | There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. Consequently, you can upgrade the producers and consumers in **any order**. | diff --git a/site2/website-next/versioned_docs/version-2.7.3/schema-get-started.md b/site2/website-next/versioned_docs/version-2.7.3/schema-get-started.md index 85695d52c318b..97529fbb125ca 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/schema-get-started.md +++ b/site2/website-next/versioned_docs/version-2.7.3/schema-get-started.md @@ -1,13 +1,14 @@ --- id: schema-get-started title: Get started -sidebar_label: Get started +sidebar_label: "Get started" original_id: schema-get-started --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This chapter introduces Pulsar schemas and explains why they are important. ## Schema Registry @@ -20,7 +21,7 @@ Applications typically adopt one of the following approaches to guarantee type s #### Note > -> Currently, the Pulsar schema registry is only available for the [Java client](client-libraries-java.md), [CGo client](client-libraries-cgo.md), [Python client](client-libraries-python.md), and [C++ client](client-libraries-cpp.md). +> Currently, the Pulsar schema registry is only available for the [Java client](client-libraries-java.md), [CGo client](client-libraries-cgo.md), [Python client](client-libraries-python.md), and [C++ client](client-libraries-cpp). ### Client-side approach @@ -53,11 +54,14 @@ Pulsar schema enables you to use language-specific types of data when constructi **Example** You can use the _User_ class to define the messages sent to Pulsar topics. + ``` + public class User { String name; int age; } + ``` When constructing a producer with the _User_ class, you can specify a schema or not as below. @@ -67,14 +71,18 @@ When constructing a producer with the _User_ class, you can specify a schema or If you construct a producer without specifying a schema, then the producer can only produce messages of type `byte[]`. If you have a POJO class, you need to serialize the POJO into bytes before sending messages. **Example** + ``` + Producer producer = client.newProducer() .topic(topic) .create(); User user = new User("Tom", 28); byte[] message = … // serialize the `user` by yourself; producer.send(message); + ``` + ### With schema If you construct a producer with specifying a schema, then you can send a class to a topic directly without worrying about how to serialize POJOs into bytes. @@ -82,12 +90,15 @@ If you construct a producer with specifying a schema, then you can send a class **Example** This example constructs a producer with the _JSONSchema_, and you can send the _User_ class to topics directly without worrying about how to serialize it into bytes. + ``` + Producer producer = client.newProducer(JSONSchema.of(User.class)) .topic(topic) .create(); User user = new User("Tom", 28); producer.send(user); + ``` ### Summary diff --git a/site2/website-next/versioned_docs/version-2.7.3/schema-manage.md b/site2/website-next/versioned_docs/version-2.7.3/schema-manage.md index b264f5369572e..b8377c0e6f4fb 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/schema-manage.md +++ b/site2/website-next/versioned_docs/version-2.7.3/schema-manage.md @@ -1,13 +1,14 @@ --- id: schema-manage title: Manage schema -sidebar_label: Manage schema +sidebar_label: "Manage schema" original_id: schema-manage --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This guide demonstrates the ways to manage schemas: * Automatically @@ -34,25 +35,25 @@ For a producer, the `AutoUpdate` happens in the following cases: * If a **producer doesn’t carry a schema**: - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. - - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. + + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. * If a **producer carries a schema**: - A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. - - * If the schema is registered, a producer is connected to a broker. - - * If the schema is not registered: - - * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. - - * If `isAllowAutoUpdateSchema` sets to **true**: - - * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. - - * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. + A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. + + * If the schema is registered, a producer is connected to a broker. + + * If the schema is not registered: + + * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. + + * If `isAllowAutoUpdateSchema` sets to **true**: + + * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. + + * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. ![AutoUpdate Producer](/assets/schema-producer.png) @@ -64,18 +65,18 @@ For a consumer, the `AutoUpdate` happens in the following cases: * If a **consumer connects to a topic with a schema**. - * If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - - * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. - + * If a topic does not have all of them (a schema/data/a local consumer and a local producer): + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + + * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + ![AutoUpdate Consumer](/assets/schema-consumer.png) @@ -94,7 +95,9 @@ You can use the `pulsar-admin` command to manage the `AutoUpdate` strategy as be To enable `AutoUpdate` on a namespace, you can use the `pulsar-admin` command. ```bash + bin/pulsar-admin namespaces set-is-allow-auto-update-schema --enable tenant/namespace + ``` #### Disable AutoUpdate @@ -102,7 +105,9 @@ bin/pulsar-admin namespaces set-is-allow-auto-update-schema --enable tenant/name To disable `AutoUpdate` on a namespace, you can use the `pulsar-admin` command. ```bash + bin/pulsar-admin namespaces set-is-allow-auto-update-schema --disable tenant/namespace + ``` Once the `AutoUpdate` is disabled, you can only register a new schema using the `pulsar-admin` command. @@ -112,7 +117,9 @@ Once the `AutoUpdate` is disabled, you can only register a new schema using the To adjust the schema compatibility level on a namespace, you can use the `pulsar-admin` command. ```bash + bin/pulsar-admin namespaces set-schema-compatibility-strategy --compatibility tenant/namespace + ``` ### Schema validation @@ -130,7 +137,9 @@ However, if you want a stronger guarantee on the topics with schemas, you can en To enable `schemaValidationEnforced` on a namespace, you can use the `pulsar-admin` command. ```bash + bin/pulsar-admin namespaces set-schema-validation-enforce --enable tenant/namespace + ``` #### Disable schema validation @@ -138,45 +147,20 @@ bin/pulsar-admin namespaces set-schema-validation-enforce --enable tenant/namesp To disable `schemaValidationEnforced` on a namespace, you can use the `pulsar-admin` command. ```bash + bin/pulsar-admin namespaces set-schema-validation-enforce --disable tenant/namespace + ``` ## Schema manual management To manage schemas, you can use one of the following methods. - - - - - - - - - - - - - - - - - - - -
    MethodDescription
    -**Admin CLI** - -You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. -For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin.md). -
    -**REST API** - -Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. -For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). -
    -**Java Admin API** - Pulsar provides Java admin library.
    +| Method | Description | +| --- | --- | +| **Admin CLI**
  • | You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin). | +| **REST API**
  • | Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). | +| **Java Admin API**
  • | Pulsar provides Java admin library. | ### Upload a schema @@ -204,68 +188,49 @@ To upload (register) a new schema for a topic, you can use one of the following Use the `upload` subcommand. ```bash + $ pulsar-admin schemas upload --filename + ``` The `schema-definition-file` is in JSON format. ```json + { "type": "", "schema": "", "properties": {} // the properties associated with the schema } + ``` The `schema-definition-file` includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | Here are examples of the `schema-definition-file` for a JSON schema. **Example 1** ```json + { "type": "JSON", "schema": "{\"type\":\"record\",\"name\":\"User\",\"namespace\":\"com.foo\",\"fields\":[{\"name\":\"file1\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"file2\",\"type\":\"string\",\"default\":null},{\"name\":\"file3\",\"type\":[\"null\",\"string\"],\"default\":\"dfdf\"}]}", "properties": {} } + ``` **Example 2** ```json + { "type": "STRING", "schema": "", @@ -273,6 +238,7 @@ Here are examples of the `schema-definition-file` for a JSON schema. "key1": "value1" } } + ```
    @@ -283,99 +249,54 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v2/schema The post payload is in JSON format. ```json + { "type": "", "schema": "", "properties": {} // the properties associated with the schema } + ``` The post payload includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. |
    ```java + void createSchema(String topic, PostSchemaPayload schemaPayload) + ``` The `PostSchemaPayload` includes the following fields: - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`type` - - The schema type.
    -`schema` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `type` | The schema type. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `PostSchemaPayload`: ```java + PulsarAdmin admin = …; PostSchemaPayload payload = new PostSchemaPayload(); payload.setType("INT8"); payload.setSchema(""); -admin.createSchema("my-tenant/my-ns/my-topic", payload); +admin.createSchema("my-tenant/my-ns/my-topic", payload); + ``` +
    @@ -406,6 +327,7 @@ To get the latest schema for a topic, you can use one of the following methods. Use the `get` subcommand. ```bash + $ pulsar-admin schemas get { @@ -418,6 +340,7 @@ $ pulsar-admin schemas get "property2": "string" } } + ```
    @@ -428,6 +351,7 @@ Send a `GET` request to this endpoint: {@inject: endpoint|GET|/admin/v2/schemas/ Here is an example of a response, which is returned in JSON format. ```json + { "version": "", "type": "", @@ -435,110 +359,45 @@ Here is an example of a response, which is returned in JSON format. "data": "", "properties": {} // the properties associated with the schema } + ``` The response includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`version` - - The schema version, which is a long number.
    -`type` - - The schema type.
    -`timestamp` - - The timestamp of creating this version of schema.
    -`data` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `version` | The schema version, which is a long number. | +| `type` | The schema type. | +| `timestamp` | The timestamp of creating this version of schema. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. |
    ```java + SchemaInfo createSchema(String topic) + ``` The `SchemaInfo` includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`name` - - The schema name.
    -`type` - - The schema type.
    -`schema` - -A byte array of the schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this byte array should be empty. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `name` | The schema name. | +| `type` | The schema type. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: ```java + PulsarAdmin admin = …; -SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); +SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); + ```
    @@ -571,7 +430,9 @@ To get a specific version of a schema, you can use one of the following methods. Use the `get` subcommand. ```bash -$ pulsar-admin schemas get --version= + +$ pulsar-admin schemas get --version= + ```
    @@ -582,6 +443,7 @@ Send a `GET` request to a schema endpoint: {@inject: endpoint|GET|/admin/v2/sche Here is an example of a response, which is returned in JSON format. ```json + { "version": "", "type": "", @@ -589,108 +451,45 @@ Here is an example of a response, which is returned in JSON format. "data": "", "properties": {} // the properties associated with the schema } + ``` The response includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`version` - - The schema version, which is a long number.
    -`type` - - The schema type.
    -`timestamp` - - The timestamp of creating this version of schema.
    -`data` - -The schema definition data, which is encoded in UTF 8 charset. -* If the schema is a **primitive** schema, this field should be blank. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `version` | The schema version, which is a long number. | +| `type` | The schema type. | +| `timestamp` | The timestamp of creating this version of schema. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | +| `properties` | The additional properties associated with the schema. |
    ```java + SchemaInfo createSchema(String topic, long version) + ``` The `SchemaInfo` includes the following fields: - - - - - - - - - - - - - - - - - - - - - - - -
    FieldDescription
    -`name` - The schema name.
    -`type` - The schema type.
    -`schema` - -A byte array of the schema definition data, which is encoded in UTF 8. -* If the schema is a **primitive** schema, this byte array should be empty. -* If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. -
    - `properties` - The additional properties associated with the schema.
    +| Field | Description | +| --- | --- | +| `name` | The schema name. | +| `type` | The schema type. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | +| `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: ```java + PulsarAdmin admin = …; SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic", 1L); + ```
    @@ -715,7 +514,9 @@ To provide a schema via a topic, you can use the following method. Use the `extract` subcommand. ```bash + $ pulsar-admin schemas extract --classname --jar --type + ```
    @@ -726,9 +527,11 @@ $ pulsar-admin schemas extract --classname --jar --type To delete a schema for a topic, you can use one of the following methods. -> #### Note -> -> In any case, the **delete** action deletes **all versions** of a schema registered for a topic. +:::note + +In any case, the **delete** action deletes **all versions** of a schema registered for a topic. + +::: + ```
    @@ -763,9 +568,11 @@ Send a `DELETE` request to a schema endpoint: {@inject: endpoint|DELETE|/admin/v Here is an example of a response, which is returned in JSON format. ```json + { "version": "", } + ``` The response includes the following field: @@ -778,15 +585,19 @@ Field | Description | ```java + void deleteSchema(String topic) + ``` Here is an example of deleting a schema. ```java + PulsarAdmin admin = …; -admin.deleteSchema("my-tenant/my-ns/my-topic"); +admin.deleteSchema("my-tenant/my-ns/my-topic"); + ``` @@ -812,6 +623,7 @@ To use a non-default (non-BookKeeper) storage system for Pulsar schemas, you nee The `SchemaStorage` interface has the following methods: ```java + public interface SchemaStorage { // How schemas are updated CompletableFuture put(String key, byte[] value, byte[] hash); @@ -831,26 +643,33 @@ public interface SchemaStorage { // Shutdown behavior for the schema storage client void close() throws Exception; } + ``` -> #### Tip -> -> For a complete example of **schema storage** implementation, see [BookKeeperSchemaStorage](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. +:::tip + +For a complete example of **schema storage** implementation, see [BookKeeperSchemaStorage](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. + +::: #### SchemaStorageFactory interface The `SchemaStorageFactory` interface has the following method: ```java + public interface SchemaStorageFactory { @NotNull SchemaStorage create(PulsarService pulsar) throws Exception; } + ``` -> Tip -> -> For a complete example of **schema storage factory** implementation, see [BookKeeperSchemaStorageFactory](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. +:::tip + +For a complete example of **schema storage factory** implementation, see [BookKeeperSchemaStorageFactory](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. + +::: ### Deploy diff --git a/site2/website-next/versioned_docs/version-2.7.3/schema-understand.md b/site2/website-next/versioned_docs/version-2.7.3/schema-understand.md index 2a6ea7ca0f163..25fd65e9453b2 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/schema-understand.md +++ b/site2/website-next/versioned_docs/version-2.7.3/schema-understand.md @@ -1,13 +1,14 @@ --- id: schema-understand title: Understand schema -sidebar_label: Understand schema +sidebar_label: "Understand schema" original_id: schema-understand --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; + This chapter explains the basic concepts of Pulsar schema, focuses on the topics of particular importance, and provides additional background. ## SchemaInfo @@ -18,91 +19,26 @@ The `SchemaInfo` is stored and enforced on a per-topic basis and cannot be store A `SchemaInfo` consists of the following fields: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -Field - - - -Description - -
    -`name` - -Schema name (a string). -
    -`type` - -Schema type, which determines how to interpret the schema data. -* Predefined schema: see [here](schema-understand.md#schema-type). -* Customized schema: it is left as an empty string. -
    -`schema`(`payload`) - -Schema data, which is a sequence of 8-bit unsigned bytes and schema-type specific. -
    -`properties` - -It is a user defined properties as a string/string map. -Applications can use this bag for carrying any application specific logics. -Possible properties might be the Git hash associated with the schema, an environment string like `dev` or `prod`. -
    +| Field | Description | +| --- | --- | +| `name` | Schema name (a string). | +| `type` | Schema type, which determines how to interpret the schema data.
  • Predefined schema: see [here](schema-understand.md#schema-type).
  • Customized schema: it is left as an empty string.
  • | +| `schema`(`payload`) | Schema data, which is a sequence of 8-bit unsigned bytes and schema-type specific. | +| `properties` | It is a user defined properties as a string/string map. Applications can use this bag for carrying any application specific logics. Possible properties might be the Git hash associated with the schema, an environment string like `dev` or `prod`. | **Example** This is the `SchemaInfo` of a string. ```json + { "name": "test-string-schema", "type": "STRING", "schema": "", "properties": {} } + ``` ## Schema type @@ -165,17 +101,21 @@ This example demonstrates how to use a string schema. 1. Create a producer with a string schema and send messages. -```java - Producer producer = client.newProducer(Schema.STRING).create(); - producer.newMessage().value("Hello Pulsar!").send(); -``` + ```java + + Producer producer = client.newProducer(Schema.STRING).create(); + producer.newMessage().value("Hello Pulsar!").send(); + + ``` 2. Create a consumer with a string schema and receive messages. -```java - Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); - consumer.receive(); -``` + ```java + + Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); + consumer.receive(); + + ``` ### Complex type @@ -214,64 +154,72 @@ This example shows how to construct a key/value schema and then use it to produc 1. Construct a key/value schema with `INLINE` encoding type. -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.INLINE - ); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.INLINE + ); + + ``` 2. Optionally, construct a key/value schema with `SEPARATED` encoding type. -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + ``` 3. Produce messages using a key/value schema. -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Producer> producer = client.newProducer(kvSchema) - .topic(TOPIC) - .create(); - - final int key = 100; - final String value = "value-100"; - - // send the key/value message - producer.newMessage() - .value(new KeyValue<>(key, value)) - .send(); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Producer> producer = client.newProducer(kvSchema) + .topic(TOPIC) + .create(); + + final int key = 100; + final String value = "value-100"; + + // send the key/value message + producer.newMessage() + .value(new KeyValue(key, value)) + .send(); + + ``` 4. Consume messages using a key/value schema. -```java - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Consumer> consumer = client.newConsumer(kvSchema) - ... - .topic(TOPIC) - .subscriptionName(SubscriptionName).subscribe(); - - // receive key/value pair - Message> msg = consumer.receive(); - KeyValue kv = msg.getValue(); -``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Consumer> consumer = client.newConsumer(kvSchema) + ... + .topic(TOPIC) + .subscriptionName(SubscriptionName).subscribe(); + + // receive key/value pair + Message> msg = consumer.receive(); + KeyValue kv = msg.getValue(); + + ``` #### struct @@ -299,26 +247,32 @@ Pulsar gets the schema definition from the predefined `struct` using an Avro lib 1. Create the _User_ class to define the messages sent to Pulsar topics. -```java - public class User { - String name; - int age; - } -``` + ```java + + public class User { + String name; + int age; + } + + ``` 2. Create a producer with a `struct` schema and send messages. -```java - Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); - producer.newMessage().value(User.builder().userName("pulsar-user").userId(1L).build()).send(); -``` + ```java + + Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); + producer.newMessage().value(User.builder().userName("pulsar-user").userId(1L).build()).send(); + + ``` 3. Create a consumer with a `struct` schema and receive messages -```java - Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); - User user = consumer.receive(); -``` + ```java + + Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); + User user = consumer.receive(); + + ``` ##### generic @@ -330,21 +284,25 @@ You can define the `struct` schema using the `GenericSchemaBuilder`, generate a 1. Use `RecordSchemaBuilder` to build a schema. -```java - RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); - recordSchemaBuilder.field("intField").type(SchemaType.INT32); - SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); + ```java + + RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); + recordSchemaBuilder.field("intField").type(SchemaType.INT32); + SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); -``` + Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + + ``` 2. Use `RecordBuilder` to build the struct records. -```java - producer.newMessage().value(schema.newRecordBuilder() - .set("intField", 32) - .build()).send(); -``` + ```java + + producer.newMessage().value(schema.newRecordBuilder() + .set("intField", 32) + .build()).send(); + + ``` ### Auto Schema @@ -372,6 +330,7 @@ Suppose that: In this case, you can use `AUTO_PRODUCE` to verify whether the bytes produced by _K_ can be sent to _P_ or not. ```java + Produce pulsarProducer = client.newProducer(Schema.AUTO_PRODUCE()) … .create(); @@ -379,6 +338,7 @@ Produce pulsarProducer = client.newProducer(Schema.AUTO_PRODUCE()) byte[] kafkaMessageBytes = … ; pulsarProducer.produce(kafkaMessageBytes); + ``` #### AUTO_CONSUME @@ -400,12 +360,14 @@ Suppose that: In this case, you can use `AUTO_CONSUME` to verify whether the bytes produced by _P_ can be sent to MySQL or not. ```java + Consumer pulsarConsumer = client.newConsumer(Schema.AUTO_CONSUME()) … .subscribe(); Message msg = consumer.receive() ; GenericRecord record = msg.getValue(); + ``` ## Schema version @@ -422,9 +384,10 @@ Once a version is assigned/fetched to/for a schema, all subsequent messages prod The following example illustrates how the schema version works. -Suppose that a Pulsar [Java client](client-libraries-java.md) created using the code below attempts to connect to Pulsar and begins to send messages: +Suppose that a Pulsar [Java client](client-libraries-java) created using the code below attempts to connect to Pulsar and begins to send messages: ```java + PulsarClient client = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .build(); @@ -433,65 +396,15 @@ Producer producer = client.newProducer(JSONSchema.of(SensorReadin .topic("sensor-data") .sendTimeout(3, TimeUnit.SECONDS) .create(); + ``` The table below lists the possible scenarios when this connection attempt occurs and what happens in each scenario: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    ScenarioWhat happens
    -* No schema exists for the topic. - -(1) The producer is created using the given schema. -(2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. -(3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. -
    -* A schema already exists. -* The producer connects using the same schema that is already stored. - -(1) The schema is transmitted to the broker. -(2) The broker determines that the schema is compatible. -(3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. -
    -* A schema already exists. -* The producer connects using a new schema that is compatible. - -(1) The schema is transmitted to the broker. -(2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). -
    +| Scenario | What happens | +| --- | --- | +|
  • No schema exists for the topic.
  • | (1) The producer is created using the given schema. (2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. (3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. | +|
  • A schema already exists.
  • The producer connects using the same schema that is already stored.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible. (3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. |
  • A schema already exists.
  • The producer connects using a new schema that is compatible.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). | ## How does schema work @@ -507,9 +420,9 @@ This diagram illustrates how does schema work on the Producer side. 1. The application uses a schema instance to construct a producer instance. - The schema instance defines the schema for the data being produced using the producer instance. + The schema instance defines the schema for the data being produced using the producer instance. - Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. + Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. 2. The producer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. @@ -533,7 +446,7 @@ For how to set `isAllowAutoUpdateSchema` via Pulsar admin API, see [Manage AutoU * If the schema is compatible, the broker stores it and returns the schema version to the producer. - All the messages produced by this producer are tagged with the schema version. + All the messages produced by this producer are tagged with the schema version. * If the schema is incompatible, the broker rejects it. @@ -545,24 +458,24 @@ This diagram illustrates how does Schema work on the consumer side. 1. The application uses a schema instance to construct a consumer instance. - The schema instance defines the schema that the consumer uses for decoding messages received from a broker. + The schema instance defines the schema that the consumer uses for decoding messages received from a broker. 2. The consumer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. 3. The broker determines whether the topic has one of them (a schema/data/a local consumer and a local producer). 4. If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + 5. If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. 6. The consumer receives messages from the broker. - If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. + If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-athenz.md b/site2/website-next/versioned_docs/version-2.7.3/security-athenz.md index 8241dd7eb17b2..c4f79cb7f229c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-athenz.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-athenz.md @@ -1,7 +1,7 @@ --- id: security-athenz title: Authentication using Athenz -sidebar_label: Authentication using Athenz +sidebar_label: "Authentication using Athenz" original_id: security-athenz --- @@ -13,7 +13,7 @@ import TabItem from '@theme/TabItem'; ## Athenz authentication settings -A [decentralized Athenz system](https://github.com/AthenZ/athenz/blob/master/docs/decent_authz_flow.md) contains an [authori**Z**ation **M**anagement **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zms.md) (ZMS) server and an [authori**Z**ation **T**oken **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zts.md) (ZTS) server. +A [decentralized Athenz system](https://github.com/AthenZ/athenz/blob/master/docs/decent_authz_flow.md) contains an [authori**Z**ation **M**anagement **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zms.md) (ZMS) server and an [authori**Z**ation **T**oken **S**ystem](https://github.com/AthenZ/athenz/blob/master/docs/setup_zts) (ZTS) server. To begin, you need to set up Athenz service access control. You need to create domains for the *provider* (which provides some resources to other services with some authentication/authorization policies) and the *tenant* (which is provisioned to access some resources in a provider). In this case, the provider corresponds to the Pulsar service itself and the tenant corresponds to each application using Pulsar (typically, a [tenant](reference-terminology.md#tenant) in Pulsar). @@ -46,7 +46,7 @@ For more specific steps involving UI, refer to [Example Service Access Control S > ### TLS encryption > > Note that when you are using Athenz as an authentication provider, you had better use TLS encryption -> as it can protect role tokens from being intercepted and reused. (for more details involving TLS encryption see [Architecture - Data Model](https://github.com/AthenZ/athenz/blob/master/docs/data_model.md)). +> as it can protect role tokens from being intercepted and reused. (for more details involving TLS encryption see [Architecture - Data Model](https://github.com/AthenZ/athenz/blob/master/docs/data_model)). In the `conf/broker.conf` configuration file in your Pulsar installation, you need to provide the class name of the Athenz authentication provider as well as a comma-separated list of provider domain names. @@ -80,7 +80,7 @@ For more information on Pulsar client authentication using Athenz, see the follo ## Configure CLI tools for Athenz -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following authentication parameters to the `conf/client.conf` config file to use Athenz with CLI tools of Pulsar: @@ -99,3 +99,4 @@ tlsAllowInsecureConnection=false tlsTrustCertsFilePath=/path/to/cacert.pem ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-authorization.md b/site2/website-next/versioned_docs/version-2.7.3/security-authorization.md index 53ab9d4657f09..9ca30ea1dd999 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-authorization.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-authorization.md @@ -1,7 +1,7 @@ --- id: security-authorization title: Authentication and authorization in Pulsar -sidebar_label: Authorization and ACLs +sidebar_label: "Authorization and ACLs" original_id: security-authorization --- @@ -31,7 +31,7 @@ superUserRoles=my-super-user-1,my-super-user-2 > A full list of parameters is available in the `conf/broker.conf` file. > You can also find the default values for those parameters in [Broker Configuration](reference-configuration.md#broker). -Typically, you use superuser roles for administrators, clients as well as broker-to-broker authorization. When you use [geo-replication](concepts-replication.md), every broker needs to be able to publish to all the other topics of clusters. +Typically, you use superuser roles for administrators, clients as well as broker-to-broker authorization. When you use [geo-replication](concepts-replication), every broker needs to be able to publish to all the other topics of clusters. You can also enable the authorization for the proxy in the proxy configuration file (`conf/proxy.conf`). Once you enable the authorization on the proxy, the proxy does an additional authorization check before forwarding the request to a broker. If you enable authorization on the broker, the broker checks the authorization of the request when the broker receives the forwarded request. @@ -50,7 +50,7 @@ Another approach is to make the proxy role a superuser. This allows the proxy to You can specify the roles as proxy roles in [`conf/broker.conf`](reference-configuration.md#broker). -```properties +``` proxyRoles=my-proxy-role @@ -63,13 +63,13 @@ superUserRoles=my-super-user-1,my-super-user-2,my-proxy-role Pulsar [instance](reference-terminology.md#instance) administrators or some kind of self-service portal typically provisions a Pulsar [tenant](reference-terminology.md#tenant). -You can manage tenants using the [`pulsar-admin`](reference-pulsar-admin.md) tool. +You can manage tenants using the [`pulsar-admin`](reference-pulsar-admin) tool. ### Create a new tenant The following is an example tenant creation command: -```shell +``` $ bin/pulsar-admin tenants create my-tenant \ --admin-roles my-admin-role \ @@ -83,7 +83,7 @@ A client that successfully identifies itself as having the role `my-admin-role` The structure of topic names in Pulsar reflects the hierarchy between tenants, clusters, and namespaces: -```shell +``` persistent://tenant/namespace/topic @@ -91,11 +91,11 @@ persistent://tenant/namespace/topic ### Manage permissions -You can use [Pulsar Admin Tools](admin-api-permissions.md) for managing permission in Pulsar. +You can use [Pulsar Admin Tools](admin-api-permissions) for managing permission in Pulsar. ### Pulsar admin authentication -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("http://broker:8080") @@ -106,7 +106,7 @@ PulsarAdmin admin = PulsarAdmin.builder() To use TLS: -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("https://broker:8080") @@ -115,3 +115,4 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-bouncy-castle.md b/site2/website-next/versioned_docs/version-2.7.3/security-bouncy-castle.md index fa2608d0ccf06..b9bb8155bab83 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-bouncy-castle.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-bouncy-castle.md @@ -1,7 +1,7 @@ --- id: security-bouncy-castle title: Bouncy Castle Providers -sidebar_label: Bouncy Castle Providers +sidebar_label: "Bouncy Castle Providers" original_id: security-bouncy-castle --- @@ -20,7 +20,7 @@ In Pulsar, security and crypto have dependencies on BouncyCastle Jars. For the d `Bouncy Castle` provides both [FIPS](https://www.bouncycastle.org/fips_faq.html) and non-FIPS version. But in a JVM, you can not include both of the 2 versions, and you need to exclude the current version before include the other. -In Pulsar, the security and crypto methods also depends on `Bouncy Castle`, especially in [TLS Authentication](security-tls-authentication.md) and [Transport Encryption](security-encryption.md). This document contains the configuration between BouncyCastle FIPS(BC-FIPS) and non-FIPS(BC-non-FIPS) version while using Pulsar. +In Pulsar, the security and crypto methods also depends on `Bouncy Castle`, especially in [TLS Authentication](security-tls-authentication.md) and [Transport Encryption](security-encryption). This document contains the configuration between BouncyCastle FIPS(BC-FIPS) and non-FIPS(BC-non-FIPS) version while using Pulsar. ## How BouncyCastle modules packaged in Pulsar @@ -35,11 +35,13 @@ Usually, You will meet error like `java.lang.SecurityException: Invalid signatur You could exclude these signatures in mvn pom file to avoid above error, by ```access transformers + META-INF/*.SF META-INF/*.DSA META-INF/*.RSA ``` + But it can also lead to new, cryptic errors, e.g. `java.security.NoSuchAlgorithmException: PBEWithSHA256And256BitAES-CBC-BC SecretKeyFactory not available` By explicitly specifying where to find the algorithm like this: `SecretKeyFactory.getInstance("PBEWithSHA256And256BitAES-CBC-BC","BC")` It will get the real error: `java.security.NoSuchProviderException: JCE cannot authenticate the provider BC` @@ -153,6 +155,7 @@ If you want to switch from BC-non-FIPS to BC-FIPS version, Here is an example fo ``` + For more example, you can reference module `bcfips-include-test`. diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-encryption.md b/site2/website-next/versioned_docs/version-2.7.3/security-encryption.md index 7e45c08991360..b0f2f4616b649 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-encryption.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-encryption.md @@ -1,7 +1,7 @@ --- id: security-encryption title: Pulsar Encryption -sidebar_label: End-to-End Encryption +sidebar_label: "End-to-End Encryption" original_id: security-encryption --- @@ -103,6 +103,7 @@ producer.close(); pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -180,6 +181,7 @@ If you need to encrypt the messages using 2 keys (myapp.messagekey1 and myapp.me PulsarClient.newProducer().addEncryptionKey("myapp.messagekey1").addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypt encrypted messages at the consumer application Consumers require access one of the private keys to decrypt messages that the producer produces. If you want to receive encrypted messages, create a public or private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-extending.md b/site2/website-next/versioned_docs/version-2.7.3/security-extending.md index 3c89b51d51679..57128da01c25d 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-extending.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-extending.md @@ -1,7 +1,7 @@ --- id: security-extending title: Extending Authentication and Authorization in Pulsar -sidebar_label: Extending +sidebar_label: "Extending" original_id: security-extending --- @@ -13,7 +13,7 @@ Pulsar provides a way to use custom authentication and authorization mechanisms. ## Authentication -Pulsar supports mutual TLS and Athenz authentication plugins. For how to use these authentication plugins, you can refer to the description in [Security](security-overview.md). +Pulsar supports mutual TLS and Athenz authentication plugins. For how to use these authentication plugins, you can refer to the description in [Security](security-overview). You can use a custom authentication mechanism by providing the implementation in the form of two plugins. One plugin is for the Client library and the other plugin is for the Pulsar Proxy and/or Pulsar Broker to validate the credentials. @@ -55,6 +55,7 @@ In `conf/broker.conf` you can choose to specify a list of valid providers: authenticationProviders= ``` + To implement `org.apache.pulsar.broker.authentication.AuthenticationProvider` on one single interface: ```java @@ -110,10 +111,10 @@ the Authorization plugin is designed only for use on the Broker however the Prox To provide a custom provider, you need to implement the `org.apache.pulsar.broker.authorization.AuthorizationProvider` interface, put this class in the Pulsar broker classpath and configure the class in `conf/broker.conf`: ```properties - + # Authorization provider fully qualified class-name authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - + ``` ```java @@ -182,8 +183,8 @@ public interface AuthorizationProvider extends Closeable { * @param authDataJson * additional authdata in json format * @return CompletableFuture - * @completesWith
    - * IllegalArgumentException when namespace not found
    + * @completesWith
    + * IllegalArgumentException when namespace not found
    * IllegalStateException when failed to grant permission */ CompletableFuture grantPermissionAsync(NamespaceName namespace, Set actions, String role, @@ -197,8 +198,8 @@ public interface AuthorizationProvider extends Closeable { * @param authDataJson * additional authdata in json format * @return CompletableFuture - * @completesWith
    - * IllegalArgumentException when namespace not found
    + * @completesWith
    + * IllegalArgumentException when namespace not found
    * IllegalStateException when failed to grant permission */ CompletableFuture grantPermissionAsync(TopicName topicName, Set actions, String role, @@ -207,3 +208,4 @@ public interface AuthorizationProvider extends Closeable { } ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-jwt.md b/site2/website-next/versioned_docs/version-2.7.3/security-jwt.md index 459f2197d05fe..006c6642c8b36 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-jwt.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-jwt.md @@ -1,7 +1,7 @@ --- id: security-jwt title: Client authentication using tokens based on JSON Web Tokens -sidebar_label: Authentication using JWT +sidebar_label: "Authentication using JWT" original_id: security-jwt --- @@ -21,6 +21,7 @@ A user typically gets a token string from the administrator (or some automated s The compact representation of a signed JWT is a string that looks like as the following: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY ``` @@ -29,11 +30,11 @@ Application specifies the token when you create the client instance. An alternat > #### Always use TLS transport encryption > Sending a token is equivalent to sending a password over the wire. You had better use TLS encryption all the time when you connect to the Pulsar service. See -> [Transport Encryption using TLS](security-tls-transport.md) for more details. +> [Transport Encryption using TLS](security-tls-transport) for more details. ### CLI Tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use the token authentication with CLI tools of Pulsar: @@ -49,6 +50,7 @@ authParams=token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPt The token string can also be read from a file, for example: ``` + authParams=file:///path/to/token/file ``` @@ -144,6 +146,7 @@ client, err := NewClient(ClientOptions{ }) ``` + Similarly, you can also pass a `Supplier`: ```go @@ -162,6 +165,7 @@ client, err := NewClient(ClientOptions{ ```c++ + #include pulsar::ClientConfiguration config; @@ -175,6 +179,7 @@ pulsar::Client client("pulsar://broker.example.com:6650/", config); ```c# + var client = PulsarClient.Builder() .AuthenticateUsingToken("eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY") .Build(); @@ -315,7 +320,7 @@ tokenSecretKey=file:///path/to/secret.key To configure proxies to authenticate clients, add the following parameters to `proxy.conf`: -The proxy uses its own token when connecting to brokers. You need to configure the role token for this key pair in the `proxyRoles` of the brokers. For more details, see the [authorization guide](security-authorization.md). +The proxy uses its own token when connecting to brokers. You need to configure the role token for this key pair in the `proxyRoles` of the brokers. For more details, see the [authorization guide](security-authorization). ```properties @@ -336,3 +341,4 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 forwardAuthorizationCredentials=true ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-kerberos.md b/site2/website-next/versioned_docs/version-2.7.3/security-kerberos.md index 6bdf8bb1c53e0..32e12f1b6988f 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-kerberos.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-kerberos.md @@ -1,7 +1,7 @@ --- id: security-kerberos title: Authentication using Kerberos -sidebar_label: Authentication using Kerberos +sidebar_label: "Authentication using Kerberos" original_id: security-kerberos --- @@ -56,11 +56,13 @@ You need to enter the command below to specify the path to the `krb5.conf` file -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + Here is an example of the krb5.conf file: In the configuration file, `EXAMPLE.COM` is the default realm; `kdc = localhost:62037` is the kdc server url for realm `EXAMPLE.COM `: ``` + [libdefaults] default_realm = EXAMPLE.COM @@ -78,6 +80,7 @@ Usually machines configured with kerberos already have a system wide configurati You need JAAS configuration file for the client side and the broker side. JAAS configuration file provides the section of information that is used to connect KDC. Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -102,7 +105,7 @@ You need to set the `JAAS` configuration file path as JVM parameter for client a ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf ``` @@ -110,7 +113,7 @@ In the `pulsar_jaas.conf` file above 1. `PulsarBroker` is a section name in the JAAS file that each broker uses. This section tells the broker to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarBroker` allows the broker to use the keytab specified in this section. 2. `PulsarClient` is a section name in the JASS file that each broker uses. This section tells the client to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarClient` allows the client to use the keytab specified in this section. - The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. + The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. You can have 2 separate JAAS configuration files: * the file for a broker that has sections of both `PulsarBroker` and `PulsarClient`; @@ -135,6 +138,7 @@ You can have 2 separate JAAS configuration files: Here is an example: ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -152,9 +156,10 @@ brokerClientAuthenticationParameters={"saslJaasClientSectionName":"PulsarClient" ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + You can add this at the end of `PULSAR_EXTRA_OPTS` in the file [`pulsar_env.sh`](https://github.com/apache/pulsar/blob/master/conf/pulsar_env.sh) You must ensure that the operating system user who starts broker can reach the keytabs configured in the `pulsar_jaas.conf` file and kdc server in the `krb5.conf` file. @@ -166,6 +171,7 @@ You must ensure that the operating system user who starts broker can reach the k In client application, include `pulsar-client-auth-sasl` in your project dependency. ``` + org.apache.pulsar pulsar-client-auth-sasl @@ -183,10 +189,9 @@ You need 2 parameters: When you authenticate between client and broker with the setting in above JAAS configuration file, we need to set `saslJaasClientSectionName` to `PulsarClient` and set `serverType` to `broker`. The following is an example of creating a Java client: - ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -201,12 +206,13 @@ The following is an example of creating a Java client: .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -230,7 +236,7 @@ Step 2. Enter the command below to set JVM parameters for JAAS configuration fil ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` @@ -272,6 +278,7 @@ In comparison with the above configuration, add a new section for Pulsar Proxy i Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -306,7 +313,7 @@ Here is an example named `pulsar_jaas.conf`: Pulsar client configuration is similar with client and broker configuration, except that you need to set `serverType` to `proxy` instead of `broker`, for the reason that you need to do the Kerberos authentication between client and proxy. ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -321,12 +328,13 @@ Pulsar client configuration is similar with client and broker configuration, exc .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -359,6 +367,7 @@ The second part relates to authenticating between Pulsar Proxy and Pulsar Broker The broker side configuration file is the same with the above `broker.conf`, you do not need special configuration for Pulsar Proxy. ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -368,7 +377,7 @@ saslJaasBrokerSectionName=PulsarBroker ## Regarding authorization and role token -For Kerberos authentication, we usually use the authenticated principal as the role token for Pulsar authorization. For more information of authorization in Pulsar, see [security authorization](security-authorization.md). +For Kerberos authentication, we usually use the authenticated principal as the role token for Pulsar authorization. For more information of authorization in Pulsar, see [security authorization](security-authorization). If you enable 'authorizationEnabled', you need to set `superUserRoles` in `broker.conf` that corresponds to the name registered in kdc. @@ -385,6 +394,7 @@ superUserRoles=client/{clientIp}@EXAMPLE.COM Pulsar Broker acts as a Kerberos client when you authenticate with Zookeeper. According to [ZooKeeper document](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Client-Server+mutual+authentication), you need these settings in `conf/zookeeper.conf`: ``` + authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider requireClientAuthScheme=sasl @@ -393,6 +403,7 @@ requireClientAuthScheme=sasl Enter the following commands to add a section of `Client` configurations in the file `pulsar_jaas.conf`, which Pulsar Broker uses: ``` + Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -411,6 +422,7 @@ In this setting, the principal of Pulsar Broker and keyTab file indicates the ro Pulsar Broker acts as a Kerberos client when you authenticate with Bookie. According to [BookKeeper document](http://bookkeeper.apache.org/docs/latest/security/sasl/), you need to add `bookkeeperClientAuthenticationPlugin` parameter in `broker.conf`: ``` + bookkeeperClientAuthenticationPlugin=org.apache.bookkeeper.sasl.SASLClientProviderFactory ``` @@ -420,6 +432,7 @@ In this setting, `SASLClientProviderFactory` creates a BookKeeper SASL client in Enter the following commands to add a section of `BookKeeper` configurations in the `pulsar_jaas.conf` that Pulsar Broker uses: ``` + BookKeeper { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-oauth2.md b/site2/website-next/versioned_docs/version-2.7.3/security-oauth2.md index dff0944c747e3..9060ed314a2c6 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-oauth2.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-oauth2.md @@ -1,7 +1,7 @@ --- id: security-oauth2 title: Client authentication using OAuth 2.0 access tokens -sidebar_label: Authentication using OAuth 2.0 access tokens +sidebar_label: "Authentication using OAuth 2.0 access tokens" original_id: security-oauth2 --- @@ -114,6 +114,7 @@ PulsarClient client = PulsarClient.builder() The C++ client is similar to the Java client. You need to provide parameters of `issuerUrl`, `private_key` (the credentials file path), and the audience. ```c++ + #include pulsar::ClientConfiguration config; @@ -179,6 +180,7 @@ This section describes how to use Pulsar CLI tools to connect a cluster through This example shows how to use pulsar-admin to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-admin --admin-url https://streamnative.cloud:443 \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", @@ -196,6 +198,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-client to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-client \ --url SERVICE_URL \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ @@ -214,6 +217,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-perf to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-perf produce --service-url pulsar+ssl://streamnative.cloud:6651 \ --auth_plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-overview.md b/site2/website-next/versioned_docs/version-2.7.3/security-overview.md index 174662e61784a..989c08359bb3c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-overview.md @@ -1,7 +1,7 @@ --- id: security-overview title: Pulsar security overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: security-overview --- @@ -25,15 +25,15 @@ You had better secure the service components in your Apache Pulsar deployment. In Pulsar, a *role* is a string, like `admin` or `app1`, which can represent a single client or multiple clients. You can use roles to control permission for clients to produce or consume from certain topics, administer the configuration for tenants, and so on. -Apache Pulsar uses a [Authentication Provider](#authentication-providers) to establish the identity of a client and then assign a *role token* to that client. This role token is then used for [Authorization and ACLs](security-authorization.md) to determine what the client is authorized to do. +Apache Pulsar uses a [Authentication Provider](#authentication-providers) to establish the identity of a client and then assign a *role token* to that client. This role token is then used for [Authorization and ACLs](security-authorization) to determine what the client is authorized to do. ## Authentication providers Currently Pulsar supports the following authentication providers: -- [TLS Authentication](security-tls-authentication.md) -- [Athenz](security-athenz.md) -- [Kerberos](security-kerberos.md) -- [JSON Web Token Authentication](security-jwt.md) +- [TLS Authentication](security-tls-authentication) +- [Athenz](security-athenz) +- [Kerberos](security-kerberos) +- [JSON Web Token Authentication](security-jwt) diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-tls-authentication.md b/site2/website-next/versioned_docs/version-2.7.3/security-tls-authentication.md index 4416e9f238d3c..c1fe1d8fb2295 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-tls-authentication.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-tls-authentication.md @@ -1,7 +1,7 @@ --- id: security-tls-authentication title: Authentication using TLS -sidebar_label: Authentication using TLS +sidebar_label: "Authentication using TLS" original_id: security-tls-authentication --- @@ -11,9 +11,9 @@ import TabItem from '@theme/TabItem'; ## TLS authentication overview -TLS authentication is an extension of [TLS transport encryption](security-tls-transport.md). Not only servers have keys and certs that the client uses to verify the identity of servers, clients also have keys and certs that the server uses to verify the identity of clients. You must have TLS transport encryption configured on your cluster before you can use TLS authentication. This guide assumes you already have TLS transport encryption configured. +TLS authentication is an extension of [TLS transport encryption](security-tls-transport). Not only servers have keys and certs that the client uses to verify the identity of servers, clients also have keys and certs that the server uses to verify the identity of clients. You must have TLS transport encryption configured on your cluster before you can use TLS authentication. This guide assumes you already have TLS transport encryption configured. -`Bouncy Castle Provider` provides TLS related cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle.md). +`Bouncy Castle Provider` provides TLS related cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle). ### Create client certificates @@ -46,6 +46,7 @@ $ openssl req -config openssl.cnf \ -key admin.key.pem -new -sha256 -out admin.csr.pem ``` + :::note If openssl.cnf is not specified, read [Certificate authority](http://pulsar.apache.org/docs/en/security-tls-transport/#certificate-authority) to get the openssl.cnf. @@ -104,7 +105,7 @@ brokerClientTrustCertsFilePath=/path/my-ca/certs/ca.cert.pem To configure proxies to authenticate clients, add the following parameters to `proxy.conf`, alongside [the configuration to enable tls transport](security-tls-transport.md#proxy-configuration): -The proxy should have its own client key pair for connecting to brokers. You need to configure the role token for this key pair in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization.md) for more details. +The proxy should have its own client key pair for connecting to brokers. You need to configure the role token for this key pair in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization) for more details. ```properties @@ -124,11 +125,11 @@ When you use TLS authentication, client connects via TLS transport. You need to ### CLI tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin.md), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use TLS authentication with the CLI tools of Pulsar: -```properties +``` webServiceUrl=https://broker.example.com:8443/ brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ @@ -142,7 +143,7 @@ authParams=tlsCertFile:/path/to/my-role.cert.pem,tlsKeyFile:/path/to/my-role.key ### Java client -```java +``` import org.apache.pulsar.client.api.PulsarClient; @@ -158,7 +159,7 @@ PulsarClient client = PulsarClient.builder() ### Python client -```python +``` from pulsar import Client, AuthenticationTLS @@ -172,7 +173,8 @@ client = Client("pulsar+ssl://broker.example.com:6651/", ### C++ client -```c++ +``` + #include pulsar::ClientConfiguration config; @@ -190,7 +192,7 @@ pulsar::Client client("pulsar+ssl://broker.example.com:6651/", config); ### Node.js client -```JavaScript +``` const Pulsar = require('pulsar-client'); @@ -211,10 +213,12 @@ const Pulsar = require('pulsar-client'); ### C# client -```c# +``` + var clientCertificate = new X509Certificate2("admin.pfx"); var client = PulsarClient.Builder() .AuthenticateUsingClientCertificate(clientCertificate) .Build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-tls-keystore.md b/site2/website-next/versioned_docs/version-2.7.3/security-tls-keystore.md index addf0e1d00501..94f20d6dcb730 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-tls-keystore.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-tls-keystore.md @@ -1,7 +1,7 @@ --- id: security-tls-keystore title: Using TLS with KeyStore configure -sidebar_label: Using TLS with KeyStore configure +sidebar_label: "Using TLS with KeyStore configure" original_id: security-tls-keystore --- @@ -11,7 +11,7 @@ import TabItem from '@theme/TabItem'; ## Overview -Apache Pulsar supports [TLS encryption](security-tls-transport.md) and [TLS authentication](security-tls-authentication.md) between clients and Apache Pulsar service. +Apache Pulsar supports [TLS encryption](security-tls-transport.md) and [TLS authentication](security-tls-authentication) between clients and Apache Pulsar service. By default it uses PEM format file configuration. This page tries to describe use [KeyStore](https://en.wikipedia.org/wiki/Java_KeyStore) type configure for TLS. @@ -32,7 +32,7 @@ keytool -keystore broker.keystore.jks -alias localhost -validity {validity} -gen You need to specify two parameters in the above command: 1. `keystore`: the keystore file that stores the certificate. The *keystore* file contains the private key of - the certificate; hence, it needs to be kept safely. + the certificate; hence, it needs to be kept safely. 2. `validity`: the valid time of the certificate in days. > Ensure that common name (CN) matches exactly with the fully qualified domain name (FQDN) of the server. @@ -155,13 +155,13 @@ NOTE: it is important to restrict access to the store files via filesystem permi Optional settings that may worth consider: 1. tlsClientAuthentication=false: Enable/Disable using TLS for authentication. This config when enabled will authenticate the other end - of the communication channel. It should be enabled on both brokers and clients for mutual TLS. + of the communication channel. It should be enabled on both brokers and clients for mutual TLS. 2. tlsCiphers=[TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256], A cipher suite is a named combination of authentication, encryption, MAC and key exchange - algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, - it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) - [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) + algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, + it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) + [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) 3. tlsProtocols=[TLSv1.2,TLSv1.1,TLSv1] (list out the TLS protocols that you are going to accept from clients). - By default, it is not set. + By default, it is not set. ### Configuring Clients @@ -169,35 +169,35 @@ This is similar to [TLS encryption configuing for client with PEM type](security For a a minimal configuration, user need to provide the TrustStore information. e.g. -1. for [Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - - ``` +1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. + + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .build(); + + ``` 1. for java admin client @@ -214,7 +214,7 @@ e.g. ## TLS authentication with KeyStore configure -This similar to [TLS authentication with PEM type](security-tls-authentication.md) +This similar to [TLS authentication with PEM type](security-tls-authentication) ### broker authentication config @@ -262,56 +262,56 @@ webSocketServiceEnabled=false Besides the TLS encryption configuring. The main work is configuring the KeyStore, which contains a valid CN as client role, for client. e.g. -1. for [Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls - authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} - - ``` +1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. + + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls + authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` 1. for java admin client - ```java - - PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` ## Enabling TLS Logging diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-tls-transport.md b/site2/website-next/versioned_docs/version-2.7.3/security-tls-transport.md index 967b5029acf8f..22e5d4b58366c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-tls-transport.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-tls-transport.md @@ -1,7 +1,7 @@ --- id: security-tls-transport title: Transport Encryption using TLS -sidebar_label: Transport Encryption using TLS +sidebar_label: "Transport Encryption using TLS" original_id: security-tls-transport --- @@ -13,7 +13,7 @@ import TabItem from '@theme/TabItem'; By default, Apache Pulsar clients communicate with the Apache Pulsar service in plain text. This means that all data is sent in the clear. You can use TLS to encrypt this traffic to protect the traffic from the snooping of a man-in-the-middle attacker. -You can also configure TLS for both encryption and authentication. Use this guide to configure just TLS transport encryption and refer to [here](security-tls-authentication.md) for TLS authentication configuration. Alternatively, you can use [another authentication mechanism](security-athenz.md) on top of TLS transport encryption. +You can also configure TLS for both encryption and authentication. Use this guide to configure just TLS transport encryption and refer to [here](security-tls-authentication.md) for TLS authentication configuration. Alternatively, you can use [another authentication mechanism](security-athenz) on top of TLS transport encryption. > Note that enabling TLS may impact the performance due to encryption overhead. @@ -23,7 +23,7 @@ TLS is a form of [public key cryptography](https://en.wikipedia.org/wiki/Public- To use TLS transport encryption, you need two kinds of key pairs, **server key pairs** and a **certificate authority**. -You can use a third kind of key pair, **client key pairs**, for [client authentication](security-tls-authentication.md). +You can use a third kind of key pair, **client key pairs**, for [client authentication](security-tls-authentication). You should store the **certificate authority** private key in a very secure location (a fully encrypted, disconnected, air gapped computer). As for the certificate authority public key, the **trust cert**, you can freely shared it. @@ -31,9 +31,9 @@ For both client and server key pairs, the administrator first generates a privat For TLS transport encryption, the clients can use the **trust cert** to verify that the server has a key pair that the certificate authority signed when the clients are talking to the server. A man-in-the-middle attacker does not have access to the certificate authority, so they couldn't create a server with such a key pair. -For TLS authentication, the server uses the **trust cert** to verify that the client has a key pair that the certificate authority signed. The common name of the **client cert** is then used as the client's role token (see [Overview](security-overview.md)). +For TLS authentication, the server uses the **trust cert** to verify that the client has a key pair that the certificate authority signed. The common name of the **client cert** is then used as the client's role token (see [Overview](security-overview)). -`Bouncy Castle Provider` provides cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle.md). +`Bouncy Castle Provider` provides cipher suites and algorithms in Pulsar. If you need [FIPS](https://www.bouncycastle.org/fips_faq.html) version of `Bouncy Castle Provider`, please reference [Bouncy Castle page](security-bouncy-castle). ## Create TLS certificates @@ -132,7 +132,7 @@ At this point, you have a cert, `broker.cert.pem`, and a key, `broker.key-pk8.pe ## Broker Configuration -To configure a Pulsar [broker](reference-terminology.md#broker) to use TLS transport encryption, you need to make some changes to `broker.conf`, which locates in the `conf` directory of your [Pulsar installation](getting-started-standalone.md). +To configure a Pulsar [broker](reference-terminology.md#broker) to use TLS transport encryption, you need to make some changes to `broker.conf`, which locates in the `conf` directory of your [Pulsar installation](getting-started-standalone). Add these values to the configuration file (substituting the appropriate certificate paths where necessary): @@ -171,7 +171,7 @@ For JDK 8, you can obtain a list of supported values from the documentation: Proxies need to configure TLS in two directions, for clients connecting to the proxy, and for the proxy connecting to brokers. -```properties +``` # For clients connecting to the proxy tlsEnabledInProxy=true @@ -187,7 +187,7 @@ brokerClientTrustCertsFilePath=/path/to/ca.cert.pem ## Client configuration -When you enable the TLS transport encryption, you need to configure the client to use ```https://``` and port 8443 for the web service URL, and ```pulsar+ssl://``` and port 6651 for the broker service URL. +When you enable the TLS transport encryption, you need to configure the client to use ``` As the server certificate that you generated above does not belong to any of the default trust chains, you also need to either specify the path the **trust cert** (recommended), or tell the client to allow untrusted server certs. @@ -203,7 +203,7 @@ The examples below show hostname verification being disabled for the Java client ### CLI tools -[Command-line tools](reference-cli-tools.md) like [`pulsar-admin`](reference-cli-tools.md#pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. +[Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools.md#pulsar-admin), [`pulsar-perf`](reference-cli-tools.md#pulsar-perf), and [`pulsar-client`](reference-cli-tools.md#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. You need to add the following parameters to that file to use TLS transport with the CLI tools of Pulsar: @@ -250,6 +250,7 @@ client = Client("pulsar+ssl://broker.example.com:6651/", #### C++ client ```c++ + #include ClientConfiguration config = ClientConfiguration(); @@ -279,6 +280,7 @@ const Pulsar = require('pulsar-client'); #### C# client ```c# + var certificate = new X509Certificate2("ca.cert.pem"); var client = PulsarClient.Builder() .TrustedCertificateAuthority(certificate) //If the CA is not trusted on the host, you can add it explicitly. @@ -287,3 +289,4 @@ var client = PulsarClient.Builder() .Build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/security-token-admin.md b/site2/website-next/versioned_docs/version-2.7.3/security-token-admin.md index c73697a9f60b6..03de65fe264df 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/security-token-admin.md +++ b/site2/website-next/versioned_docs/version-2.7.3/security-token-admin.md @@ -1,7 +1,7 @@ --- id: security-token-admin title: Token authentication admin -sidebar_label: Token authentication admin +sidebar_label: "Token authentication admin" original_id: security-token-admin --- @@ -21,9 +21,10 @@ A user will typically be given a token string by an administrator (or some autom The compact representation of a signed JWT is a string that looks like: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY - ``` +``` Application will specify the token when creating the client instance. An alternative is to pass a "token supplier", that is to say a function that returns the token when the client library @@ -32,7 +33,7 @@ will need one. > #### Always use TLS transport encryption > Sending a token is equivalent to sending a password over the wire. It is strongly recommended to > always use TLS encryption when talking to the Pulsar service. See -> [Transport Encryption using TLS](security-tls-transport.md) +> [Transport Encryption using TLS](security-tls-transport) ## Secret vs Public/Private keys @@ -59,6 +60,7 @@ the brokers to allow them to validate the clients. $ bin/pulsar tokens create-secret-key --output my-secret.key ``` + To generate base64 encoded private key ```shell @@ -165,7 +167,7 @@ tokenSecretKey=file:///path/to/secret.key To configure proxies to authenticate clients, put the following in `proxy.conf`: The proxy will have its own token used when talking to brokers. The role token for this -key pair should be configured in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization.md) for more details. +key pair should be configured in the ``proxyRoles`` of the brokers. See the [authorization guide](security-authorization) for more details. ```properties @@ -182,3 +184,4 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 # brokerClientAuthenticationParameters=file:///path/to/proxy-token.txt ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/sql-deployment-configurations.md b/site2/website-next/versioned_docs/version-2.7.3/sql-deployment-configurations.md index 218c1e2413f3b..c1df08c917346 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/sql-deployment-configurations.md +++ b/site2/website-next/versioned_docs/version-2.7.3/sql-deployment-configurations.md @@ -1,7 +1,7 @@ --- id: sql-deployment-configurations title: Pulsar SQL configuration and deployment -sidebar_label: Configuration and deployment +sidebar_label: "Configuration and deployment" original_id: sql-deployment-configurations --- @@ -15,6 +15,7 @@ You can configure Presto Pulsar connector and deploy a cluster with the followin You can configure Presto Pulsar Connector in the `${project.root}/conf/presto/catalog/pulsar.properties` properties file. The configuration for the connector and the default values are as follows. ```properties + # name of the connector to be displayed in the catalog connector.name=pulsar @@ -29,13 +30,16 @@ pulsar.entry-read-batch-size=100 # default number of splits to use per query pulsar.target-num-splits=4 + ``` You can connect Presto to a Pulsar cluster with multiple hosts. To configure multiple hosts for brokers, add multiple URLs to `pulsar.broker-service-url`. To configure multiple hosts for ZooKeeper, add multiple URIs to `pulsar.zookeeper-uri`. The following is an example. - + ``` + pulsar.broker-service-url=http://localhost:8080,localhost:8081,localhost:8082 pulsar.zookeeper-uri=localhost1,localhost2:2181 + ``` ## Query data from existing Presto clusters @@ -43,7 +47,9 @@ pulsar.zookeeper-uri=localhost1,localhost2:2181 If you already have a Presto cluster, you can copy the Presto Pulsar connector plugin to your existing cluster. Download the archived plugin package with the following command. ```bash + $ wget pulsar:binary_release_url + ``` ## Deploy a new cluster @@ -52,14 +58,14 @@ Since Pulsar SQL is powered by [Presto](https://prestosql.io), the configuration :::note -For how to set up a standalone single node environment, refer to [Query data](sql-getting-started.md). +For how to set up a standalone single node environment, refer to [Query data](sql-getting-started). ::: - You can use the same CLI args as the Presto launcher. ```bash + $ ./bin/pulsar sql-worker --help Usage: launcher [options] command @@ -93,13 +99,17 @@ The default configuration for the cluster is located in `${project.root}/conf/pr You can set the worker to read from a different configuration directory, or set a different directory to write data. ```bash + $ ./bin/pulsar sql-worker run --etc-dir /tmp/incubator-pulsar/conf/presto --data-dir /tmp/presto-1 + ``` You can start the worker as daemon process. ```bash + $ ./bin/pulsar sql-worker start + ``` ### Deploy a cluster on multiple nodes @@ -111,6 +121,7 @@ You can deploy a Pulsar SQL cluster or Presto cluster on multiple nodes. The fol The first node runs as Presto coordinator. The minimal configuration requirement in the `${project.root}/conf/presto/config.properties` file is as follows. ```properties + coordinator=true node-scheduler.include-coordinator=true http-server.http.port=8080 @@ -118,16 +129,19 @@ query.max-memory=50GB query.max-memory-per-node=1GB discovery-server.enabled=true discovery.uri= + ``` The other two nodes serve as worker nodes, you can use the following configuration for worker nodes. ```properties + coordinator=false http-server.http.port=8080 query.max-memory=50GB query.max-memory-per-node=1GB discovery.uri= + ``` 2. Modify `pulsar.broker-service-url` and `pulsar.zookeeper-uri` configuration in the `${project.root}/conf/presto/catalog/pulsar.properties` file accordingly for the three nodes. @@ -135,30 +149,38 @@ discovery.uri= 3. Start the coordinator node. ``` + $ ./bin/pulsar sql-worker run + ``` 4. Start worker nodes. ``` + $ ./bin/pulsar sql-worker run + ``` 5. Start the SQL CLI and check the status of your cluster. ```bash + $ ./bin/pulsar sql --server + ``` 6. Check the status of your nodes. ```bash + presto> SELECT * FROM system.runtime.nodes; node_id | http_uri | node_version | coordinator | state ---------+-------------------------+--------------+-------------+-------- 1 | http://192.168.2.1:8081 | testversion | true | active 3 | http://192.168.2.2:8081 | testversion | false | active - 2 | http://192.168.2.3:8081 | testversion | false | active + 2 | http://192.168.2.3:8081 | testversion | false | active + ``` For more information about deployment in Presto, refer to [Presto deployment](https://prestosql.io/docs/current/installation/deployment.html). diff --git a/site2/website-next/versioned_docs/version-2.7.3/sql-getting-started.md b/site2/website-next/versioned_docs/version-2.7.3/sql-getting-started.md index d95de607e8fbc..e8c4ecb101fdd 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/sql-getting-started.md +++ b/site2/website-next/versioned_docs/version-2.7.3/sql-getting-started.md @@ -1,7 +1,7 @@ --- id: sql-getting-started title: Query data with Pulsar SQL -sidebar_label: Query data +sidebar_label: "Query data" original_id: sql-getting-started --- @@ -21,24 +21,31 @@ To query data in Pulsar with Pulsar SQL, complete the following steps. 1. Start a Pulsar standalone cluster. ```bash + ./bin/pulsar standalone + ``` 2. Start a Pulsar SQL worker. ```bash + ./bin/pulsar sql-worker run + ``` 3. After initializing Pulsar standalone cluster and the SQL worker, run SQL CLI. ```bash + ./bin/pulsar sql + ``` 4. Test with SQL commands. ```bash + presto> show catalogs; Catalog --------- @@ -81,12 +88,15 @@ Since there is no data in Pulsar, no records is returned. 5. Start the built-in connector _DataGeneratorSource_ and ingest some mock data. ```bash + ./bin/pulsar-admin sources create --name generator --destinationTopicName generator_test --source-type data-generator + ``` And then you can query a topic in the namespace "public/default". ```bash + presto> show tables in pulsar."public/default"; Table ---------------- @@ -96,11 +106,13 @@ presto> show tables in pulsar."public/default"; Query 20180829_213202_00000_csyeu, FINISHED, 1 node Splits: 19 total, 19 done (100.00%) 0:02 [1 rows, 38B] [0 rows/s, 17B/s] + ``` You can now query the data within the topic "generator_test". ```bash + presto> select * from pulsar."public/default".generator_test; firstname | middlename | lastname | email | username | password | telephonenumber | age | companyemail | nationalidentitycardnumber | @@ -114,6 +126,7 @@ presto> select * from pulsar."public/default".generator_test; . . . + ``` You can query the mock data. @@ -122,6 +135,7 @@ You can query the mock data. If you want to query your own data, you need to ingest your own data first. You can write a simple producer and write custom defined data to Pulsar. The following is an example. ```java + public class TestProducer { public static class Foo { @@ -172,4 +186,6 @@ public class TestProducer { pulsarClient.close(); } } + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/sql-overview.md b/site2/website-next/versioned_docs/version-2.7.3/sql-overview.md index 369ac2c5e64f4..fd92526892b0a 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/sql-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/sql-overview.md @@ -1,7 +1,7 @@ --- id: sql-overview title: Pulsar SQL Overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: sql-overview --- @@ -9,7 +9,7 @@ import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -Apache Pulsar is used to store streams of event data, and the event data is structured with predefined fields. With the implementation of the [Schema Registry](schema-get-started.md), you can store structured data in Pulsar and query the data by using [Presto](https://prestosql.io/). +Apache Pulsar is used to store streams of event data, and the event data is structured with predefined fields. With the implementation of the [Schema Registry](schema-get-started), you can store structured data in Pulsar and query the data by using [Presto](https://prestosql.io/). As the core of Pulsar SQL, Presto Pulsar connector enables Presto workers within a Presto cluster to query data from Pulsar. diff --git a/site2/website-next/versioned_docs/version-2.7.3/sql-rest-api.md b/site2/website-next/versioned_docs/version-2.7.3/sql-rest-api.md index 61b76d6e06ce0..d2b078793efd0 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/sql-rest-api.md +++ b/site2/website-next/versioned_docs/version-2.7.3/sql-rest-api.md @@ -1,7 +1,7 @@ --- id: sql-rest-api title: Pulsar SQL REST APIs -sidebar_label: REST APIs +sidebar_label: "REST APIs" original_id: sql-rest-api --- @@ -20,7 +20,9 @@ To request services, use explicit URL `http://presto.service:8081/v1`. You need `POST` requests require the `X-Presto-User` header. If you use authentication, you must use the same `username` that is specified in the authentication configuration. If you do not use authentication, you can specify anything for `username`. ```properties + X-Presto-User: username + ``` For more information about headers, refer to [PrestoHeaders](https://github.com/trinodb/trino). @@ -32,6 +34,7 @@ You can use statement in the HTTP body. All data is received as JSON document th The following is an example of `show catalogs`. The query continues until the received JSON document does not contain a `nextUri` link. Since no `error` is displayed in `stats`, it means that the query completes successfully. ```powershell + ➜ ~ curl --header "X-Presto-User: test-user" --request POST --data 'show catalogs' http://localhost:8081/v1/statement { "infoUri" : "http://localhost:8081/ui/query.html?20191113_033653_00006_dg6hb", @@ -181,6 +184,7 @@ The following is an example of `show catalogs`. The query continues until the re "completedSplits" : 19 } } + ``` :::note @@ -189,5 +193,4 @@ Since the response data is not in sync with the query state from the perspective ::: - For more information about Presto REST API, refer to [Presto HTTP Protocol](https://github.com/prestosql/presto/wiki/HTTP-Protocol). diff --git a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-aws.md b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-aws.md index 17ae7d8e86a8c..49d85ad053788 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-aws.md +++ b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-aws.md @@ -1,7 +1,7 @@ --- id: tiered-storage-aws title: Use AWS S3 offloader with Pulsar -sidebar_label: AWS S3 offloader +sidebar_label: "AWS S3 offloader" original_id: tiered-storage-aws --- @@ -31,43 +31,48 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget): - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` - -:::note - -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + :::note -::: + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + ::: ## Configuration @@ -77,7 +82,6 @@ Before offloading data from BookKeeper to AWS S3, you need to configure some pro ::: - Besides, you can also configure the AWS S3 offloader to run it automatically or trigger it manually. ### Configure AWS S3 offloader driver @@ -86,21 +90,21 @@ You can configure the AWS S3 offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 - `offloadersDirectory` | Offloader directory | offloaders - `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 + `offloadersDirectory` | Offloader directory | offloaders + `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 - `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 + `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -111,14 +115,15 @@ A bucket is a basic container that holds your data. Everything you store in AWS This example names the bucket as _pulsar-topic-offload_. ```conf + s3ManagedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Bucket region A bucket region is a region where a bucket is located. If a bucket region is not specified, the **default** region (`US East (N. Virginia)`) is used. - :::tip For more information about AWS regions and endpoints, see [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). @@ -131,7 +136,9 @@ For more information about AWS regions and endpoints, see [here](https://docs.aw This example sets the bucket region as _europe-west-3_. ``` + s3ManagedLedgerOffloadRegion=eu-west-3 + ``` #### Authentication (required) @@ -143,43 +150,51 @@ but relies on the mechanisms supported by the [DefaultAWSCredentialsProviderChai Once you have created a set of credentials in the AWS IAM console, you can configure credentials using one of the following methods. -- Use EC2 instance metadata credentials. +* Use EC2 instance metadata credentials. - If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. + If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. -- Set the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` in `conf/pulsar_env.sh`. +* Set the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. + "export" is important so that the variables are made available in the environment of spawned processes. - ```bash - export AWS_ACCESS_KEY_ID=ABC123456789 - export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```bash + + export AWS_ACCESS_KEY_ID=ABC123456789 + export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` -- Add the Java system properties `aws.accessKeyId` and `aws.secretKey` to `PULSAR_EXTRA_OPTS` in `conf/pulsar_env.sh`. +* Add the Java system properties `aws.accessKeyId` and `aws.secretKey` to `PULSAR_EXTRA_OPTS` in `conf/pulsar_env.sh`. - ```bash - PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" - ``` + ```bash + + PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" + + ``` -- Set the access credentials in `~/.aws/credentials`. +* Set the access credentials in `~/.aws/credentials`. - ```conf - [default] - aws_access_key_id=ABC123456789 - aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```conf + + [default] + aws_access_key_id=ABC123456789 + aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` -- Assume an IAM role. +* Assume an IAM role. - This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. + This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. - The broker must be rebooted for credentials specified in `pulsar_env` to take effect. + The broker must be rebooted for credentials specified in `pulsar_env` to take effect. - ```conf - s3ManagedLedgerOffloadRole= - s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload - ``` + ```conf + + s3ManagedLedgerOffloadRole= + s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload + + ``` #### Size of block read/write @@ -211,7 +226,9 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the AWS S3 offloader threshold size to 10 MB using pulsar-admin. ```bash + bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip @@ -220,7 +237,6 @@ For more information about the `pulsar-admin namespaces set-offload-threshold op ::: - ### Configure AWS S3 offloader to run manually For individual topics, you can trigger AWS S3 offloader manually using one of the following methods: @@ -229,76 +245,88 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the AWS S3 offloader to run manually using pulsar-admin. - ```bash - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` - -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). - -::: + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + ::: - This example checks the AWS S3 offloader status using pulsar-admin. - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the AWS S3 offloader to complete the job, add the `-w` flag. + To wait for the AWS S3 offloader to complete the job, add the `-w` flag. - ```bash - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` + + Error in offload + null - -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + :::tip -::: + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-azure.md b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-azure.md index d57762c9f0a3c..13824394e2cc3 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-azure.md +++ b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-azure.md @@ -1,7 +1,7 @@ --- id: tiered-storage-azure title: Use Azure BlobStore offloader with Pulsar -sidebar_label: Azure BlobStore offloader +sidebar_label: "Azure BlobStore offloader" original_id: tiered-storage-azure --- @@ -31,43 +31,48 @@ This example uses Pulsar 2.6.2. * Use [wget](https://www.gnu.org/software/wget): - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders - - ls offloaders - ``` + ``` + + mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders - **Output** + ls offloaders + + ``` - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. + **Output** - ``` - tiered-storage-file-system-2.6.2.nar - tiered-storage-jcloud-2.6.2.nar - ``` + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. - -:::note + ``` + + tiered-storage-file-system-2.6.2.nar + tiered-storage-jcloud-2.6.2.nar + + ``` -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + :::note -::: + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + ::: ## Configuration @@ -77,7 +82,6 @@ Before offloading data from BookKeeper to Azure BlobStore, you need to configure ::: - Besides, you can also configure the Azure BlobStore offloader to run it automatically or trigger it manually. ### Configure Azure BlobStore offloader driver @@ -86,20 +90,20 @@ You can configure the Azure BlobStore offloader driver in the configuration file - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name | azureblob - `offloadersDirectory` | Offloader directory | offloaders - `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name | azureblob + `offloadersDirectory` | Offloader directory | offloaders + `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -110,21 +114,25 @@ A bucket is a basic container that holds your data. Everything you store in Azur This example names the bucket as _pulsar-topic-offload_. ```conf + managedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Authentication (required) To be able to access Azure BlobStore, you need to authenticate with Azure BlobStore. -- Set the environment variables `AZURE_STORAGE_ACCOUNT` and `AZURE_STORAGE_ACCESS_KEY` in `conf/pulsar_env.sh`. +* Set the environment variables `AZURE_STORAGE_ACCOUNT` and `AZURE_STORAGE_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. + "export" is important so that the variables are made available in the environment of spawned processes. - ```bash - export AZURE_STORAGE_ACCOUNT=ABC123456789 - export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - ``` + ```bash + + export AZURE_STORAGE_ACCOUNT=ABC123456789 + export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` #### Size of block read/write @@ -156,7 +164,9 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the Azure BlobStore offloader threshold size to 10 MB using pulsar-admin. ```bash + bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip @@ -165,7 +175,6 @@ For more information about the `pulsar-admin namespaces set-offload-threshold op ::: - ### Configure Azure BlobStore offloader to run manually For individual topics, you can trigger Azure BlobStore offloader manually using one of the following methods: @@ -174,73 +183,86 @@ For individual topics, you can trigger Azure BlobStore offloader manually using - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the Azure BlobStore offloader to run manually using pulsar-admin. - ```bash - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` - - -:::tip + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + :::tip -::: + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + ::: - This example checks the Azure BlobStore offloader status using pulsar-admin. - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. + To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. - ```bash - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` + + Error in offload + null - -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + :::tip -::: + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + + ::: diff --git a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-filesystem.md b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-filesystem.md index 6af4ac28a0195..316d95d7cf933 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-filesystem.md +++ b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-filesystem.md @@ -1,7 +1,7 @@ --- id: tiered-storage-filesystem title: Use filesystem offloader with Pulsar -sidebar_label: Filesystem offloader +sidebar_label: "Filesystem offloader" original_id: tiered-storage-filesystem --- @@ -33,51 +33,54 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - -:::note + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + :::note -::: + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` - :::note + :::note - * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - - * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. - - ::: + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + ::: ## Configuration @@ -87,7 +90,6 @@ Before offloading data from BookKeeper to filesystem, you need to configure some ::: - Besides, you can also configure the filesystem offloader to run it automatically or trigger it manually. ### Configure filesystem offloader driver @@ -96,18 +98,18 @@ You can configure filesystem offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem - `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 - `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem + `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 + `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml - **Optional** configurations are as below. - Optional configuration| Description | Example value - |---|---|--- - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional configuration| Description | Example value + |---|---|--- + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Offloader driver (required) @@ -116,7 +118,9 @@ Offloader driver name, which is case-insensitive. This example sets the offloader driver name as _filesystem_. ```conf + managedLedgerOffloadDriver=filesystem + ``` #### Connection address (required) @@ -128,7 +132,9 @@ Connection address is the URI to access the default Hadoop distributed file syst This example sets the connection address as _hdfs://127.0.0.1:9000_. ```conf + fileSystemURI=hdfs://127.0.0.1:9000 + ``` #### Hadoop profile path (required) @@ -140,12 +146,15 @@ The configuration file is stored in the Hadoop profile path. It contains various This example sets the Hadoop profile path as _../conf/filesystem_offload_core_site.xml_. ```conf + fileSystemProfilePath=../conf/filesystem_offload_core_site.xml + ``` You can set the following configurations in the _filesystem_offload_core_site.xml_ file. ``` + fs.defaultFS @@ -175,6 +184,7 @@ You can set the following configurations in the _filesystem_offload_core_site.xm io.map.index.interval 128 + ``` :::tip @@ -183,7 +193,6 @@ For more information about the Hadoop HDFS, see [here](https://hadoop.apache.org ::: - ### Configure filesystem offloader to run automatically Namespace policy can be configured to offload data automatically once a threshold is reached. The threshold is based on the size of data that a topic has stored on a Pulsar cluster. Once the topic reaches the threshold, an offload operation is triggered automatically. @@ -203,7 +212,9 @@ You can configure the threshold size using CLI tools, such as pulsar-admin. This example sets the filesystem offloader threshold size to 10 MB using pulsar-admin. ```bash + pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip @@ -212,7 +223,6 @@ For more information about the `pulsar-admin namespaces set-offload-threshold op ::: - ### Configure filesystem offloader to run manually For individual topics, you can trigger filesystem offloader manually using one of the following methods: @@ -227,70 +237,82 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh - This example triggers the filesystem offloader to run manually using pulsar-admin. - ```bash - pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` - - -:::tip + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + :::tip -::: + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + ::: - This example checks filesystem offloader status using pulsar-admin. - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for the filesystem to complete the job, add the `-w` flag. + To wait for the filesystem to complete the job, add the `-w` flag. - ```bash - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - - ``` - Offload was a success - ``` + **Output** - If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` + + Error in offload + null - -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + :::tip -::: + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-gcs.md b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-gcs.md index f2657c807604c..5f0d00349296c 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-gcs.md +++ b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-gcs.md @@ -1,7 +1,7 @@ --- id: tiered-storage-gcs title: Use GCS offloader with Pulsar -sidebar_label: GCS offloader +sidebar_label: "GCS offloader" original_id: tiered-storage-gcs --- @@ -31,44 +31,49 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` - -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. - -::: + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. + As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` ## Configuration @@ -78,7 +83,6 @@ Before offloading data from BookKeeper to GCS, you need to configure some proper ::: - Besides, you can also configure the GCS offloader to run it automatically or trigger it manually. ### Configure GCS offloader driver @@ -87,22 +91,22 @@ You can configure GCS offloader driver in the configuration file `broker.conf` o - **Required** configurations are as below. - **Required** configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage - `offloadersDirectory`|Offloader directory|offloaders - `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload - `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 - `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json + **Required** configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage + `offloadersDirectory`|Offloader directory|offloaders + `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload + `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 + `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json - **Optional** configurations are as below. - Optional configuration|Description|Example value - |---|---|--- - `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 - `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 + Optional configuration|Description|Example value + |---|---|--- + `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 + `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 #### Bucket (required) @@ -113,27 +117,29 @@ A bucket is a basic container that holds your data. Everything you store in GCS This example names the bucket as _pulsar-topic-offload_. ```conf + gcsManagedLedgerOffloadBucket=pulsar-topic-offload + ``` #### Bucket region (required) Bucket region is the region where a bucket is located. If a bucket region is not specified, the **default** region (`us multi-regional location`) is used. - :::tip For more information about bucket location, see [here](https://cloud.google.com/storage/docs/bucket-locations). ::: - ##### Example This example sets the bucket region as _europe-west3_. ``` + gcsManagedLedgerOffloadRegion=europe-west3 + ``` #### Authentication (required) @@ -155,31 +161,30 @@ To generate service account credentials or view the public credentials that you' 4. In the **Create service account** window, type a name for the service account and select **Furnish a new private key**. - If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. + If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. 5. Click **Create**. - :::note + :::note - Make sure the service account you create has permission to operate GCS, you need to assign **Storage Admin** permission to your service account [here](https://cloud.google.com/storage/docs/access-control/iam). - - ::: + Make sure the service account you create has permission to operate GCS, you need to assign **Storage Admin** permission to your service account [here](https://cloud.google.com/storage/docs/access-control/iam). + ::: 6. You can get the following information and set this in `broker.conf`. - - ```conf - gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" - ``` - :::tip + ```conf + + gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" + + ``` - - For more information about how to create `gcsManagedLedgerOffloadServiceAccountKeyFile`, see [here](https://support.google.com/googleapi/answer/6158849). - - - For more information about Google Cloud IAM, see [here](https://cloud.google.com/storage/docs/access-control/iam). + :::tip - ::: + - For more information about how to create `gcsManagedLedgerOffloadServiceAccountKeyFile`, see [here](https://support.google.com/googleapi/answer/6158849). + - For more information about Google Cloud IAM, see [here](https://cloud.google.com/storage/docs/access-control/iam). + ::: #### Size of block read/write @@ -211,7 +216,9 @@ The offload configurations in `broker.conf` and `standalone.conf` are used for t This example sets the GCS offloader threshold size to 10 MB using pulsar-admin. ```bash + pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace + ``` :::tip @@ -220,7 +227,6 @@ For more information about the `pulsar-admin namespaces set-offload-threshold op ::: - ### Configure GCS offloader to run manually For individual topics, you can trigger GCS offloader manually using one of the following methods: @@ -229,76 +235,88 @@ For individual topics, you can trigger GCS offloader manually using one of the f - Use CLI tools (such as pulsar-admin). - To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. + To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the GCS offloader to run manually using pulsar-admin with the command `pulsar-admin topics offload (topic-name) (threshold)`. - ```bash - pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M - ``` + ```bash + + pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M + + ``` - **Output** + **Output** - ```bash - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` - -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). - -::: + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + ::: - This example checks the GCS offloader status using pulsar-admin with the command `pulsar-admin topics offload-status options`. - ```bash - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ```bash - Offload is currently running - ``` + ```bash + + Offload is currently running + + ``` - To wait for GCS to complete the job, add the `-w` flag. + To wait for GCS to complete the job, add the `-w` flag. - ```bash - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - ``` + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ``` - Offload was a success - ``` + ``` + + Offload was a success + + ``` - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - ```bash + ```bash + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - ``` - - **Output** + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` + + Error in offload + null - -:::tip + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + :::tip -::: + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-overview.md b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-overview.md index 2f8c29ace5388..8536268c48bc8 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-overview.md +++ b/site2/website-next/versioned_docs/version-2.7.3/tiered-storage-overview.md @@ -1,7 +1,7 @@ --- id: tiered-storage-overview title: Overview of tiered storage -sidebar_label: Overview +sidebar_label: "Overview" original_id: tiered-storage-overview --- @@ -11,33 +11,26 @@ import TabItem from '@theme/TabItem'; Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -- Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. - + :::tip -:::tip + For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws). + For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs). -For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws.md). - -For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs.md). + ::: -::: +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystems for long term storage. + With Hadoop, it is easy to add support for more filesystems in the future. -- Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystems for long term storage. + :::tip - With Hadoop, it is easy to add support for more filesystems in the future. - - - -:::tip - -For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem.md). - -::: + For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem). + ::: ## When to use tiered storage? diff --git a/site2/website-next/versioned_docs/version-2.7.3/transactions-api.md b/site2/website-next/versioned_docs/version-2.7.3/transaction-api.md similarity index 98% rename from site2/website-next/versioned_docs/version-2.7.3/transactions-api.md rename to site2/website-next/versioned_docs/version-2.7.3/transaction-api.md index 9be87bb9e7d8e..ba8dedab3befe 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/transactions-api.md +++ b/site2/website-next/versioned_docs/version-2.7.3/transaction-api.md @@ -1,7 +1,7 @@ --- id: transactions-api title: Transactions API (Developer Preview) -sidebar_label: Transactions API +sidebar_label: "Transactions API" original_id: transactions-api --- @@ -17,13 +17,17 @@ Currently, Pulsar transaction is a developer preview feature. It is disabled by 1. To enable transactions in Pulsar, you need to configure the parameter in the `broker.conf` file. ``` + transactionCoordinatorEnabled=true + ``` 2. Initialize transaction coordinator metadata, so the transaction coordinators can leverage advantages of the partitioned topic, such as load balance. ``` + bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone + ``` After initializing transaction coordinator metadata, you can use the transactions API. The following APIs are available. @@ -33,21 +37,25 @@ After initializing transaction coordinator metadata, you can use the transaction You can enable transaction for transaction client and initialize transaction coordinator client. ``` + PulsarClient pulsarClient = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .enableTransaction(true) .build(); + ``` ## Start transactions You can start transaction in the following way. ``` + Transaction txn = pulsarClient .newTransaction() .withTransactionTimeout(5, TimeUnit.MINUTES) .build() .get(); + ``` ## Produce transaction messages @@ -55,7 +63,9 @@ Transaction txn = pulsarClient A transaction parameter is required when producing new transaction messages. The semantic of the transaction messages in Pulsar is `read-committed`, so the consumer cannot receive the ongoing transaction messages before the transaction is committed. ``` + producer.newMessage(txn).value("Hello Pulsar Transaction".getBytes()).sendAsync(); + ``` ## Acknowledge the messages with the transaction @@ -63,8 +73,10 @@ producer.newMessage(txn).value("Hello Pulsar Transaction".getBytes()).sendAsync( The transaction acknowledgement requires a transaction parameter. The transaction acknowledgement marks the messages state to pending-ack state. When the transaction is committed, the pending-ack state becomes ack state. If the transaction is aborted, the pending-ack state becomes unack state. ``` + Message message = consumer.receive(); consumer.acknowledgeAsync(message.getMessageId(), txn); + ``` ## Commit transactions @@ -72,7 +84,9 @@ consumer.acknowledgeAsync(message.getMessageId(), txn); When the transaction is committed, consumers receive the transaction messages and the pending-ack state becomes ack state. ``` + txn.commit().get(); + ``` ## Abort transaction @@ -80,13 +94,16 @@ txn.commit().get(); When the transaction is aborted, the transaction acknowledgement is canceled and the pending-ack messages are redelivered. ``` + txn.abort().get(); + ``` ### Example The following example shows how messages are processed in transaction. - + ``` + PulsarClient pulsarClient = PulsarClient.builder() .serviceUrl(getPulsarServiceList().get(0).getBrokerServiceUrl()) .statsInterval(0, TimeUnit.SECONDS) @@ -129,6 +146,7 @@ sourceConsumer.acknowledgeAsync(message.getMessageId(), txn); sinkProducer.newMessage(txn).value("sink data").sendAsync(); txn.commit().get(); + ``` ## Enable batch messages in transactions @@ -138,12 +156,15 @@ To enable batch messages in transactions, you need to enable the batch index ack To enable batch index acknowledgement, you need to set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. ``` + acknowledgmentAtBatchIndexLevelEnabled=true + ``` And then you need to call the `enableBatchIndexAcknowledgment(true)` method in the consumer builder. ``` + Consumer sinkConsumer = pulsarClient .newConsumer() .topic(transferTopic) @@ -152,4 +173,6 @@ Consumer sinkConsumer = pulsarClient .subscriptionType(SubscriptionType.Shared) .enableBatchIndexAcknowledgment(true) // enable batch index acknowledgement .subscribe(); + ``` + diff --git a/site2/website-next/versioned_docs/version-2.7.3/transactions-guarantee.md b/site2/website-next/versioned_docs/version-2.7.3/transaction-guarantee.md similarity index 95% rename from site2/website-next/versioned_docs/version-2.7.3/transactions-guarantee.md rename to site2/website-next/versioned_docs/version-2.7.3/transaction-guarantee.md index c934f271450eb..9de4d588a2574 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/transactions-guarantee.md +++ b/site2/website-next/versioned_docs/version-2.7.3/transaction-guarantee.md @@ -1,7 +1,7 @@ --- id: transactions-guarantee title: Transactions Guarantee -sidebar_label: Transactions Guarantee +sidebar_label: "Transactions Guarantee" original_id: transactions-guarantee --- diff --git a/site2/website-next/versioned_docs/version-2.7.3/window-functions-context.md b/site2/website-next/versioned_docs/version-2.7.3/window-functions-context.md index ff7bd5871601c..940c06c574de5 100644 --- a/site2/website-next/versioned_docs/version-2.7.3/window-functions-context.md +++ b/site2/website-next/versioned_docs/version-2.7.3/window-functions-context.md @@ -50,8 +50,8 @@ The `getInputTopics` method gets the **name list** of all input topics. This example demonstrates how to get the name list of all input topics in a Java window function. - ```java + public class GetInputTopicsWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -62,6 +62,7 @@ public class GetInputTopicsWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -81,6 +82,7 @@ public class GetOutputTopicWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -101,6 +103,7 @@ public class GetTenantWindowFunction implements WindowFunction { } } + ``` ### Get namespace @@ -109,8 +112,8 @@ The `getNamespace` method gets the namespace associated with the window function This example demonstrates how to get the namespace in a Java window function. - ```java + public class GetNamespaceWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -121,6 +124,7 @@ public class GetNamespaceWindowFunction implements WindowFunction } } + ``` ### Get function name @@ -129,8 +133,8 @@ The `getFunctionName` method gets the window function name. This example demonstrates how to get the function name in a Java window function. - ```java + public class GetNameOfWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -141,6 +145,7 @@ public class GetNameOfWindowFunction implements WindowFunction { } } + ``` ### Get function ID @@ -149,8 +154,8 @@ The `getFunctionId` method gets the window function ID. This example demonstrates how to get the function ID in a Java window function. - ```java + public class GetFunctionIDWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -161,6 +166,7 @@ public class GetFunctionIDWindowFunction implements WindowFunction } } + ``` ### Get function version @@ -169,8 +175,8 @@ The `getFunctionVersion` method gets the window function version. This example demonstrates how to get the function version of a Java window function. - ```java + public class GetVersionOfWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -181,6 +187,7 @@ public class GetVersionOfWindowFunction implements WindowFunction } } + ``` ### Get instance ID @@ -189,8 +196,8 @@ The `getInstanceId` method gets the instance ID of a window function. This example demonstrates how to get the instance ID in a Java window function. - ```java + public class GetInstanceIDWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -201,6 +208,7 @@ public class GetInstanceIDWindowFunction implements WindowFunction } } + ``` ### Get num instances @@ -209,8 +217,8 @@ The `getNumInstances` method gets the number of instances that invoke the window This example demonstrates how to get the number of instances in a Java window function. - ```java + public class GetNumInstancesWindowFunction implements WindowFunction { @Override public Void process(Collection> inputs, WindowContext context) throws Exception { @@ -221,6 +229,7 @@ public class GetNumInstancesWindowFunction implements WindowFunction { @Override @@ -241,6 +250,7 @@ public class GetOutputSchemaTypeWindowFunction implements WindowFunction { } } + ``` If you need your function to produce logs, specify a log topic when creating or running the function. - ```bash + bin/pulsar-admin functions create \ --jar my-functions.jar \ --classname my.package.LoggingFunction \ --log-topic persistent://public/default/logging-function-logs \ # Other function configs + ``` You can access all logs produced by `LoggingFunction` via the `persistent://public/default/logging-function-logs` topic. @@ -287,16 +299,18 @@ You can access all logs produced by `LoggingFunction` via the `persistent://publ Pulsar window functions can publish arbitrary metrics to the metrics interface which can be queried. -> **Note** -> -> If a Pulsar window function uses the language-native interface for Java, that function is not able to publish metrics and stats to Pulsar. +:::note + +If a Pulsar window function uses the language-native interface for Java, that function is not able to publish metrics and stats to Pulsar. + +::: You can record metrics using the context object on a per-key basis. This example sets a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message in a Java function. - ```java + import java.util.Collection; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.WindowContext; @@ -320,6 +334,7 @@ public class UserMetricWindowFunction implements WindowFunction { return null; } } + ``` ## User config @@ -328,12 +343,13 @@ When you run or update Pulsar Functions that are created using SDK, you can pass This example passes a user configured key/value to a function. - ```bash + bin/pulsar-admin functions create \ --name word-filter \ --user-config '{"forbidden-word":"rosebud"}' \ # Other function configs + ``` ### API @@ -342,24 +358,23 @@ You can use the following APIs to get user-defined information for window functi `getUserConfigMap` API gets a map of all user-defined key/value configurations for the window function. - - ```java + /** * Get a map of all user-defined key/value configs for the function. * * @return The full map of user-defined config values */ Map getUserConfigMap(); -``` +``` #### getUserConfigValue The `getUserConfigValue` API gets a user-defined key/value. - ```java + /** * Get any user-defined key/value. * @@ -367,14 +382,15 @@ The `getUserConfigValue` API gets a user-defined key/value. * @return The Optional value specified by the user for that key. */ Optional getUserConfigValue(String key); + ``` #### getUserConfigValueOrDefault The `getUserConfigValueOrDefault` API gets a user-defined key/value or a default value if none is present. - ```java + /** * Get any user-defined key/value or a default value if none is present. * @@ -383,23 +399,27 @@ The `getUserConfigValueOrDefault` API gets a user-defined key/value or a default * @return Either the user config value associated with a given key or a supplied default value */ Object getUserConfigValueOrDefault(String key, Object defaultValue); + ``` This example demonstrates how to access key/value pairs provided to Pulsar window functions. Java SDK context object enables you to access key/value pairs provided to Pulsar window functions via the command line (as JSON). ->**Tip** -> -> For all key/value pairs passed to Java window functions, both the `key` and the `value` are `String`. To set the value to be a different type, you need to deserialize it from the `String` type. +:::tip -This example passes a key/value pair in a Java window function. +For all key/value pairs passed to Java window functions, both the `key` and the `value` are `String`. To set the value to be a different type, you need to deserialize it from the `String` type. +::: + +This example passes a key/value pair in a Java window function. ```bash + bin/pulsar-admin functions create \ --user-config '{"word-of-the-day":"verdure"}' \ # Other function configs + ``` This example accesses values in a Java window function. @@ -407,8 +427,8 @@ This example accesses values in a Java window function. The `UserConfigFunction` function logs the string `"The word of the day is verdure"` every time the function is invoked (which means every time a message arrives). The user config of `word-of-the-day` is changed **only** when the function is updated with a new config value via multiple ways, such as the command line tool or REST API. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.slf4j.Logger; @@ -427,17 +447,19 @@ public class UserConfigWindowFunction implements WindowFunction } } + ``` If no value is provided, you can access the entire user config map or set a default value. - ```java + // Get the whole config map Map allConfigs = context.getUserConfigMap(); // Get value or resort to default String wotd = context.getUserConfigValueOrDefault("word-of-the-day", "perspicacious"); + ``` ## Routing @@ -446,8 +468,8 @@ You can use the `context.publish()` interface to publish as many results as you This example shows that the `PublishFunction` class uses the built-in function in the context to publish messages to the `publishTopic` in a Java function. - ```java + public class PublishWindowFunction implements WindowFunction { @Override public Void process(Collection> input, WindowContext context) throws Exception { @@ -459,6 +481,7 @@ public class PublishWindowFunction implements WindowFunction { } } + ``` ## State storage @@ -485,14 +508,15 @@ The `incrCounter` API increases a built-in distributed counter referred by key. Applications use the `incrCounter` API to change the counter of a given `key` by the given `amount`. If the `key` does not exist, a new key is created. - ```java + /** * Increment the builtin distributed counter referred by key * @param key The name of the key * @param amount The amount to be incremented */ void incrCounter(String key, long amount); + ``` #### getCounter @@ -501,8 +525,8 @@ The `getCounter` API gets the counter value for the key. Applications uses the `getCounter` API to retrieve the counter of a given `key` changed by the `incrCounter` API. - ```java + /** * Retrieve the counter value for the key. * @@ -510,6 +534,7 @@ Applications uses the `getCounter` API to retrieve the counter of a given `key` * @return the amount of the counter value for this key */ long getCounter(String key); + ``` Except the `getCounter` API, Pulsar also exposes a general key/value API (`putState`) for functions to store general key/value state. @@ -518,8 +543,8 @@ Except the `getCounter` API, Pulsar also exposes a general key/value API (`putSt The `putState` API updates the state value for the key. - ```java + /** * Update the state value for the key. * @@ -527,6 +552,7 @@ The `putState` API updates the state value for the key. * @param value state value of the key */ void putState(String key, ByteBuffer value); + ``` This example demonstrates how applications store states in Pulsar window functions. @@ -537,8 +563,8 @@ The logic of the `WordCountWindowFunction` is simple and straightforward. 2. For each `word`, the function increments the corresponding `counter` by 1 via `incrCounter(key, amount)`. - ```java + import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -554,5 +580,6 @@ public class WordCountWindowFunction implements WindowFunction { } } + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/adaptors-kafka.md b/site2/website-next/versioned_docs/version-2.8.0/adaptors-kafka.md index 3a40188d0cc5d..18e38314d8949 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/adaptors-kafka.md +++ b/site2/website-next/versioned_docs/version-2.8.0/adaptors-kafka.md @@ -1,7 +1,7 @@ --- id: adaptors-kafka title: Pulsar adaptor for Apache Kafka -sidebar_label: Kafka client wrapper +sidebar_label: "Kafka client wrapper" original_id: adaptors-kafka --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/adaptors-spark.md b/site2/website-next/versioned_docs/version-2.8.0/adaptors-spark.md index 0989ab0ca032e..3f77e65d55915 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/adaptors-spark.md +++ b/site2/website-next/versioned_docs/version-2.8.0/adaptors-spark.md @@ -1,7 +1,7 @@ --- id: adaptors-spark title: Pulsar adaptor for Apache Spark -sidebar_label: Apache Spark +sidebar_label: "Apache Spark" original_id: adaptors-spark --- @@ -91,4 +91,5 @@ SparkStreamingPulsarReceiver pulsarReceiver = new SparkStreamingPulsarReceiver( pulsarConf, new AuthenticationToken("token:")); -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/adaptors-storm.md b/site2/website-next/versioned_docs/version-2.8.0/adaptors-storm.md index 38595681e777b..e4b07db19090c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/adaptors-storm.md +++ b/site2/website-next/versioned_docs/version-2.8.0/adaptors-storm.md @@ -1,7 +1,7 @@ --- id: adaptors-storm title: Pulsar adaptor for Apache Storm -sidebar_label: Apache Storm +sidebar_label: "Apache Storm" original_id: adaptors-storm --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-brokers.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-brokers.md index aa0d22caa1c29..b84c7d0eb8d93 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-brokers.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-brokers.md @@ -1,7 +1,7 @@ --- id: admin-api-brokers title: Managing Brokers -sidebar_label: Brokers +sidebar_label: "Brokers" original_id: admin-api-brokers --- @@ -55,6 +55,7 @@ $ pulsar-admin brokers list use ``` ``` + broker1.use.org.com:8080 ``` @@ -106,6 +107,7 @@ $ pulsar-admin brokers leader-broker ``` ``` + BrokerInfo(serviceUrl=broker1.use.org.com:8080) ``` @@ -123,6 +125,7 @@ BrokerInfo(serviceUrl=broker1.use.org.com:8080) admin.brokers().getLeaderBroker() ``` + For the detail of the code above, see [here](https://github.com/apache/pulsar/blob/master/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/BrokersImpl.java#L80) @@ -169,6 +172,7 @@ $ pulsar-admin brokers namespaces use \ } ``` + @@ -182,6 +186,7 @@ $ pulsar-admin brokers namespaces use \ admin.brokers().getOwnedNamespaces(cluster,brokerUrl); ``` + @@ -236,6 +241,7 @@ $ pulsar-admin brokers update-dynamic-config --config brokerShutdownTimeoutMs -- admin.brokers().updateDynamicConfiguration(configName, configValue); ``` + @@ -281,6 +287,7 @@ brokerShutdownTimeoutMs admin.brokers().getDynamicConfigurationNames(); ``` + @@ -327,6 +334,7 @@ brokerShutdownTimeoutMs:100 admin.brokers().getAllDynamicConfigurations(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-clusters.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-clusters.md index 4fa869860bb60..17aa3a3cff93b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-clusters.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-clusters.md @@ -1,7 +1,7 @@ --- id: admin-api-clusters title: Managing Clusters -sidebar_label: Clusters +sidebar_label: "Clusters" original_id: admin-api-clusters --- @@ -73,6 +73,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().createCluster(clusterName, clusterData); ``` + @@ -163,6 +164,7 @@ $ pulsar-admin clusters get cluster-1 admin.clusters().getCluster(clusterName); ``` + @@ -218,6 +220,7 @@ ClusterData clusterData = new ClusterData( admin.clusters().updateCluster(clusterName, clusterData); ``` + @@ -247,6 +250,7 @@ Clusters can be deleted from a Pulsar [instance](reference-terminology.md#instan Use the [`delete`](reference-pulsar-admin.md#clusters-delete) subcommand and specify the name of the cluster. ``` + $ pulsar-admin clusters delete cluster-1 ``` @@ -264,6 +268,7 @@ $ pulsar-admin clusters delete cluster-1 admin.clusters().deleteCluster(clusterName); ``` + @@ -313,6 +318,7 @@ cluster-2 admin.clusters().getClusters(); ``` + @@ -342,6 +348,7 @@ Peer clusters can be configured for a given cluster in a Pulsar [instance](refer Use the [`update-peer-clusters`](reference-pulsar-admin.md#clusters-update-peer-clusters) subcommand and specify the list of peer-cluster names. ``` + $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 ``` @@ -359,6 +366,7 @@ $ pulsar-admin update-peer-clusters cluster-1 --peer-clusters cluster-2 admin.clusters().updatePeerClusterNames(clusterName, peerClusterList); ``` + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-functions.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-functions.md index f5890ae6274b8..6b31c12746c13 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-functions.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-functions.md @@ -1,7 +1,7 @@ --- id: admin-api-functions title: Manage Functions -sidebar_label: Functions +sidebar_label: "Functions" original_id: admin-api-functions --- @@ -91,6 +91,7 @@ functionConfig.setOutput(sinkTopic); admin.functions().createFunction(functionConfig, fileName); ``` + @@ -154,6 +155,7 @@ updateOptions.setUpdateAuthData(updateAuthData); admin.functions().updateFunction(functionConfig, userCodeFile, updateOptions); ``` + @@ -205,6 +207,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -257,6 +260,7 @@ $ pulsar-admin functions start \ admin.functions().startFunction(tenant, namespace, functionName); ``` + @@ -310,6 +314,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -362,6 +367,7 @@ $ pulsar-admin functions stop \ admin.functions().stopFunction(tenant, namespace, functionName); ``` + @@ -415,6 +421,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -467,6 +474,7 @@ $ pulsar-admin functions restart \ admin.functions().restartFunction(tenant, namespace, functionName); ``` + @@ -518,6 +526,7 @@ $ pulsar-admin functions list \ admin.functions().getFunctions(tenant, namespace); ``` + @@ -553,7 +562,7 @@ Use the [`delete`](reference-pulsar-admin.md#functions-delete) subcommand. $ pulsar-admin functions delete \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -570,6 +579,7 @@ $ pulsar-admin functions delete \ admin.functions().deleteFunction(tenant, namespace, functionName); ``` + @@ -605,7 +615,7 @@ Use the [`get`](reference-pulsar-admin.md#functions-get) subcommand. $ pulsar-admin functions get \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -622,6 +632,7 @@ $ pulsar-admin functions get \ admin.functions().getFunction(tenant, namespace, functionName); ``` + @@ -674,6 +685,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -709,7 +721,7 @@ Use the [`status`](reference-pulsar-admin.md#functions-status) subcommand. $ pulsar-admin functions status \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -726,6 +738,7 @@ $ pulsar-admin functions status \ admin.functions().getFunctionStatus(tenant, namespace, functionName); ``` + @@ -778,6 +791,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName, Integer.parseInt(instanceId)); ``` + @@ -813,7 +827,7 @@ Use the [`stats`](reference-pulsar-admin.md#functions-stats) subcommand. $ pulsar-admin functions stats \ --tenant public \ --namespace default \ - --name (the name of Pulsar Functions) + --name (the name of Pulsar Functions) ``` @@ -830,6 +844,7 @@ $ pulsar-admin functions stats \ admin.functions().getFunctionStats(tenant, namespace, functionName); ``` + @@ -871,6 +886,7 @@ $ pulsar-admin functions trigger \ # or --trigger-file (the path of trigger file) ``` + @@ -884,6 +900,7 @@ $ pulsar-admin functions trigger \ admin.functions().triggerFunction(tenant, namespace, functionName, topic, triggerValue, triggerFile); ``` + @@ -920,7 +937,7 @@ $ pulsar-admin functions putstate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" + --state "{\"key\":\"pulsar\", \"stringValue\":\"hello pulsar\"}" ``` @@ -939,6 +956,7 @@ FunctionState stateRepr = ObjectMapperFactory.getThreadLocal().readValue(state, admin.functions().putFunctionState(tenant, namespace, functionName, stateRepr); ``` + @@ -975,7 +993,7 @@ $ pulsar-admin functions querystate \ --tenant public \ --namespace default \ --name (the name of Pulsar Functions) \ - --key (the key of state) + --key (the key of state) ``` @@ -992,6 +1010,7 @@ $ pulsar-admin functions querystate \ admin.functions().getFunctionState(tenant, namespace, functionName, key); ``` + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-namespaces.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-namespaces.md index bbabe03564d61..8dae54b80e0c6 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-namespaces.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-namespaces.md @@ -1,7 +1,7 @@ --- id: admin-api-namespaces title: Managing Namespaces -sidebar_label: Namespaces +sidebar_label: "Namespaces" original_id: admin-api-namespaces --- @@ -62,6 +62,7 @@ $ pulsar-admin namespaces create test-tenant/test-namespace admin.namespaces().createNamespace(namespace); ``` + @@ -130,6 +131,7 @@ $ pulsar-admin namespaces policies test-tenant/test-namespace admin.namespaces().getPolicies(namespace); ``` + @@ -179,6 +181,7 @@ test-tenant/ns2 admin.namespaces().getNamespaces(tenant); ``` + @@ -226,6 +229,7 @@ $ pulsar-admin namespaces delete test-tenant/ns1 admin.namespaces().deleteNamespace(namespace); ``` + @@ -255,6 +259,7 @@ It sets replication clusters for a namespace, so Pulsar can internally replicate ``` + $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ --clusters cl1 @@ -264,6 +269,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replication|operation/setNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -276,6 +282,7 @@ $ pulsar-admin namespaces set-clusters test-tenant/ns1 \ admin.namespaces().setNamespaceReplicationClusters(namespace, clusters); ``` + @@ -303,11 +310,13 @@ It gives a list of replication clusters for a given namespace. ``` + $ pulsar-admin namespaces get-clusters test-tenant/cl1/ns1 ``` ``` + cl2 ``` @@ -316,6 +325,7 @@ cl2 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replication|operation/getNamespaceReplicationClusters?version=@pulsar:version_number@} ``` @@ -328,6 +338,7 @@ cl2 admin.namespaces().getNamespaceReplicationClusters(namespace) ``` + @@ -365,11 +376,13 @@ Backlog quota helps the broker to restrict bandwidth/storage of a namespace once ``` + $ pulsar-admin namespaces set-backlog-quota --limit 10G --limitTime 36000 --policy producer_request_hold test-tenant/ns1 ``` ``` + N/A ``` @@ -378,6 +391,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/setBacklogQuota?version=@pulsar:version_number@} ``` @@ -390,6 +404,7 @@ N/A admin.namespaces().setBacklogQuota(namespace, new BacklogQuota(limit, limitTime, policy)) ``` + @@ -417,6 +432,7 @@ It shows a configured backlog quota for a given namespace. ``` + $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` @@ -436,6 +452,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/backlogQuotaMap|operation/getBacklogQuotaMap?version=@pulsar:version_number@} ``` @@ -448,6 +465,7 @@ $ pulsar-admin namespaces get-backlog-quotas test-tenant/ns1 admin.namespaces().getBacklogQuotaMap(namespace); ``` + @@ -475,11 +493,13 @@ It removes backlog quota policies for a given namespace ``` + $ pulsar-admin namespaces remove-backlog-quota test-tenant/ns1 ``` ``` + N/A ``` @@ -488,6 +508,7 @@ N/A ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/backlogQuota|operation/removeBacklogQuota?version=@pulsar:version_number@} ``` @@ -500,6 +521,7 @@ N/A admin.namespaces().removeBacklogQuota(namespace, backlogQuotaType) ``` + @@ -537,11 +559,13 @@ Persistence policies allow to configure persistency-level for all topic messages ``` + $ pulsar-admin namespaces set-persistence --bookkeeper-ack-quorum 2 --bookkeeper-ensemble 3 --bookkeeper-write-quorum 2 --ml-mark-delete-max-rate 0 test-tenant/ns1 ``` ``` + N/A ``` @@ -550,6 +574,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/setPersistence?version=@pulsar:version_number@} ``` @@ -562,6 +587,7 @@ N/A admin.namespaces().setPersistence(namespace,new PersistencePolicies(bookkeeperEnsemble, bookkeeperWriteQuorum,bookkeeperAckQuorum,managedLedgerMaxMarkDeleteRate)) ``` + @@ -589,6 +615,7 @@ It shows the configured persistence policies of a given namespace. ``` + $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` @@ -608,6 +635,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/persistence|operation/getPersistence?version=@pulsar:version_number@} ``` @@ -620,6 +648,7 @@ $ pulsar-admin namespaces get-persistence test-tenant/ns1 admin.namespaces().getPersistence(namespace) ``` + @@ -649,11 +678,13 @@ The namespace bundle is a virtual group of topics which belong to the same names ``` + $ pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` ``` + N/A ``` @@ -662,6 +693,7 @@ N/A ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/unload|operation/unloadNamespaceBundle?version=@pulsar:version_number@} ``` @@ -674,6 +706,7 @@ N/A admin.namespaces().unloadNamespaceBundle(namespace, bundle) ``` + @@ -702,11 +735,13 @@ If a single bundle is creating an excessive load on a broker, an admin splits th ``` + $ pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff test-tenant/ns1 ``` ``` + N/A ``` @@ -715,6 +750,7 @@ N/A ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/{bundle}/split|operation/splitNamespaceBundle?version=@pulsar:version_number@} ``` @@ -727,6 +763,7 @@ N/A admin.namespaces().splitNamespaceBundle(namespace, bundle) ``` + @@ -756,11 +793,13 @@ It configures message’s time to live (in seconds) duration. ``` + $ pulsar-admin namespaces set-message-ttl --messageTTL 100 test-tenant/ns1 ``` ``` + N/A ``` @@ -769,6 +808,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/setNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -781,6 +821,7 @@ N/A admin.namespaces().setNamespaceMessageTTL(namespace, messageTTL) ``` + @@ -808,11 +849,13 @@ It gives a message ttl of configured namespace. ``` + $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` ``` + 100 ``` @@ -821,6 +864,7 @@ $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/getNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -833,6 +877,7 @@ $ pulsar-admin namespaces get-message-ttl test-tenant/ns1 admin.namespaces().getNamespaceMessageTTL(namespace) ``` + @@ -860,11 +905,13 @@ Remove a message TTL of the configured namespace. ``` + $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` ``` + 100 ``` @@ -873,6 +920,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 ``` + {@inject: endpoint|DELETE|/admin/v2/namespaces/:tenant/:namespace/messageTTL|operation/removeNamespaceMessageTTL?version=@pulsar:version_number@} ``` @@ -885,6 +933,7 @@ $ pulsar-admin namespaces remove-message-ttl test-tenant/ns1 admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + @@ -915,11 +964,13 @@ It clears all message backlog for all the topics that belong to a specific names ``` + $ pulsar-admin namespaces clear-backlog --sub my-subscription test-tenant/ns1 ``` ``` + N/A ``` @@ -928,6 +979,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/clearBacklog|operation/clearNamespaceBacklogForSubscription?version=@pulsar:version_number@} ``` @@ -940,6 +992,7 @@ N/A admin.namespaces().clearNamespaceBacklogForSubscription(namespace, subscription) ``` + @@ -967,11 +1020,13 @@ It clears all message backlog for all the topics that belong to a specific Names ``` + $ pulsar-admin namespaces clear-backlog --bundle 0x00000000_0xffffffff --sub my-subscription test-tenant/ns1 ``` ``` + N/A ``` @@ -980,6 +1035,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/{bundle}/clearBacklog|operation?version=@pulsar:version_number@/clearNamespaceBundleBacklogForSubscription} ``` @@ -992,6 +1048,7 @@ N/A admin.namespaces().clearNamespaceBundleBacklogForSubscription(namespace, bundle, subscription) ``` + @@ -1021,11 +1078,13 @@ Each namespace contains multiple topics and the retention size (storage size) of ``` + $ pulsar-admin set-retention --size 100 --time 10 test-tenant/ns1 ``` ``` + N/A ``` @@ -1034,6 +1093,7 @@ N/A ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/retention|operation/setRetention?version=@pulsar:version_number@} ``` @@ -1046,6 +1106,7 @@ N/A admin.namespaces().setRetention(namespace, new RetentionPolicies(retentionTimeInMin, retentionSizeInMB)) ``` + @@ -1073,6 +1134,7 @@ It shows retention information of a given namespace. ``` + $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` @@ -1090,6 +1152,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/retention|operation/getRetention?version=@pulsar:version_number@} ``` @@ -1102,6 +1165,7 @@ $ pulsar-admin namespaces get-retention test-tenant/ns1 admin.namespaces().getRetention(namespace) ``` + @@ -1144,6 +1208,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1155,6 +1220,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1167,6 +1233,7 @@ $ pulsar-admin namespaces set-dispatch-rate test-tenant/ns1 \ admin.namespaces().setDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1194,6 +1261,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` @@ -1212,6 +1280,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/dispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1224,6 +1293,7 @@ $ pulsar-admin namespaces get-dispatch-rate test-tenant/ns1 admin.namespaces().getDispatchRate(namespace) ``` + @@ -1256,6 +1326,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1267,6 +1338,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1279,6 +1351,7 @@ $ pulsar-admin namespaces set-subscription-dispatch-rate test-tenant/ns1 \ admin.namespaces().setSubscriptionDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1306,6 +1379,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` @@ -1324,6 +1398,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/subscriptionDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1336,6 +1411,7 @@ $ pulsar-admin namespaces get-subscription-dispatch-rate test-tenant/ns1 admin.namespaces().getSubscriptionDispatchRate(namespace) ``` + @@ -1368,6 +1444,7 @@ disables the throttling. ``` + $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ --msg-dispatch-rate 1000 \ --byte-dispatch-rate 1048576 \ @@ -1379,6 +1456,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/setDispatchRate?version=@pulsar:version_number@} ``` @@ -1391,6 +1469,7 @@ $ pulsar-admin namespaces set-replicator-dispatch-rate test-tenant/ns1 \ admin.namespaces().setReplicatorDispatchRate(namespace, new DispatchRate(1000, 1048576, 1)) ``` + @@ -1418,6 +1497,7 @@ It shows configured message-rate for the namespace (topics under this namespace ``` + $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` @@ -1436,6 +1516,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/replicatorDispatchRate|operation/getDispatchRate?version=@pulsar:version_number@} ``` @@ -1448,6 +1529,7 @@ $ pulsar-admin namespaces get-replicator-dispatch-rate test-tenant/ns1 admin.namespaces().getReplicatorDispatchRate(namespace) ``` + @@ -1477,6 +1559,7 @@ It shows configured `deduplicationSnapshotInterval` for a namespace (Each topic ``` + $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1485,6 +1568,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|GET|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1497,6 +1581,7 @@ $ pulsar-admin namespaces get-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().getDeduplicationSnapshotInterval(namespace) ``` + @@ -1525,7 +1610,8 @@ Set configured `deduplicationSnapshotInterval` for a namespace. Each topic under ``` -$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 + +$ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 --interval 1000 ``` @@ -1533,6 +1619,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1553,6 +1640,7 @@ $ pulsar-admin namespaces set-deduplication-snapshot-interval test-tenant/ns1 -- admin.namespaces().setDeduplicationSnapshotInterval(namespace, 1000) ``` + @@ -1580,6 +1668,7 @@ Remove configured `deduplicationSnapshotInterval` of a namespace (Each topic und ``` + $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` @@ -1588,6 +1677,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 ``` + {@inject: endpoint|POST|/admin/v2/namespaces/:tenant/:namespace/deduplicationSnapshotInterval?version=@pulsar:version_number@} ``` @@ -1600,6 +1690,7 @@ $ pulsar-admin namespaces remove-deduplication-snapshot-interval test-tenant/ns1 admin.namespaces().removeDeduplicationSnapshotInterval(namespace) ``` + @@ -1644,6 +1735,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns ``` + {@inject: endpoint|PUT|/admin/v2/namespaces/:tenant/:namespace/unload|operation/unloadNamespace?version=@pulsar:version_number@} ``` @@ -1656,6 +1748,7 @@ $ pulsar-admin namespaces unload my-tenant/my-ns admin.namespaces().unload(namespace) ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-partitioned-topics.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-partitioned-topics.md index fba7ca0d827c6..232ce3468cc21 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-partitioned-topics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-partitioned-topics title: Managing non-partitioned topics -sidebar_label: Non-partitioned topics +sidebar_label: "Non-partitioned topics" original_id: admin-api-non-partitioned-topics --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-persistent-topics.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-persistent-topics.md index a36cc1cd9e64a..b468be2671780 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-persistent-topics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-non-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-non-persistent-topics title: Managing non-persistent topics -sidebar_label: Non-Persistent topics +sidebar_label: "Non-Persistent topics" original_id: admin-api-non-persistent-topics --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-overview.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-overview.md index f50ebbd2e9a8f..de9d304e65891 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-overview.md @@ -1,7 +1,7 @@ --- id: admin-api-overview title: Pulsar admin interface -sidebar_label: Overview +sidebar_label: "Overview" original_id: admin-api-overview --- @@ -17,13 +17,13 @@ You can interact with the admin interface via: - A Java client interface. - The `pulsar-admin` CLI tool, which is available in the `bin` folder of your Pulsar installation: - ```shell - - $ bin/pulsar-admin - - ``` + ```shell + + $ bin/pulsar-admin + + ``` - For complete commands of `pulsar-admin` tool, see [Pulsar admin snapshot](https://pulsar.apache.org/tools/pulsar-admin/). + For complete commands of `pulsar-admin` tool, see [Pulsar admin snapshot](https://pulsar.apache.org/tools/pulsar-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. @@ -111,6 +111,7 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + @@ -131,12 +132,10 @@ Kubernetes requires a name that can be used as a DNS subdomain name as defined i - Dots (.) - Replace beginning and ending non-alphanumeric characters with 0 - + :::tip - - If you get an error in translating Pulsar object names into Kubernetes resource labels (for example, you may have a naming collision if your Pulsar object name is too long) or want to customize the translating rules, see [customize Kubernetes runtime](https://pulsar.apache.org/docs/en/next/functions-runtime/#customize-kubernetes-runtime). - - For how to configure Kubernetes runtime, see [here](https://pulsar.apache.org/docs/en/next/functions-runtime/#configure-kubernetes-runtime). ::: diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-packages.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-packages.md index 8c662a791e750..9277d656b08b3 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-packages.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-packages.md @@ -1,7 +1,7 @@ --- id: admin-api-packages title: Manage packages -sidebar_label: Packages +sidebar_label: "Packages" original_id: admin-api-packages --- @@ -105,6 +105,7 @@ bin/pulsar-admin packages upload functions://public/default/example@v0.1 --path + Upload a package to the package management service synchronously. ```java @@ -112,6 +113,7 @@ Upload a package to the package management service synchronously. void upload(PackageMetadata metadata, String packageName, String path) throws PulsarAdminException; ``` + Upload a package to the package management service asynchronously. ```java @@ -119,6 +121,7 @@ Upload a package to the package management service asynchronously. CompletableFuture uploadAsync(PackageMetadata metadata, String packageName, String path); ``` + @@ -157,6 +160,7 @@ bin/pulsar-admin packages download functions://public/default/example@v0.1 --pat + Download a package to the package management service synchronously. ```java @@ -172,6 +176,7 @@ Download a package to the package management service asynchronously. CompletableFuture downloadAsync(String packageName, String path); ``` + @@ -209,6 +214,7 @@ bin/pulsar-admin packages list --type function public/default + List all versions of a package synchronously. ```java @@ -216,6 +222,7 @@ List all versions of a package synchronously. List listPackageVersions(String packageName) throws PulsarAdminException; ``` + List all versions of a package asynchronously. ```java @@ -223,6 +230,7 @@ List all versions of a package asynchronously. CompletableFuture> listPackageVersionsAsync(String packageName); ``` + @@ -260,6 +268,7 @@ bin/pulsar-admin packages list --type function public/default + List all the packages with the given type in a namespace synchronously. ```java @@ -267,6 +276,7 @@ List all the packages with the given type in a namespace synchronously. List listPackages(String type, String namespace) throws PulsarAdminException; ``` + List all the packages with the given type in a namespace asynchronously. ```java @@ -274,6 +284,7 @@ List all the packages with the given type in a namespace asynchronously. CompletableFuture> listPackagesAsync(String type, String namespace); ``` + @@ -312,6 +323,7 @@ bin/pulsar-admin packages get-metadata function://public/default/test@v1 + Get the metadata of a package synchronously. ```java @@ -319,6 +331,7 @@ Get the metadata of a package synchronously. PackageMetadata getMetadata(String packageName) throws PulsarAdminException; ``` + Get the metadata of a package asynchronously. ```java @@ -326,6 +339,7 @@ Get the metadata of a package asynchronously. CompletableFuture getMetadataAsync(String packageName); ``` + @@ -363,6 +377,7 @@ bin/pulsar-admin packages update-metadata function://public/default/example@v0.1 + Update a package metadata information synchronously. ```java @@ -370,6 +385,7 @@ Update a package metadata information synchronously. void updateMetadata(String packageName, PackageMetadata metadata) throws PulsarAdminException; ``` + Update a package metadata information asynchronously. ```java @@ -377,6 +393,7 @@ Update a package metadata information asynchronously. CompletableFuture updateMetadataAsync(String packageName, PackageMetadata metadata); ``` + @@ -401,6 +418,7 @@ You can delete a specified package with its package name in the following ways. } ]}> + The following command example deletes a package of version 0.1. ```shell @@ -416,6 +434,7 @@ bin/pulsar-admin packages delete functions://public/default/example@v0.1 + Delete a specified package synchronously. ```java @@ -423,6 +442,7 @@ Delete a specified package synchronously. void delete(String packageName) throws PulsarAdminException; ``` + Delete a specified package asynchronously. ```java @@ -430,6 +450,7 @@ Delete a specified package asynchronously. CompletableFuture deleteAsync(String packageName); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-partitioned-topics.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-partitioned-topics.md index f2d4e0b6646ec..f521bccf34c56 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-partitioned-topics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-partitioned-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-partitioned-topics title: Managing partitioned topics -sidebar_label: Partitioned topics +sidebar_label: "Partitioned topics" original_id: admin-api-partitioned-topics --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-permissions.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-permissions.md index beabbf4521c88..190122b9ed69f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-permissions.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-permissions.md @@ -1,7 +1,7 @@ --- id: admin-api-permissions title: Managing permissions -sidebar_label: Permissions +sidebar_label: "Permissions" original_id: admin-api-permissions --- @@ -96,6 +96,7 @@ Roles `my.1.role`, `my.2.role`, `my.foo.role`, `my.bar.role`, etc. **cannot** pr admin.namespaces().grantPermissionOnNamespace(namespace, role, getAuthActions(actions)); ``` + @@ -132,7 +133,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 "produce", "consume" ] -} +} ``` @@ -149,6 +150,7 @@ $ pulsar-admin namespaces permissions test-tenant/ns1 admin.namespaces().getPermissions(namespace); ``` + @@ -197,6 +199,7 @@ $ pulsar-admin namespaces revoke-permission test-tenant/ns1 \ admin.namespaces().revokePermissionsOnNamespace(namespace, role); ``` + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-persistent-topics.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-persistent-topics.md index ebd1d7385abf8..753781fc386b3 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-persistent-topics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-persistent-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-persistent-topics title: Managing persistent topics -sidebar_label: Persistent topics +sidebar_label: "Persistent topics" original_id: admin-api-persistent-topics --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-schemas.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-schemas.md index 9bf86b9aff322..cd56ead2f32ba 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-schemas.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-schemas.md @@ -1,7 +1,7 @@ --- id: admin-api-schemas title: Managing Schemas -sidebar_label: Schemas +sidebar_label: "Schemas" original_id: admin-api-schemas --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-tenants.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-tenants.md index 23c5070d46b74..82880d1305c2a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-tenants.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-tenants.md @@ -1,7 +1,7 @@ --- id: admin-api-tenants title: Managing Tenants -sidebar_label: Tenants +sidebar_label: "Tenants" original_id: admin-api-tenants --- @@ -61,6 +61,7 @@ my-tenant-2 admin.tenants().getTenants(); ``` + @@ -106,6 +107,7 @@ $ pulsar-admin tenants create my-tenant \ -r role1 ``` + @@ -163,6 +165,7 @@ $ pulsar-admin tenants get my-tenant } ``` + @@ -224,6 +227,7 @@ $ pulsar-admin tenants delete my-tenant admin.Tenants().deleteTenant(tenantName); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/admin-api-topics.md b/site2/website-next/versioned_docs/version-2.8.0/admin-api-topics.md index fdcdf367ea7b1..255f142bb4eb0 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/admin-api-topics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/admin-api-topics.md @@ -1,7 +1,7 @@ --- id: admin-api-topics title: Manage topics -sidebar_label: Topics +sidebar_label: "Topics" original_id: admin-api-topics --- @@ -24,6 +24,7 @@ Non-persistent topics are used in applications that only consume real-time publi non-persistent://tenant/namespace/topic ``` + ## Manage topic resources Whether it is persistent or non-persistent topic, you can obtain the topic resources through `pulsar-admin` tool, REST API and Java. @@ -77,6 +78,7 @@ String namespace = "my-tenant/my-namespace"; admin.topics().getList(namespace); ``` + @@ -472,6 +474,7 @@ The following is an example of a topic status. } ``` + To get the status of a topic, you can use the following ways. + When you create non-partitioned topics with the [`create`](reference-pulsar-admin.md#create-3) command, you need to specify the topic name as an argument. ```shell @@ -1248,6 +1253,7 @@ $ bin/pulsar-admin topics create \ persistent://my-tenant/my-namespace/my-topic ``` + :::note When you create a non-partitioned topic with the suffix '-partition-' followed by numeric value like 'xyz-topic-partition-x' for the topic name, if a partitioned topic with same suffix 'xyz-topic-partition-y' exists, then the numeric value(x) for the non-partitioned topic must be larger than the number of partitions(y) of the partitioned topic. Otherwise, you cannot create such a non-partitioned topic. @@ -1403,6 +1409,7 @@ You can check the current statistics of a given topic. The following is an examp } ``` + You can check the current statistics of a given topic and its connected producers and consumers in the following ways. @@ -1476,6 +1484,7 @@ You can create partitioned topics in the following ways. } ]}> + When you create partitioned topics with the [`create-partitioned-topic`](reference-pulsar-admin.md#create-partitioned-topic) command, you need to specify the topic name as an argument and the number of partitions using the `-p` or `--partitions` flag. @@ -1534,6 +1543,7 @@ When topic auto-creation is disabled, and you have a partitioned topic without a } ]}> + You can create missed partitions with the [`create-missed-partitions`](reference-pulsar-admin.md#create-missed-partitions) command and specify the topic name as an argument. ```shell @@ -1587,6 +1597,7 @@ Field | Description } ]}> + You can check the number of partitions in a partitioned topic with the [`get-partitioned-topic-metadata`](reference-pulsar-admin.md#get-partitioned-topic-metadata) subcommand. ```shell @@ -1641,6 +1652,7 @@ Producers and consumers can find the newly created partitions automatically. } ]}> + You can update partitioned topics with the [`update-partitioned-topic`](reference-pulsar-admin.md#update-partitioned-topic) command. ```shell @@ -2047,4 +2059,6 @@ If a message has a key, it supersedes the round robin routing policy. The follow return signSafeMod(PARTITION_INDEX_UPDATER.getAndIncrement(this), topicMetadata.numPartitions()); } -``` \ No newline at end of file +``` + + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-dashboard.md b/site2/website-next/versioned_docs/version-2.8.0/administration-dashboard.md index 0e31aa64d8d13..64f903045add5 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-dashboard.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-dashboard.md @@ -1,14 +1,13 @@ --- id: administration-dashboard title: Pulsar dashboard -sidebar_label: Dashboard +sidebar_label: "Dashboard" original_id: administration-dashboard --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; - :::note Pulsar dashboard is deprecated. If you want to manage and monitor the stats of your topics, use [Pulsar Manager](administration-pulsar-manager). @@ -55,6 +54,7 @@ $ docker run -p 80:80 \ apachepulsar/pulsar-dashboard ``` + You need to specify only one service URL for a Pulsar cluster. Internally, the collector figures out all the existing clusters and the brokers from where it needs to pull the metrics. If you connect the dashboard to Pulsar running in standalone mode, the URL is `http://:8080` by default. `` is the ip address or hostname of the machine running Pulsar standalone. The ip address or hostname should be accessible from the docker instance running dashboard. diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-geo.md b/site2/website-next/versioned_docs/version-2.8.0/administration-geo.md index 84fab917b44a0..d08d79caf65dd 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-geo.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-geo.md @@ -1,7 +1,7 @@ --- id: administration-geo title: Pulsar geo-replication -sidebar_label: Geo-replication +sidebar_label: "Geo-replication" original_id: administration-geo --- @@ -64,35 +64,34 @@ Suppose that you have 3 replication clusters: `us-west`, `us-cent`, and `us-east Run the following command on `us-west`. -```shell - -$ bin/pulsar-admin clusters create \ - --broker-url pulsar://: \ - --url http://: \ - us-east - -``` - -:::tip + ```shell + + $ bin/pulsar-admin clusters create \ + --broker-url pulsar://: \ + --url http://: \ + us-east + + ``` + :::tip -- If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). -- Different clusters may have different authentications. You can use the authentication flag `--auth-plugin` and `--auth-parameters` together to set cluster authentication, which overrides `brokerClientAuthenticationPlugin` and `brokerClientAuthenticationParameters` if `authenticationEnabled` sets to `true` in `broker.conf` and `standalone.conf`. For more information, see [authentication and authorization](concepts-authentication). + - If you want to use a secure connection for a cluster, you can use the flags `--broker-url-secure` and `--url-secure`. For more information, see [pulsar-admin clusters create](https://pulsar.apache.org/tools/pulsar-admin/). + - Different clusters may have different authentications. You can use the authentication flag `--auth-plugin` and `--auth-parameters` together to set cluster authentication, which overrides `brokerClientAuthenticationPlugin` and `brokerClientAuthenticationParameters` if `authenticationEnabled` sets to `true` in `broker.conf` and `standalone.conf`. For more information, see [authentication and authorization](concepts-authentication). -::: + ::: 2. Configure the connection from `us-west` to `us-cent`. Run the following command on `us-west`. -```shell - -$ bin/pulsar-admin clusters create \ - --broker-url pulsar://: \ - --url http://: \ - us-cent - -``` + ```shell + + $ bin/pulsar-admin clusters create \ + --broker-url pulsar://: \ + --url http://: \ + us-cent + + ``` 3. Run similar commands on `us-east` and `us-cent` to create connections among clusters. diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-isolation.md b/site2/website-next/versioned_docs/version-2.8.0/administration-isolation.md index 7ebaceaf4732a..3c5d521b7707c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-isolation.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-isolation.md @@ -1,7 +1,7 @@ --- id: administration-isolation title: Pulsar isolation -sidebar_label: Pulsar isolation +sidebar_label: "Pulsar isolation" original_id: administration-isolation --- @@ -39,6 +39,7 @@ You can set a namespace isolation policy for a cluster using one of the followin ``` + pulsar-admin ns-isolation-policy set options ``` @@ -97,6 +98,7 @@ You can set a bookie affinity group using one of the following methods. ``` + pulsar-admin namespaces set-bookie-affinity-group options ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-load-balance.md b/site2/website-next/versioned_docs/version-2.8.0/administration-load-balance.md index e611d13491a73..a14142bcac7de 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-load-balance.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-load-balance.md @@ -1,7 +1,7 @@ --- id: administration-load-balance title: Pulsar load balance -sidebar_label: Load balance +sidebar_label: "Load balance" original_id: administration-load-balance --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-proxy.md b/site2/website-next/versioned_docs/version-2.8.0/administration-proxy.md index f70cc6f28125e..821aa4d905a02 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-proxy.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-proxy.md @@ -1,7 +1,7 @@ --- id: administration-proxy title: Pulsar proxy -sidebar_label: Pulsar proxy +sidebar_label: "Pulsar proxy" original_id: administration-proxy --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-pulsar-manager.md b/site2/website-next/versioned_docs/version-2.8.0/administration-pulsar-manager.md index f354c254668fb..6c8945e9e9783 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-pulsar-manager.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-pulsar-manager.md @@ -1,7 +1,7 @@ --- id: administration-pulsar-manager title: Pulsar Manager -sidebar_label: Pulsar Manager +sidebar_label: "Pulsar Manager" original_id: administration-pulsar-manager --- @@ -36,20 +36,21 @@ docker run -it \ ### Set administrator account and password ```shell - -CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) -curl \ - -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ - -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ - -H "Content-Type: application/json" \ - -X PUT http://localhost:7750/pulsar-manager/users/superuser \ - -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' - -``` + + CSRF_TOKEN=$(curl http://localhost:7750/pulsar-manager/csrf-token) + curl \ + -H 'X-XSRF-TOKEN: $CSRF_TOKEN' \ + -H 'Cookie: XSRF-TOKEN=$CSRF_TOKEN;' \ + -H "Content-Type: application/json" \ + -X PUT http://localhost:7750/pulsar-manager/users/superuser \ + -d '{"name": "admin", "password": "apachepulsar", "description": "test", "email": "username@test.org"}' + + ``` You can find the docker image in the [Docker Hub](https://github.com/apache/pulsar-manager/tree/master/docker) directory and build an image from the source code as well: ``` + git clone https://github.com/apache/pulsar-manager cd pulsar-manager/front-end npm install --save @@ -70,6 +71,7 @@ If you have a large amount of data, you can use a custom database. The following 2. Modify the [configuration file](https://github.com/apache/pulsar-manager/blob/master/src/main/resources/application.properties) and add PostgreSQL configuration. ``` + spring.datasource.driver-class-name=org.postgresql.Driver spring.datasource.url=jdbc:postgresql://127.0.0.1:5432/pulsar_manager spring.datasource.username=postgres @@ -80,6 +82,7 @@ spring.datasource.password=postgres 3. Compile to generate a new executable jar package. ``` + ./gradlew build -x test ``` @@ -103,15 +106,17 @@ If you want to enable JWT authentication, use one of the following methods. * Method 1: use command-line tool ``` + wget https://dist.apache.org/repos/dist/release/pulsar/pulsar-manager/apache-pulsar-manager-0.2.0/apache-pulsar-manager-0.2.0-bin.tar.gz tar -zxvf apache-pulsar-manager-0.2.0-bin.tar.gz cd pulsar-manager tar -zxvf pulsar-manager.tar cd pulsar-manager cp -r ../dist ui -./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key +./bin/pulsar-manager --redirect.host=http://localhost --redirect.port=9527 insert.stats.interval=600000 --backend.jwt.token=token --jwt.broker.token.mode=PRIVATE --jwt.broker.private.key=file:///path/broker-private.key --jwt.broker.public.key=file:///path/broker-public.key ``` + Firstly, [set the administrator account and password](#set-administrator-account-and-password) Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. @@ -119,6 +124,7 @@ Secondly, log in to Pulsar manager through http://localhost:7750/ui/index.html. * Method 2: configure the application.properties file ``` + backend.jwt.token=token jwt.broker.token.mode=PRIVATE @@ -134,6 +140,7 @@ jwt.broker.secret.key=file:///path/broker-secret.key * Method 3: use Docker and enable token authentication. ``` + export JWT_TOKEN="your-token" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -v $PWD:/data apachepulsar/pulsar-manager:v0.2.0 /bin/sh @@ -151,6 +158,7 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 4: use Docker and turn on **token authentication** and **token management** by private key and public key. ``` + export JWT_TOKEN="your-token" export PRIVATE_KEY="file:///pulsar-manager/secret/my-private.key" export PUBLIC_KEY="file:///pulsar-manager/secret/my-public.key" @@ -173,6 +181,7 @@ docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e RE * Method 5: use Docker and turn on **token authentication** and **token management** by secret key. ``` + export JWT_TOKEN="your-token" export SECRET_KEY="file:///pulsar-manager/secret/my-secret.key" docker run -it -p 9527:9527 -p 7750:7750 -e REDIRECT_HOST=http://localhost -e REDIRECT_PORT=9527 -e DRIVER_CLASS_NAME=org.postgresql.Driver -e URL='jdbc:postgresql://127.0.0.1:5432/pulsar_manager' -e USERNAME=pulsar -e PASSWORD=pulsar -e LOG_LEVEL=DEBUG -e JWT_TOKEN=$JWT_TOKEN -e SECRET_KEY=$SECRET_KEY -v $PWD:/data -v $PWD/secret:/pulsar-manager/secret apachepulsar/pulsar-manager:v0.2.0 /bin/sh diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-stats.md b/site2/website-next/versioned_docs/version-2.8.0/administration-stats.md index fade1697d83cc..029ebf2d1d8f8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-stats.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-stats.md @@ -1,7 +1,7 @@ --- id: administration-stats title: Pulsar stats -sidebar_label: Pulsar statistics +sidebar_label: "Pulsar statistics" original_id: administration-stats --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-upgrade.md b/site2/website-next/versioned_docs/version-2.8.0/administration-upgrade.md index 78b290bb2bfb4..aefdd8125ddcc 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-upgrade.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-upgrade.md @@ -1,7 +1,7 @@ --- id: administration-upgrade title: Upgrade Guide -sidebar_label: Upgrade +sidebar_label: "Upgrade" original_id: administration-upgrade --- @@ -35,21 +35,24 @@ To upgrade an Apache Pulsar cluster, follow the upgrade sequence. 2. Upgrade bookies - Canary test: test an upgraded version in one or a small set of bookies. - Rolling upgrade: - - a. Disable `autorecovery` with the following command. + - a. Disable `autorecovery` with the following command. - ```shell + ```shell + + bin/bookkeeper shell autorecovery -disable + + ``` - bin/bookkeeper shell autorecovery -disable + + - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. + - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. - ``` - - b. Rollout the upgraded version to all bookies in the cluster after you determine that a version is safe after canary. - - c. After you upgrade all bookies, re-enable `autorecovery` with the following command. + ```shell + + bin/bookkeeper shell autorecovery -enable + + ``` - ```shell - - bin/bookkeeper shell autorecovery -enable - - ``` 3. Upgrade brokers - Canary test: test an upgraded version in one or a small set of brokers. - Rolling upgrade: rollout the upgraded version to all brokers in the cluster after you determine that a version is safe after canary. @@ -98,17 +101,19 @@ To upgrade bookie to a new version, complete the following steps: 3. Start the bookie in `ReadOnly` mode to verify if the bookie of this new version runs well for read workload. ```shell - + bin/pulsar bookie --readOnly - + ``` + 4. When the bookie runs successfully in `ReadOnly` mode, stop the bookie and restart it in `Write/Read` mode. ```shell - + bin/pulsar bookie - + ``` + 5. Observe and make sure the cluster serves both write and read traffic. #### Canary rollback diff --git a/site2/website-next/versioned_docs/version-2.8.0/administration-zk-bk.md b/site2/website-next/versioned_docs/version-2.8.0/administration-zk-bk.md index ced1ead37fa7d..a4da422aaa960 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/administration-zk-bk.md +++ b/site2/website-next/versioned_docs/version-2.8.0/administration-zk-bk.md @@ -1,7 +1,7 @@ --- id: administration-zk-bk title: ZooKeeper and BookKeeper administration -sidebar_label: ZooKeeper and BookKeeper +sidebar_label: "ZooKeeper and BookKeeper" original_id: administration-zk-bk --- @@ -100,6 +100,7 @@ Again, given the very low expected load on the configuration store servers, you For example, you can assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also you can assume, each cluster has its own local ZK servers named such as ``` + zk[1-3].${CLUSTER}.example.com ``` @@ -269,9 +270,9 @@ And then you can decommission bookies safely. To decommission bookies, complete 3. Run the decommission command. - If you have logged in to the node to be decommissioned, you do not need to provide `-bookieid`. - If you are running the decommission command for the target bookie node from another bookie node, you should mention the target bookie ID in the arguments for `-bookieid` - `$ bin/bookkeeper shell decommissionbookie` - or - `$ bin/bookkeeper shell decommissionbookie -bookieid ` + `$ bin/bookkeeper shell decommissionbookie` + or + `$ bin/bookkeeper shell decommissionbookie -bookieid ` 4. Validate that no ledgers are on the decommissioned bookie. `$ bin/bookkeeper shell listledgers -bookieid ` diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cgo.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cgo.md index 47448a0ef1070..b7e216f8f9985 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cgo.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cgo.md @@ -1,7 +1,7 @@ --- id: client-libraries-cgo title: Pulsar CGo client -sidebar_label: CGo(deprecated) +sidebar_label: "CGo(deprecated)" original_id: client-libraries-cgo --- @@ -579,4 +579,5 @@ fmt.Println(s.ID) // output: 100 fmt.Println(s.Name) // output: pulsar defer consumer.Close() -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cpp.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cpp.md index 67f198e0f8c0c..8fe06e107989c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cpp.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-cpp.md @@ -1,7 +1,7 @@ --- id: client-libraries-cpp title: Pulsar C++ client -sidebar_label: C++ +sidebar_label: "C++" original_id: client-libraries-cpp --- @@ -128,6 +128,7 @@ These libraries rely on some other libraries. If you want to get detailed versio g++ --std=c++11 PulsarTest.cpp -o test /usr/lib/libpulsarwithdeps.a -lssl -lcrypto -ldl -lpthread -I/usr/local/ssl/include -L/usr/local/ssl/lib ``` + The `libpulsarwithdeps.a` does not include library openssl related libraries `libssl` and `libcrypto`, because these two libraries are related to security. It is more reasonable and easier to use the versions provided by the local system to handle security issues and upgrade libraries. ### Install RPM @@ -305,6 +306,7 @@ pulsar+ssl://pulsar.us-west.example.com:6651 To use Pulsar as a consumer, you need to create a consumer on the C++ client. The following is an example. ```c++ + Client client("pulsar://localhost:6650"); Consumer consumer; @@ -333,6 +335,7 @@ client.close(); To use Pulsar as a producer, you need to create a producer on the C++ client. The following is an example. ```c++ + Client client("pulsar://localhost:6650"); Producer producer; @@ -405,3 +408,4 @@ consumerConf.setSchema(SchemaInfo(AVRO, "Avro", exampleSchema)); client.subscribe("topic-avro", "sub-2", consumerConf, consumer) ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-dotnet.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-dotnet.md index 8fe67bd908361..68c58105d1f18 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-dotnet.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-dotnet.md @@ -1,7 +1,7 @@ --- id: client-libraries-dotnet title: Pulsar C# client -sidebar_label: C# +sidebar_label: "C#" original_id: client-libraries-dotnet --- @@ -31,10 +31,11 @@ To install the Pulsar C# client library, following these steps: 3. Create the project using the following command. - ``` - dotnet new console - - ``` + ``` + + dotnet new console + + ``` 4. Use `dotnet run` to test that the app has been created properly. @@ -42,20 +43,21 @@ To install the Pulsar C# client library, following these steps: 1. Use the following command to install the `DotPulsar` package. - ``` - dotnet add package DotPulsar - - ``` + ``` + + dotnet add package DotPulsar + + ``` 2. After the command completes, open the `.csproj` file to see the added reference. - ```xml - - - - - - ``` + ```xml + + + + + + ``` ## Client @@ -66,6 +68,7 @@ This section describes some configuration examples for the Pulsar C# client. This example shows how to create a Pulsar C# client connected to localhost. ```c# + var client = PulsarClient.Builder().Build(); ``` @@ -83,20 +86,22 @@ This section describes how to create a producer. - Create a producer by using the builder. - ```c# - var producer = client.NewProducer() - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var producer = client.NewProducer() + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a producer without using the builder. - ```c# - var options = new ProducerOptions("persistent://public/default/mytopic"); - var producer = client.CreateProducer(options); - - ``` + ```c# + + var options = new ProducerOptions("persistent://public/default/mytopic"); + var producer = client.CreateProducer(options); + + ``` ### Create consumer @@ -104,21 +109,23 @@ This section describes how to create a consumer. - Create a consumer by using the builder. - ```c# - var consumer = client.NewConsumer() - .SubscriptionName("MySubscription") - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var consumer = client.NewConsumer() + .SubscriptionName("MySubscription") + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a consumer without using the builder. - ```c# - var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); - var consumer = client.CreateConsumer(options); - - ``` + ```c# + + var options = new ConsumerOptions("MySubscription", "persistent://public/default/mytopic"); + var consumer = client.CreateConsumer(options); + + ``` ### Create reader @@ -126,21 +133,23 @@ This section describes how to create a reader. - Create a reader by using the builder. - ```c# - var reader = client.NewReader() - .StartMessageId(MessageId.Earliest) - .Topic("persistent://public/default/mytopic") - .Create(); - - ``` + ```c# + + var reader = client.NewReader() + .StartMessageId(MessageId.Earliest) + .Topic("persistent://public/default/mytopic") + .Create(); + + ``` - Create a reader without using the builder. - ```c# - var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); - var reader = client.CreateReader(options); - - ``` + ```c# + + var options = new ReaderOptions(MessageId.Earliest, "persistent://public/default/mytopic"); + var reader = client.CreateReader(options); + + ``` ### Configure encryption policies @@ -154,6 +163,7 @@ The Pulsar C# client supports four kinds of encryption policies: This example shows how to set the `EnforceUnencrypted` encryption policy. ```c# + var client = PulsarClient.Builder() .ConnectionSecurity(EncryptionPolicy.EnforceEncrypted) .Build(); @@ -168,20 +178,22 @@ If you have followed [Authentication using TLS](security-tls-authentication), yo 1. Create an unencrypted and password-less pfx file. - ```c# - openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: - - ``` + ```c# + + openssl pkcs12 -export -keypbe NONE -certpbe NONE -out admin.pfx -inkey admin.key.pem -in admin.cert.pem -passout pass: + + ``` 2. Use the admin.pfx file to create an X509Certificate2 and pass it to the Pulsar C# client. - ```c# - var clientCertificate = new X509Certificate2("admin.pfx"); - var client = PulsarClient.Builder() - .AuthenticateUsingClientCertificate(clientCertificate) - .Build(); - - ``` + ```c# + + var clientCertificate = new X509Certificate2("admin.pfx"); + var client = PulsarClient.Builder() + .AuthenticateUsingClientCertificate(clientCertificate) + .Build(); + + ``` ## Producer @@ -192,6 +204,7 @@ A producer is a process that attaches to a topic and publishes messages to a Pul This example shows how to send data. ```c# + var data = Encoding.UTF8.GetBytes("Hello World"); await producer.Send(data); @@ -201,23 +214,25 @@ await producer.Send(data); - Send messages with customized metadata by using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var messageId = await producer.NewMessage() - .Property("SomeKey", "SomeValue") - .Send(data); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var messageId = await producer.NewMessage() + .Property("SomeKey", "SomeValue") + .Send(data); + + ``` - Send messages with customized metadata without using the builder. - ```c# - var data = Encoding.UTF8.GetBytes("Hello World"); - var metadata = new MessageMetadata(); - metadata["SomeKey"] = "SomeValue"; - var messageId = await producer.Send(metadata, data)); - - ``` + ```c# + + var data = Encoding.UTF8.GetBytes("Hello World"); + var metadata = new MessageMetadata(); + metadata["SomeKey"] = "SomeValue"; + var messageId = await producer.Send(metadata, data)); + + ``` ## Consumer @@ -228,6 +243,7 @@ A consumer is a process that attaches to a topic through a subscription and then This example shows how a consumer receives messages from a topic. ```c# + await foreach (var message in consumer.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -241,26 +257,29 @@ Messages can be acknowledged individually or cumulatively. For details about mes - Acknowledge messages individually. - ```c# - await foreach (var message in consumer.Messages()) - { - Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); - } - - ``` + ```c# + + await foreach (var message in consumer.Messages()) + { + Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); + } + + ``` - Acknowledge messages cumulatively. - ```c# - await consumer.AcknowledgeCumulative(message); - - ``` + ```c# + + await consumer.AcknowledgeCumulative(message); + + ``` ### Unsubscribe from topics This example shows how a consumer unsubscribes from a topic. ```c# + await consumer.Unsubscribe(); ``` @@ -276,6 +295,7 @@ A reader is actually just a consumer without a cursor. This means that Pulsar do This example shows how a reader receives messages. ```c# + await foreach (var message in reader.Messages()) { Console.WriteLine("Received: " + Encoding.UTF8.GetString(message.Data.ToArray())); @@ -301,6 +321,7 @@ The following table lists states available for the producer. This example shows how to monitor the producer state. ```c# + private static async ValueTask Monitor(IProducer producer, CancellationToken cancellationToken) { var state = ProducerState.Disconnected; @@ -343,6 +364,7 @@ The following table lists states available for the consumer. This example shows how to monitor the consumer state. ```c# + private static async ValueTask Monitor(IConsumer consumer, CancellationToken cancellationToken) { var state = ConsumerState.Disconnected; @@ -386,6 +408,7 @@ The following table lists states available for the reader. This example shows how to monitor the reader state. ```c# + private static async ValueTask Monitor(IReader reader, CancellationToken cancellationToken) { var state = ReaderState.Disconnected; @@ -412,3 +435,4 @@ private static async ValueTask Monitor(IReader reader, CancellationToken cancell } ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-go.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-go.md index 1ea06dc09c4c9..dfd6ddcb8a3cb 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-go.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-go.md @@ -1,7 +1,7 @@ --- id: client-libraries-go title: Pulsar Go client -sidebar_label: Go +sidebar_label: "Go" original_id: client-libraries-go --- @@ -52,6 +52,7 @@ pulsar://localhost:6650 If you have multiple brokers, you can set the URL as below. ``` + pulsar://localhost:6550,localhost:6651,localhost:6652 ``` @@ -103,6 +104,7 @@ func main() { If you have multiple brokers, you can initiate a client object as below. ``` + import ( "log" "time" @@ -883,3 +885,4 @@ client, err := pulsar.NewClient(pulsar.ClientOptions{ }) ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-java.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-java.md index dfaf6632c7cbd..2a939a18822ab 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-java.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-java.md @@ -1,7 +1,7 @@ --- id: client-libraries-java title: Pulsar Java client -sidebar_label: Java +sidebar_label: "Java" original_id: client-libraries-java --- @@ -176,22 +176,29 @@ stringProducer.send("My message"); ``` > Make sure that you close your producers, consumers, and clients when you do not need them. + > ```java -> +> > producer.close(); > consumer.close(); > client.close(); +> +> > ``` + > > Close operations can also be asynchronous: + > ```java -> +> > producer.closeAsync() > .thenRun(() -> System.out.println("Producer closed")) > .exceptionally((ex) -> { > System.err.println("Failed to close producer: " + ex); > return null; > }); +> +> > ``` ### Configure producer @@ -403,9 +410,7 @@ consumer.acknowledge(messages) :::note - Batch receive policy limits the number and bytes of messages in a single batch. You can specify a timeout to wait for enough messages. - The batch receive is completed if any of the following condition is met: enough number of messages, bytes of messages, wait timeout. ```java @@ -421,6 +426,7 @@ Consumer consumer = client.newConsumer() .subscribe(); ``` + The default batch receive policy is: ```java @@ -482,7 +488,6 @@ pulsarClient.newConsumer() :::note - By default, the `subscriptionTopicsMode` of the consumer is `PersistentOnly`. Available options of `subscriptionTopicsMode` are `PersistentOnly`, `NonPersistentOnly`, and `AllTopics`. ::: @@ -579,7 +584,6 @@ Only the first consumer is allowed to the subscription, other consumers receive :::note - If topic is a partitioned topic, the first consumer subscribes to all partitioned topics, other consumers are not assigned with partitions and receive an error. ::: @@ -603,8 +607,6 @@ Consumer consumer2 = client.newConsumer() //conumser1 is the active consumer, consumer2 is the standby consumer. //consumer1 receives 5 messages and then crashes, consumer2 takes over as an active consumer. - - ``` Multiple consumers can attach to the same subscription, yet only the first consumer is active, and others are standby. When the active consumer is disconnected, messages will be dispatched to one of standby consumers, and the standby consumer then becomes active consumer. @@ -612,6 +614,7 @@ Multiple consumers can attach to the same subscription, yet only the first consu If the first active consumer is disconnected after receiving 5 messages, the standby consumer becomes active consumer. Consumer1 will receive: ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -623,6 +626,7 @@ If the first active consumer is disconnected after receiving 5 messages, the sta consumer2 will receive: ``` + ("key-2", "message-2-3") ("key-3", "message-3-1") ("key-3", "message-3-2") @@ -633,7 +637,6 @@ consumer2 will receive: :::note - If a topic is a partitioned topic, each partition has only one active consumer, messages of one partition are distributed to only one consumer, and messages of multiple partitions are distributed to multiple consumers. ::: @@ -664,6 +667,7 @@ In shared subscription mode, multiple consumers can attach to the same subscript If a broker dispatches only one message at a time, consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-3") ("key-2", "message-2-2") @@ -675,6 +679,7 @@ If a broker dispatches only one message at a time, consumer1 receives the follow consumer2 receives the following information. ``` + ("key-1", "message-1-2") ("key-2", "message-2-1") ("key-2", "message-2-3") @@ -711,6 +716,7 @@ Consumer consumer2 = client.newConsumer() consumer1 receives the following information. ``` + ("key-1", "message-1-1") ("key-1", "message-1-2") ("key-1", "message-1-3") @@ -722,6 +728,7 @@ consumer1 receives the following information. consumer2 receives the following information. ``` + ("key-2", "message-2-1") ("key-2", "message-2-2") ("key-2", "message-2-3") @@ -740,6 +747,7 @@ Producer producer = client.newProducer() .create(); ``` + Or the producer can disable batching. ```java @@ -750,8 +758,8 @@ Producer producer = client.newProducer() .create(); ``` -:::note +:::note If the message key is not specified, messages without key are dispatched to one consumer in order by default. @@ -874,62 +882,61 @@ The following schema formats are currently available for Java: * No schema or the byte array schema (which can be applied using `Schema.BYTES`): ```java - + Producer bytesProducer = client.newProducer(Schema.BYTES) - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` Or, equivalently: ```java - + Producer bytesProducer = client.newProducer() - .topic("some-raw-bytes-topic") - .create(); - + .topic("some-raw-bytes-topic") + .create(); + ``` * `String` for normal UTF-8-encoded string data. Apply the schema using `Schema.STRING`: ```java - + Producer stringProducer = client.newProducer(Schema.STRING) - .topic("some-string-topic") - .create(); - + .topic("some-string-topic") + .create(); + ``` * Create JSON schemas for POJOs using `Schema.JSON`. The following is an example. ```java - + Producer pojoProducer = client.newProducer(Schema.JSON(MyPojo.class)) - .topic("some-pojo-topic") - .create(); - + .topic("some-pojo-topic") + .create(); + ``` * Generate Protobuf schemas using `Schema.PROTOBUF`. The following example shows how to create the Protobuf schema and use it to instantiate a new producer: ```java - + Producer protobufProducer = client.newProducer(Schema.PROTOBUF(MyProtobuf.class)) - .topic("some-protobuf-topic") - .create(); - + .topic("some-protobuf-topic") + .create(); + ``` * Define Avro schemas with `Schema.AVRO`. The following code snippet demonstrates how to create and use Avro schema. - ```java - + Producer avroProducer = client.newProducer(Schema.AVRO(MyAvro.class)) - .topic("some-avro-topic") - .create(); - + .topic("some-avro-topic") + .create(); + ``` ### ProtobufNativeSchema example @@ -1030,3 +1037,4 @@ PulsarClient client = PulsarClient.builder() .build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-node.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-node.md index b1ef130b8b8c8..c1163da07e36a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-node.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-node.md @@ -1,7 +1,7 @@ --- id: client-libraries-node title: The Pulsar Node.js client -sidebar_label: Node.js +sidebar_label: "Node.js" original_id: client-libraries-node --- @@ -47,7 +47,6 @@ $ npm install pulsar-client :::note - Also, this library works only in Node.js 10.x or later because it uses the [`node-addon-api`](https://github.com/nodejs/node-addon-api) module to wrap the C++ library. ::: @@ -126,7 +125,7 @@ Here is an example: ```JavaScript const producer = await client.createProducer({ - topic: 'my-topic', // or 'my-tenant/my-namespace/my-topic' to specify topic's tenant and namespace + topic: 'my-topic', }); await producer.send({ @@ -480,6 +479,7 @@ The following static methods are available for the message id object: If you want to use the end-to-end encryption feature in the Node.js client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -497,148 +497,151 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** + **Input** - ```shell - - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem - - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://localhost:6650', - operationTimeoutSeconds: 30, - }); - - // Create a producer - const producer = await client.createProducer({ - topic: 'persistent://public/default/my-topic', - sendTimeoutMs: 30000, - batchingEnabled: true, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem", - encryptionKey: "encryption-key" - }); - - console.log(producer.ProducerConfig) - // Send messages - for (let i = 0; i < 10; i += 1) { - const msg = `my-message-${i}`; - producer.send({ - data: Buffer.from(msg), - }); - console.log(`Sent message: ${msg}`); - } - await producer.flush(); - - await producer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://localhost:6650', + operationTimeoutSeconds: 30, + }); + + // Create a producer + const producer = await client.createProducer({ + topic: 'persistent://public/default/my-topic', + sendTimeoutMs: 30000, + batchingEnabled: true, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem", + encryptionKey: "encryption-key" + }); + + console.log(producer.ProducerConfig) + // Send messages + for (let i = 0; i < 10; i += 1) { + const msg = `my-message-${i}`; + producer.send({ + data: Buffer.from(msg), + }); + console.log(`Sent message: ${msg}`); + } + await producer.flush(); + + await producer.close(); + await client.close(); + })(); + + ``` 3. Create a consumer to receive encrypted messages. - **Input** - - ```nodejs - - const Pulsar = require('pulsar-client'); - - (async () => { - // Create a client - const client = new Pulsar.Client({ - serviceUrl: 'pulsar://172.25.0.3:6650', - operationTimeoutSeconds: 30 - }); - - // Create a consumer - const consumer = await client.subscribe({ - topic: 'persistent://public/default/my-topic', - subscription: 'sub1', - subscriptionType: 'Shared', - ackTimeoutMs: 10000, - publicKeyPath: "./public.pem", - privateKeyPath: "./private.pem" - }); - - console.log(consumer) - // Receive messages - for (let i = 0; i < 10; i += 1) { - const msg = await consumer.receive(); - console.log(msg.getData().toString()); - consumer.acknowledge(msg); - } - - await consumer.close(); - await client.close(); - })(); - - ``` + **Input** + + ```nodejs + + const Pulsar = require('pulsar-client'); + + (async () => { + // Create a client + const client = new Pulsar.Client({ + serviceUrl: 'pulsar://172.25.0.3:6650', + operationTimeoutSeconds: 30 + }); + + // Create a consumer + const consumer = await client.subscribe({ + topic: 'persistent://public/default/my-topic', + subscription: 'sub1', + subscriptionType: 'Shared', + ackTimeoutMs: 10000, + publicKeyPath: "./public.pem", + privateKeyPath: "./private.pem" + }); + + console.log(consumer) + // Receive messages + for (let i = 0; i < 10; i += 1) { + const msg = await consumer.receive(); + console.log(msg.getData().toString()); + consumer.acknowledge(msg); + } + + await consumer.close(); + await client.close(); + })(); + + ``` 4. Run the consumer to receive encrypted messages. - **Input** + **Input** - ```shell - - node consumer.js - - ``` + ```shell + + node consumer.js + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - node producer.js - - ``` - - Now you can see the producer sends messages and the consumer receives messages successfully. - - **Output** - - This is from the producer side. - - ``` - Sent message: my-message-0 - Sent message: my-message-1 - Sent message: my-message-2 - Sent message: my-message-3 - Sent message: my-message-4 - Sent message: my-message-5 - Sent message: my-message-6 - Sent message: my-message-7 - Sent message: my-message-8 - Sent message: my-message-9 - - ``` - - This is from the consumer side. - - ``` - my-message-0 - my-message-1 - my-message-2 - my-message-3 - my-message-4 - my-message-5 - my-message-6 - my-message-7 - my-message-8 - my-message-9 + **Input** + + ```shell + + node producer.js + + ``` + + Now you can see the producer sends messages and the consumer receives messages successfully. + + **Output** + + This is from the producer side. + + ``` + + Sent message: my-message-0 + Sent message: my-message-1 + Sent message: my-message-2 + Sent message: my-message-3 + Sent message: my-message-4 + Sent message: my-message-5 + Sent message: my-message-6 + Sent message: my-message-7 + Sent message: my-message-8 + Sent message: my-message-9 + + ``` + + This is from the consumer side. + + ``` + + my-message-0 + my-message-1 + my-message-2 + my-message-3 + my-message-4 + my-message-5 + my-message-6 + my-message-7 + my-message-8 + my-message-9 + + ``` - ``` \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-python.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-python.md index 4162d92995821..bf713ef9903df 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-python.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-python.md @@ -1,7 +1,7 @@ --- id: client-libraries-python title: Pulsar Python client -sidebar_label: Python +sidebar_label: "Python" original_id: client-libraries-python --- @@ -166,6 +166,7 @@ while True: # No acknowledgment ``` + ### Multi-topic subscriptions In addition to subscribing a consumer to a single Pulsar topic, you can also subscribe to multiple topics simultaneously. To use multi-topic subscriptions, you can supply a regular expression (regex) or a `List` of topics. If you select topics via regex, all topics must be within the same Pulsar namespace. @@ -350,6 +351,7 @@ class Example(Record): To use the end-to-end encryption feature in the Python client, you need to configure `publicKeyPath` and `privateKeyPath` for both producer and consumer. ``` + publicKeyPath: "./public.pem" privateKeyPath: "./private.pem" @@ -367,89 +369,92 @@ This section provides step-by-step instructions on how to use the end-to-end enc 1. Create both public and private key pairs. - **Input** - - ```shell - - openssl genrsa -out private.pem 2048 - openssl rsa -in private.pem -pubout -out public.pem + **Input** - ``` + ```shell + + openssl genrsa -out private.pem 2048 + openssl rsa -in private.pem -pubout -out public.pem + + ``` 2. Create a producer to send encrypted messages. - **Input** + **Input** - ```python + ```python + + import pulsar - import pulsar - - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) - producer.send('encryption message'.encode('utf8')) - print('sent message') - producer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + producer = client.create_producer(topic='encryption', encryption_key='encryption', crypto_key_reader=crypto_key_reader) + producer.send('encryption message'.encode('utf8')) + print('sent message') + producer.close() + client.close() + + ``` 3. Create a consumer to receive encrypted messages. - **Input** - - ```python + **Input** - import pulsar + ```python + + import pulsar - publicKeyPath = "./public.pem" - privateKeyPath = "./private.pem" - crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) - msg = consumer.receive() - print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) - consumer.close() - client.close() - - ``` + publicKeyPath = "./public.pem" + privateKeyPath = "./private.pem" + crypto_key_reader = pulsar.CryptoKeyReader(publicKeyPath, privateKeyPath) + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe(topic='encryption', subscription_name='encryption-sub', crypto_key_reader=crypto_key_reader) + msg = consumer.receive() + print("Received msg '{}' id = '{}'".format(msg.data(), msg.message_id())) + consumer.close() + client.close() + + ``` 4. Run the consumer to receive encrypted messages. - **Input** - - ```shell - - python consumer.py + **Input** - ``` + ```shell + + python consumer.py + + ``` 5. In a new terminal tab, run the producer to produce encrypted messages. - **Input** - - ```shell - - python producer.py - - ``` + **Input** - Now you can see the producer sends messages and the consumer receives messages successfully. + ```shell + + python producer.py + + ``` - **Output** + Now you can see the producer sends messages and the consumer receives messages successfully. - This is from the producer side. + **Output** - ``` - sent message + This is from the producer side. - ``` + ``` + + sent message + + ``` - This is from the consumer side. + This is from the consumer side. - ``` - Received msg 'b'encryption message'' id = '(0,0,-1,-1)' + ``` + + Received msg 'b'encryption message'' id = '(0,0,-1,-1)' + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-websocket.md b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-websocket.md index 69eca1c990929..ec76079e54ccb 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/client-libraries-websocket.md +++ b/site2/website-next/versioned_docs/version-2.8.0/client-libraries-websocket.md @@ -1,7 +1,7 @@ --- id: client-libraries-websocket title: Pulsar WebSocket API -sidebar_label: WebSocket +sidebar_label: "WebSocket" original_id: client-libraries-websocket --- @@ -99,7 +99,7 @@ The producer endpoint requires you to specify a tenant, namespace, and topic in ```http -ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic +ws://broker-service-url:8080/ws/v2/producer/persistent/:tenant/:namespace/:topic ``` @@ -152,6 +152,7 @@ Key | Type | Required? | Explanation } ``` + ##### Example failure response ```json @@ -621,3 +622,4 @@ ws.on('message', function(message) { }); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-architecture-overview.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-architecture-overview.md index e0741fa00da5c..a98e03ba3454d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-architecture-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-architecture-overview.md @@ -1,7 +1,7 @@ --- id: concepts-architecture-overview title: Architecture Overview -sidebar_label: Architecture +sidebar_label: "Architecture" original_id: concepts-architecture-overview --- @@ -173,3 +173,4 @@ client = Client('pulsar://pulsar-cluster.acme.com:6650') In Pulsar, each topic is handled by only one broker. Initial requests from a client to read, update or delete a topic are sent to a broker that may not be the topic owner. If the broker cannot handle the request for this topic, it redirects the request to the appropriate broker. ::: + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-authentication.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-authentication.md index e2eaca517fbdf..5580a0b019fb4 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-authentication.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-authentication.md @@ -1,7 +1,7 @@ --- id: concepts-authentication title: Authentication and Authorization -sidebar_label: Authentication and Authorization +sidebar_label: "Authentication and Authorization" original_id: concepts-authentication --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-clients.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-clients.md index c61a821cf282a..36bef54264652 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-clients.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-clients.md @@ -1,7 +1,7 @@ --- id: concepts-clients title: Pulsar Clients -sidebar_label: Clients +sidebar_label: "Clients" original_id: concepts-clients --- @@ -93,3 +93,4 @@ Reader reader = pulsarClient.newReader() .create(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-messaging.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-messaging.md index 944232f58dbc5..be921afd93b7f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-messaging.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-messaging.md @@ -1,7 +1,7 @@ --- id: concepts-messaging title: Messaging -sidebar_label: Messaging +sidebar_label: "Messaging" original_id: concepts-messaging --- @@ -32,21 +32,22 @@ The default size of a message is 5 MB. You can configure the max size of a messa - In the `broker.conf` file. - ```bash - - # The max size of a message (in bytes). - maxMessageSize=5242880 - - ``` + ```bash + + # The max size of a message (in bytes). + maxMessageSize=5242880 + + ``` - In the `bookkeeper.conf` file. - ```bash - - # The max size of the netty frame (in bytes). Any messages received larger than this value are rejected. The default value is 5 MB. - nettyMaxFrameSizeBytes=5253120 + ```bash + + # The max size of the netty frame (in bytes). Any messages received larger than this value are rejected. The default value is 5 MB. + nettyMaxFrameSizeBytes=5253120 + + ``` - ``` > For more information on Pulsar message contents, see Pulsar [binary protocol](developing-binary-protocol). ## Producers @@ -74,9 +75,7 @@ You can have different types of access modes on topics for producers. :::note - Once an application creates a producer with the `Exclusive` or `WaitForExclusive` access mode successfully, the instance of the application is guaranteed to be the **only one writer** on the topic. Other producers trying to produce on this topic get errors immediately or have to wait until they get the `Exclusive` access. - For more information, see [PIP 68: Exclusive Producer](https://github.com/apache/pulsar/wiki/PIP-68:-Exclusive-Producer). ::: @@ -215,15 +214,17 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .maxRedeliverCount(maxRedeliveryCount) .build()) .subscribe(); - ``` + The default dead letter topic uses this format: ``` + --DLQ ``` + If you want to specify the name of the dead letter topic, use this Java client example: @@ -238,10 +239,9 @@ Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) .deadLetterTopic("your-topic-name") .build()) .subscribe(); - ``` - + Dead letter topic depends on message re-delivery. Messages are redelivered either due to [acknowledgement timeout](#acknowledgement-timeout) or [negative acknowledgement](#negative-acknowledgement). If you are going to use negative acknowledgement on a message, make sure it is negatively acknowledged before the acknowledgement timeout. :::note @@ -581,6 +581,7 @@ A broker saves a message without any check. When a consumer consumes a message, Delayed message delivery is enabled by default. You can change it in the broker configuration file as below: ``` + # Whether to enable the delayed delivery for messages. # If disabled, messages are immediately delivered and there is no tracking overhead. delayedDeliveryEnabled=true @@ -601,3 +602,4 @@ The following is an example of delayed message delivery for a producer in Java: producer.newMessage().deliverAfter(3L, TimeUnit.Minute).value("Hello Pulsar!").send(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-multi-tenancy.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-multi-tenancy.md index 8818e0b391894..2c556c9dce554 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-multi-tenancy.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-multi-tenancy.md @@ -1,7 +1,7 @@ --- id: concepts-multi-tenancy title: Multi Tenancy -sidebar_label: Multi Tenancy +sidebar_label: "Multi Tenancy" original_id: concepts-multi-tenancy --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-multiple-advertised-listeners.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-multiple-advertised-listeners.md index 4e805430fef06..a5e9663c17caa 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-multiple-advertised-listeners.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-multiple-advertised-listeners.md @@ -1,7 +1,7 @@ --- id: concepts-multiple-advertised-listeners title: Multiple advertised listeners -sidebar_label: Multiple advertised listeners +sidebar_label: "Multiple advertised listeners" original_id: concepts-multiple-advertised-listeners --- @@ -44,4 +44,5 @@ PulsarClient client = PulsarClient.builder() .listenerName("external") .build(); -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-overview.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-overview.md index 71ee9feb7d080..d16f57bf3c2af 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-overview.md @@ -1,7 +1,7 @@ --- id: concepts-overview title: Pulsar Overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: concepts-overview --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-proxy-sni-routing.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-proxy-sni-routing.md index 97fc00c643997..b9f9c15302742 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-proxy-sni-routing.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-proxy-sni-routing.md @@ -1,7 +1,7 @@ --- id: concepts-proxy-sni-routing title: Proxy support with SNI routing -sidebar_label: Proxy support with SNI routing +sidebar_label: "Proxy support with SNI routing" original_id: concepts-proxy-sni-routing --- @@ -36,6 +36,7 @@ To configure the `records.config` files, complete the following steps. The following is an example. ``` + # PROXY TLS PORT CONFIG proxy.config.http.server_ports STRING 4443:ssl 4080 # PROXY CERTS FILE PATH @@ -54,6 +55,7 @@ The [ssl_server_name](https://docs.trafficserver.apache.org/en/8.0.x/admin-guide The following example shows mapping of the inbound SNI hostname coming from the client, and the actual broker service URL where request should be redirected. For example, if the client sends the SNI header `pulsar-broker1`, the proxy creates a TLS tunnel by redirecting request to the `pulsar-broker1:6651` service URL. ``` + server_config = { { fqdn = 'pulsar-broker-vip', @@ -123,6 +125,7 @@ PulsarClient pulsarClient = clientBuilder.build(); ```c++ + ClientConfiguration config = ClientConfiguration(); config.setUseTls(true); config.setTlsTrustCertsFilePath("/path/to/cacert.pem"); @@ -163,6 +166,7 @@ In this example, a Pulsar cluster is deployed into two separate regions, `us-wes (a) Configure the cluster metadata for `us-east` with `us-east` broker service URL and `us-east` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://east-broker-vip:6651 \ --url http://east-broker-vip:8080 \ @@ -174,6 +178,7 @@ In this example, a Pulsar cluster is deployed into two separate regions, `us-wes (b) Configure the cluster metadata for `us-west` with `us-west` broker service URL and `us-west` ATS proxy URL with SNI proxy-protocol. ``` + ./pulsar-admin clusters update \ --broker-url-secure pulsar+ssl://west-broker-vip:6651 \ --url http://west-broker-vip:8080 \ @@ -181,3 +186,4 @@ In this example, a Pulsar cluster is deployed into two separate regions, `us-wes --proxy-url pulsar+ssl://west-ats-proxy:443 ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-replication.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-replication.md index fb76f6e9abb7b..0be527ab82a2f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-replication.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-replication.md @@ -1,7 +1,7 @@ --- id: concepts-replication title: Geo Replication -sidebar_label: Geo Replication +sidebar_label: "Geo Replication" original_id: concepts-replication --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-tiered-storage.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-tiered-storage.md index 61a9859e355e5..8207661cabfaa 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-tiered-storage.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-tiered-storage.md @@ -1,7 +1,7 @@ --- id: concepts-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" original_id: concepts-tiered-storage --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-topic-compaction.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-topic-compaction.md index 53975e2cb6cbc..c685721e83c70 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-topic-compaction.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-topic-compaction.md @@ -1,7 +1,7 @@ --- id: concepts-topic-compaction title: Topic Compaction -sidebar_label: Topic Compaction +sidebar_label: "Topic Compaction" original_id: concepts-topic-compaction --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/concepts-transactions.md b/site2/website-next/versioned_docs/version-2.8.0/concepts-transactions.md index c0297e148c516..9e44d4f4e0258 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/concepts-transactions.md +++ b/site2/website-next/versioned_docs/version-2.8.0/concepts-transactions.md @@ -1,7 +1,7 @@ --- id: transactions title: Transactions -sidebar_label: Overview +sidebar_label: "Overview" original_id: transactions --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-compaction.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-compaction.md index 6a06829decefb..ebcf4ba3237a7 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-compaction.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-compaction.md @@ -1,7 +1,7 @@ --- id: cookbooks-compaction title: Topic compaction -sidebar_label: Topic compaction +sidebar_label: "Topic compaction" original_id: cookbooks-compaction --- @@ -143,3 +143,4 @@ Message msg = MessageBuilder.create() compactedTopicProducer.send(msg); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-deduplication.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-deduplication.md index 6f1cec338230b..332228cfcaf4b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-deduplication.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-deduplication.md @@ -1,7 +1,7 @@ --- id: cookbooks-deduplication title: Message deduplication -sidebar_label: Message deduplication +sidebar_label: "Message deduplication" original_id: cookbooks-deduplication --- @@ -128,6 +128,7 @@ producer = client.create_producer( send_timeout_millis=0) ``` + @@ -152,6 +153,7 @@ Producer producer; Result result = client.createProducer(topic, producerConfig, producer); ``` + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-encryption.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-encryption.md index 67f4bf9b0f929..c782e69ddd047 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-encryption.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-encryption.md @@ -1,7 +1,7 @@ --- id: cookbooks-encryption title: Pulsar Encryption -sidebar_label: Encryption +sidebar_label: "Encryption" original_id: cookbooks-encryption --- @@ -39,6 +39,7 @@ openssl ecparam -name secp521r1 -genkey -param_enc explicit -out test_ecdsa_priv openssl ec -in test_ecdsa_privkey.pem -pubout -outform pkcs8 -out test_ecdsa_pubkey.pem ``` + 2. Add the public and private key to the key management and configure your producers to retrieve public keys and consumers clients to retrieve private keys. 3. Implement CryptoKeyReader::getPublicKey() interface from producer and CryptoKeyReader::getPrivateKey() interface from consumer, which will be invoked by Pulsar client to load the key. 4. Add encryption key to producer configuration: conf.addEncryptionKey("myapp.key") @@ -96,6 +97,7 @@ for (int i = 0; i < 10; i++) { pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -171,6 +173,7 @@ conf.addEncryptionKey("myapp.messagekey1"); conf.addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypting encrypted messages at the consumer application: Consumers require access one of the private keys to decrypt messages produced by the producer. If you would like to receive encrypted messages, create a public/private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-message-queue.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-message-queue.md index 6805f89e3020a..2e4d0e826399a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-message-queue.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-message-queue.md @@ -1,7 +1,7 @@ --- id: cookbooks-message-queue title: Using Pulsar as a message queue -sidebar_label: Message queue +sidebar_label: "Message queue" original_id: cookbooks-message-queue --- @@ -128,3 +128,4 @@ if err != nil { } ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-non-persistent.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-non-persistent.md index 8c5f0046f2524..f10d8374041ac 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-non-persistent.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-non-persistent.md @@ -1,7 +1,7 @@ --- id: cookbooks-non-persistent title: Non-persistent messaging -sidebar_label: Non-persistent messaging +sidebar_label: "Non-persistent messaging" original_id: cookbooks-non-persistent --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-partitioned.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-partitioned.md index 08edd47c2d89e..5535276402e90 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-partitioned.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-partitioned.md @@ -1,7 +1,7 @@ --- id: cookbooks-partitioned title: Partitioned topics -sidebar_label: Partitioned Topics +sidebar_label: "Partitioned Topics" original_id: cookbooks-partitioned --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-retention-expiry.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-retention-expiry.md index c62dcf2b565ca..eb9f80beed2db 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-retention-expiry.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-retention-expiry.md @@ -1,7 +1,7 @@ --- id: cookbooks-retention-expiry title: Message retention and expiry -sidebar_label: Message retention and expiry +sidebar_label: "Message retention and expiry" original_id: cookbooks-retention-expiry --- @@ -79,6 +79,7 @@ You can set a retention policy for a namespace by specifying the namespace, a si } ]}> + You can use the [`set-retention`](reference-pulsar-admin.md#namespaces-set-retention) subcommand and specify a namespace, a size limit using the `-s`/`--size` flag, and a time limit using the `-t`/`--time` flag. In the following example, the size limit is set to 10 GB and the time limit is set to 3 hours for each topic within the `my-tenant/my-ns` namespace. @@ -408,6 +409,7 @@ $ pulsar-admin namespaces remove-message-ttl my-tenant/my-ns admin.namespaces().removeNamespaceMessageTTL(namespace) ``` + ## Delete messages from namespaces If you do not have any retention period and that you never have much of a backlog, the upper limit for retaining messages, which are acknowledged, equals to the Pulsar segment rollover period + entry log rollover period + (garbage collection interval * garbage collection ratios). diff --git a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md index c31776b23ad04..cfaa5257d4527 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md +++ b/site2/website-next/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md @@ -1,7 +1,7 @@ --- id: cookbooks-tiered-storage title: Tiered Storage -sidebar_label: Tiered Storage +sidebar_label: "Tiered Storage" original_id: cookbooks-tiered-storage --- @@ -11,11 +11,9 @@ import TabItem from '@theme/TabItem'; Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -55,7 +53,7 @@ Currently we support driver of types: > though it requires that you specify an endpoint url using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if > using a S3 compatible data store, other than AWS. -```conf +``` managedLedgerOffloadDriver=aws-s3 @@ -70,7 +68,7 @@ Everything that you store in Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` s3ManagedLedgerOffloadBucket=pulsar-topic-offload @@ -81,7 +79,7 @@ but a recommended configuration. If it is not configured, It will use the defaul With AWS S3, the default region is `US East (N. Virginia)`. Page [AWS Regions and Endpoints](https://docs.aws.amazon.com/general/latest/gr/rande.html) contains more information. -```conf +``` s3ManagedLedgerOffloadRegion=eu-west-3 @@ -100,7 +98,7 @@ Once you have created a set of credentials in the AWS IAM console, they can be c If you are on AWS instance with an instance profile that provides credentials, Pulsar will use these credentials if no other mechanism is provided -2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ```conf/pulsar_env.sh```. +2. Set the environment variables **AWS_ACCESS_KEY_ID** and **AWS_SECRET_ACCESS_KEY** in ``` ```bash @@ -122,7 +120,9 @@ PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessK 4. Set the access credentials in ```~/.aws/credentials```. -```conf [default] +``` + +[default] aws_access_key_id=ABC123456789 aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c @@ -132,7 +132,7 @@ aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c If you want to assume an IAM role, this can be done via specifying the following: -```conf +``` s3ManagedLedgerOffloadRole= s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload @@ -147,7 +147,8 @@ This will use the `DefaultAWSCredentialsProviderChain` for assuming this role. Pulsar also provides some knobs to configure the size of requests sent to AWS S3. -- ```s3ManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of +- ``` + a "part" sent during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```s3ManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from AWS S3. Default is 1MB. @@ -160,7 +161,7 @@ Buckets are the basic containers that hold your data. Everything that you store Cloud Storage must be contained in a bucket. You can use buckets to organize your data and control access to your data, but unlike directories and folders, you cannot nest buckets. -```conf +``` gcsManagedLedgerOffloadBucket=pulsar-topic-offload @@ -172,7 +173,7 @@ a recommended configuration. If it is not configured, It will use the default re Regarding GCS, buckets are default created in the `us multi-regional location`, page [Bucket Locations](https://cloud.google.com/storage/docs/bucket-locations) contains more information. -```conf +``` gcsManagedLedgerOffloadRegion=europe-west3 @@ -197,7 +198,7 @@ To generate service account credentials or view the public credentials that you' > Notes: Make ensure that the service account you create has permission to operate GCS, you need to assign **Storage Admin** permission to your service account in [here](https://cloud.google.com/storage/docs/access-control/iam). -```conf +``` gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804d5e6a6f33.json" @@ -207,7 +208,8 @@ gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/hello/Downloads/project-804 Pulsar also provides some knobs to configure the size of requests sent to GCS. -- ```gcsManagedLedgerOffloadMaxBlockSizeInBytes``` configures the maximum size of a "part" sent +- ``` + during a multipart upload. This cannot be smaller than 5MB. Default is 64MB. - ```gcsManagedLedgerOffloadReadBufferSizeInBytes``` configures the block size for each individual read when reading back data from GCS. Default is 1MB. @@ -221,16 +223,17 @@ In both cases, these should not be touched unless you know what you are doing. You can configure the connection address in the `broker.conf` file. -```conf +``` fileSystemURI="hdfs://127.0.0.1:9000" ``` + #### Configure Hadoop profile path The configuration file is stored in the Hadoop profile path. It contains various settings, such as base path, authentication, and so on. -```conf +``` fileSystemProfilePath="../conf/filesystem_offload_core_site.xml" @@ -240,7 +243,7 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us **Example** -```conf +``` fs.defaultFS @@ -271,7 +274,6 @@ The model for storing topic data uses `org.apache.hadoop.io.MapFile`. You can us io.map.index.interval 128 - ``` @@ -280,7 +282,7 @@ For more information about the configurations in `org.apache.hadoop.io.MapFile`, Namespace policies can be configured to offload data automatically once a threshold is reached. The threshold is based on the size of data that the topic has stored on the pulsar cluster. Once the topic reaches the threshold, an offload operation will be triggered. Setting a negative value to the threshold will disable automatic offloading. Setting the threshold to 0 will cause the broker to offload data as soon as it possiby can. -```bash +``` $ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace @@ -293,13 +295,13 @@ $ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-name By default, once messages were offloaded to long term storage, brokers will read them from long term storage, but messages still exists in bookkeeper for a period depends on the administrator's configuration. For messages exists in both bookkeeper and long term storage, if they are preferred to read from bookkeeper, you can use command to change this configuration. -```bash +``` # default value for -orp is tiered-storage-first $ bin/pulsar-admin namespaces set-offload-policies my-tenant/my-namespace -orp bookkeeper-first $ bin/pulsar-admin topics set-offload-policies my-tenant/my-namespace/topic1 -orp bookkeeper-first -``` +``` ## Triggering offload manually @@ -307,7 +309,7 @@ Offloading can manually triggered through a REST endpoint on the Pulsar broker. When triggering offload, you must specify the maximum size, in bytes, of backlog which will be retained locally on the bookkeeper. The offload mechanism will offload segments from the start of the topic backlog until this condition is met. -```bash +``` $ bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 @@ -316,7 +318,7 @@ Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages be The command to triggers an offload will not wait until the offload operation has completed. To check the status of the offload, use offload-status. -```bash +``` $ bin/pulsar-admin topics offload-status my-tenant/my-namespace/topic1 Offload is currently running @@ -325,7 +327,7 @@ Offload is currently running To wait for offload to complete, add the -w flag. -```bash +``` $ bin/pulsar-admin topics offload-status -w my-tenant/my-namespace/topic1 Offload was a success @@ -334,7 +336,7 @@ Offload was a success If there is an error offloading, the error will be propagated to the offload-status command. -```bash +``` $ bin/pulsar-admin topics offload-status persistent://public/default/topic1 Error in offload @@ -342,5 +344,5 @@ null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-aws.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-aws.md index 8b5b4c96d68bc..78defa1e3e5ec 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-aws.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-aws.md @@ -1,7 +1,7 @@ --- id: deploy-aws title: Deploying a Pulsar cluster on AWS using Terraform and Ansible -sidebar_label: Amazon Web Services +sidebar_label: "Amazon Web Services" original_id: deploy-aws --- @@ -60,16 +60,23 @@ $ cd pulsar/deployment/terraform-ansible/aws > > 1. update `ansible.cfg` with following values: > + > ```shell -> +> > private_key_file=~/.ssh/pulsar_aws +> +> > ``` + > > 2. update `terraform.tfvars` with following values: > + > ```shell -> +> > public_key_path=~/.ssh/pulsar_aws.pub +> +> > ``` In order to create the necessary AWS resources using Terraform, you need to create an SSH key. Enter the following commands to create a private SSH key in `~/.ssh/id_rsa` and a public key in `~/.ssh/id_rsa.pub`: @@ -161,6 +168,7 @@ All EC2 instances for the cluster run in the [us-west-2](http://docs.aws.amazon. When you apply the Terraform configuration by entering the command `terraform apply`, Terraform outputs a value for the `pulsar_service_url`. The value should look something like this: ``` + pulsar://pulsar-elb-1800761694.us-west-2.elb.amazonaws.com:6650 ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal-multi-cluster.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal-multi-cluster.md index f62b81eafdeb3..986ba1286a384 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal-multi-cluster.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal-multi-cluster.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal-multi-cluster title: Deploying a multi-cluster on bare metal -sidebar_label: Bare metal multi-cluster +sidebar_label: "Bare metal multi-cluster" original_id: deploy-bare-metal-multi-cluster --- @@ -41,7 +41,6 @@ Currently, Pulsar is available for 64-bit **macOS**, **Linux**, and **Windows**. :::note - Broker is only supported on 64-bit JVM. ::: @@ -59,9 +58,9 @@ To get started running Pulsar, download a binary tarball release in one of the f * using [wget](https://www.gnu.org/software/wget): ```shell - + $ wget 'https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=pulsar/pulsar-@pulsar:version@/apache-pulsar-@pulsar:version@-bin.tar.gz' -O apache-pulsar-@pulsar:version@-bin.tar.gz - + ``` Once you download the tarball, untar it and `cd` into the resulting directory: @@ -179,6 +178,7 @@ For example, assume a Pulsar instance with the following clusters `us-west`, `us-east`, `us-central`, `eu-central`, `ap-south`. Also assume, each cluster has its own local ZK servers named such as the following: ``` + zk[1-3].${CLUSTER}.example.com ``` @@ -487,3 +487,4 @@ To report the topic stats: $ bin/pulsar-admin topics stats persistent://test-tenant/ns1/my-topic ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal.md index 0f3111c20bba4..c2df6376cd077 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-bare-metal.md @@ -1,7 +1,7 @@ --- id: deploy-bare-metal title: Deploy a cluster on bare metal -sidebar_label: Bare metal +sidebar_label: "Bare metal" original_id: deploy-bare-metal --- @@ -49,7 +49,6 @@ To run Pulsar on bare metal, the following configuration is recommended: :::note - Broker is only supported on 64-bit JVM. ::: @@ -136,9 +135,9 @@ To get started using builtin connectors, you need to download the connectors tar * using [wget](https://www.gnu.org/software/wget): ```shell - + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar - + ``` Once you download the .nar file, copy the file to directory `connectors` in the pulsar directory. @@ -172,9 +171,9 @@ To get started using tiered storage offloaders, you need to download the offload * using [wget](https://www.gnu.org/software/wget): ```shell - + $ wget pulsar:offloader_release_url - + ``` Once you download the tarball, in the pulsar directory, untar the offloaders package and copy the offloaders as `offloaders` in the pulsar directory: @@ -274,24 +273,32 @@ Flag | Description > If you do not have a DNS server, you can use multi-host format in the service URL with the following settings: > + > ```properties -> +> > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` + > > If you want to use an existing BookKeeper cluster, you can add the `--existing-bk-metadata-service-uri` flag as follows: > + > ```properties -> +> > --existing-bk-metadata-service-uri "zk+null://zk1:2181;zk2:2181/ledgers" \ > --web-service-url http://host1:8080,host2:8080,host3:8080 \ > --web-service-url-tls https://host1:8443,host2:8443,host3:8443 \ > --broker-service-url pulsar://host1:6650,host2:6650,host3:6650 \ > --broker-service-url-tls pulsar+ssl://host1:6651,host2:6651,host3:6651 +> +> > ``` + > You can obtain the metadata service URI of the existing BookKeeper cluster by using the `bin/bookkeeper shell whatisinstanceid` command. You must enclose the value in double quotes since the multiple metadata service URIs are separated with semicolons. ## Deploy a BookKeeper cluster @@ -383,8 +390,9 @@ webServicePortTls=8443 > If you deploy Pulsar in a one-node cluster, you should update the replication settings in `conf/broker.conf` to `1`. > + > ```properties -> +> > # Number of bookies to use when creating a ledger > managedLedgerDefaultEnsembleSize=1 > @@ -393,6 +401,8 @@ webServicePortTls=8443 > > # Number of guaranteed copies (acks to wait before write is complete) > managedLedgerDefaultAckQuorum=1 +> +> > ``` ### Enable Pulsar Functions (optional) @@ -401,19 +411,19 @@ If you want to enable [Pulsar Functions](functions-overview), you can follow the 1. Edit `conf/broker.conf` to enable functions worker, by setting `functionsWorkerEnabled` to `true`. - ```conf - - functionsWorkerEnabled=true - - ``` + ```conf + + functionsWorkerEnabled=true + + ``` 2. Edit `conf/functions_worker.yml` and set `pulsarFunctionsCluster` to the cluster name that you provide when you [initialize the metadata of the cluster](#initialize-cluster-metadata). - ```conf - - pulsarFunctionsCluster: pulsar-cluster-1 - - ``` + ```conf + + pulsarFunctionsCluster: pulsar-cluster-1 + + ``` If you want to learn more options about deploying the functions worker, check out [Deploy and manage functions worker](functions-worker). @@ -454,10 +464,13 @@ brokerServiceurl=pulsar://us-west.example.com:6650 > If you do not have a DNS server, you can specify multi-host in service URL as follows: > + > ```properties -> +> > webServiceUrl=http://host1:8080,host2:8080,host3:8080 > brokerServiceurl=pulsar://host1:6650,host2:6650,host3:6650 +> +> > ``` Once that is complete, you can publish a message to the Pulsar topic: @@ -528,3 +541,4 @@ You should see the following output: hello world! ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-dcos.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-dcos.md index 9cf71f01d1713..14a36352e6872 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-dcos.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-dcos.md @@ -1,7 +1,7 @@ --- id: deploy-dcos title: Deploy Pulsar on DC/OS -sidebar_label: DC/OS +sidebar_label: "DC/OS" original_id: deploy-dcos --- @@ -28,9 +28,9 @@ In order to run Pulsar on DC/OS, you need the following: * The [`PulsarGroups.json`](https://github.com/apache/pulsar/blob/master/deployment/dcos/PulsarGroups.json) configuration file from the Pulsar GitHub repo. ```bash - + $ curl -O https://raw.githubusercontent.com/apache/pulsar/master/deployment/dcos/PulsarGroups.json - + ``` Each node in the DC/OS-managed Mesos cluster must have at least: @@ -190,12 +190,13 @@ You can shut down and uninstall the `pulsar` application from DC/OS at any time 1. Using the DC/OS GUI, you can choose **Delete** at the right end of Pulsar group. - ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) + ![DC/OS pulsar uninstall](/assets/dcos_uninstall.png) 2. You can use the following command: - ```bash - - $ dcos marathon group remove /pulsar + ```bash + + $ dcos marathon group remove /pulsar + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-docker.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-docker.md index e094193c6648c..f76318f67daaf 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-docker.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-docker.md @@ -1,7 +1,7 @@ --- id: deploy-docker title: Deploy a cluster on Docker -sidebar_label: Docker +sidebar_label: "Docker" original_id: deploy-docker --- @@ -23,6 +23,7 @@ To run Pulsar on Docker, you need to create a container for each Pulsar componen You can pull a Pulsar image from [Docker Hub](https://hub.docker.com/r/apachepulsar/pulsar-all/tags) with the following command. ``` + docker pull apachepulsar/pulsar-all:latest ``` @@ -31,6 +32,7 @@ docker pull apachepulsar/pulsar-all:latest Create containers for ZooKeeper, BookKeeper and broker. In this example, they are named as `zookeeper`, `bookkeeper` and `broker` respectively. You can name them as you want with the `--name` flag. By default, the container names are created randomly. ``` + docker run -it --name bookkeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name zookeeper apachepulsar/pulsar-all:latest /bin/bash docker run -it --name broker apachepulsar/pulsar-all:latest /bin/bash @@ -41,6 +43,7 @@ docker run -it --name broker apachepulsar/pulsar-all:latest /bin/bash To deploy a Pulsar cluster on Docker, you need to create a `network` and connect the containers of ZooKeeper, BookKeeper and broker to this network. The following command creates the network `pulsar`: ``` + docker network create pulsar ``` @@ -49,6 +52,7 @@ docker network create pulsar Connect the containers of ZooKeeper, BookKeeper and broker to the `pulsar` network with the following commands. ``` + docker network connect pulsar zookeeper docker network connect pulsar bookkeeper docker network connect pulsar broker diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-kubernetes.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-kubernetes.md index 31111dfe1a4a7..f8f450042c4b9 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-kubernetes.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-kubernetes.md @@ -1,7 +1,7 @@ --- id: deploy-kubernetes title: Deploy Pulsar on Kubernetes -sidebar_label: Kubernetes +sidebar_label: "Kubernetes" original_id: deploy-kubernetes --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/deploy-monitoring.md b/site2/website-next/versioned_docs/version-2.8.0/deploy-monitoring.md index 40901d3fa16b4..eece781107ab8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/deploy-monitoring.md +++ b/site2/website-next/versioned_docs/version-2.8.0/deploy-monitoring.md @@ -1,7 +1,7 @@ --- id: deploy-monitoring title: Monitor -sidebar_label: Monitor +sidebar_label: "Monitor" original_id: deploy-monitoring --- @@ -22,17 +22,17 @@ You can collect Pulsar broker metrics from brokers and export the metrics in JSO * *Destination dumps*, which contain stats for each individual topic. You can fetch the destination dumps using the command below: ```shell - + bin/pulsar-admin broker-stats destinations - + ``` * Broker metrics, which contain the broker information and topics stats aggregated at namespace level. You can fetch the broker metrics by using the following command: ```shell - + bin/pulsar-admin broker-stats monitoring-metrics - + ``` All the message rates are updated every minute. @@ -76,7 +76,8 @@ The default port for bookie is `8000`. You can change the port by configuring `p The acknowledgment state is persistent to the ledger first. When the acknowledgment state fails to be persistent to the ledger, they are persistent to ZooKeeper. To track the stats of acknowledgement, you can configure the metrics for the managed cursor. ``` -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"") + +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:") brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"") brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"") @@ -91,6 +92,7 @@ Those metrics are added in the Prometheus interface, you can monitor and check t You can collect functions worker stats from `functions-worker` and export the metrics in JSON formats, which contain functions worker JVM metrics. ``` + pulsar-admin functions-worker monitoring-metrics ``` @@ -98,6 +100,7 @@ pulsar-admin functions-worker monitoring-metrics You can collect functions and connectors metrics from `functions-worker` and export the metrics in JSON formats. ``` + pulsar-admin functions-worker function-stats ``` @@ -105,6 +108,7 @@ pulsar-admin functions-worker function-stats The aggregated functions and connectors metrics can be exposed in Prometheus formats as below. You can get [`FUNCTIONS_WORKER_ADDRESS`](http://pulsar.apache.org/docs/en/next/functions-worker/) and `WORKER_PORT` from the `functions_worker.yml` file. ``` + http://$FUNCTIONS_WORKER_ADDRESS:$WORKER_PORT/metrics: ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/developing-binary-protocol.md b/site2/website-next/versioned_docs/version-2.8.0/developing-binary-protocol.md index f547375073cc8..3a202a38bcc4a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/developing-binary-protocol.md +++ b/site2/website-next/versioned_docs/version-2.8.0/developing-binary-protocol.md @@ -1,7 +1,7 @@ --- id: develop-binary-protocol title: Pulsar binary protocol specification -sidebar_label: Binary protocol +sidebar_label: "Binary protocol" original_id: develop-binary-protocol --- @@ -350,7 +350,7 @@ message CommandFlow { Parameters: * `consumer_id` → Id of an already established consumer * `messagePermits` → Number of additional permits to grant to the broker for - pushing more messages + pushing more messages ##### Command Message diff --git a/site2/website-next/versioned_docs/version-2.8.0/developing-load-manager.md b/site2/website-next/versioned_docs/version-2.8.0/developing-load-manager.md index de443b10025aa..ed60d5ced6193 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/developing-load-manager.md +++ b/site2/website-next/versioned_docs/version-2.8.0/developing-load-manager.md @@ -1,7 +1,7 @@ --- id: develop-load-manager title: Modular load manager -sidebar_label: Modular load manager +sidebar_label: "Modular load manager" original_id: develop-load-manager --- @@ -19,11 +19,11 @@ There are two ways that you can enable the modular load manager: 2. Using the `pulsar-admin` tool. Here's an example: ```shell - + $ pulsar-admin update-dynamic-config \ - --config loadManagerClassName \ - --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl - + --config loadManagerClassName \ + --value org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl + ``` You can use the same method to change back to the original value. In either case, any mistake in specifying the load manager will cause Pulsar to default to `SimpleLoadManagerImpl`. @@ -34,111 +34,113 @@ There are a few different ways to determine which load manager is being used: 1. Use `pulsar-admin` to examine the `loadManagerClassName` element: - ```shell - + ```shell + $ bin/pulsar-admin brokers get-all-dynamic-config { - "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" + "loadManagerClassName" : "org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl" } - + ``` If there is no `loadManagerClassName` element, then the default load manager is used. 2. Consult a ZooKeeper load report. With the module load manager, the load report in `/loadbalance/brokers/...` will have many differences. for example the `systemResourceUsage` sub-elements (`bandwidthIn`, `bandwidthOut`, etc.) are now all at the top level. Here is an example load report from the module load manager: - ```json - - { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 4.256510416666667 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 5.287239583333333 - }, - "bundles": [], - "cpu": { - "limit": 2400.0, - "usage": 5.7353247655435915 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - } - } - - ``` - - With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: - - ```json - - { - "systemResourceUsage": { - "bandwidthIn": { - "limit": 10240000.0, - "usage": 0.0 - }, - "bandwidthOut": { - "limit": 10240000.0, - "usage": 0.0 - }, - "cpu": { - "limit": 2400.0, - "usage": 0.0 - }, - "directMemory": { - "limit": 16384.0, - "usage": 1.0 - }, - "memory": { - "limit": 8192.0, - "usage": 3903.0 - } - } - } - - ``` + ```json + + { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 4.256510416666667 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 5.287239583333333 + }, + "bundles": [], + "cpu": { + "limit": 2400.0, + "usage": 5.7353247655435915 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + } + } + + ``` + + With the simple load manager, the load report in `/loadbalance/brokers/...` will look like this: + + ```json + + { + "systemResourceUsage": { + "bandwidthIn": { + "limit": 10240000.0, + "usage": 0.0 + }, + "bandwidthOut": { + "limit": 10240000.0, + "usage": 0.0 + }, + "cpu": { + "limit": 2400.0, + "usage": 0.0 + }, + "directMemory": { + "limit": 16384.0, + "usage": 1.0 + }, + "memory": { + "limit": 8192.0, + "usage": 3903.0 + } + } + } + + ``` 3. The command-line [broker monitor](reference-cli-tools.md#monitor-brokers) will have a different output format depending on which load manager implementation is being used. - Here is an example from the modular load manager: - - ``` - =================================================================================================================== - ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |4 |0 || - ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || - =================================================================================================================== - - ``` - - Here is an example from the simple load manager: - - ``` - =================================================================================================================== - ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || - || |4 |4 |0 |2 |0 |0 || - ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || - ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || - || |0.20 |1.89 | |1.27 |3.21 |3.21 || - ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || - ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || - || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || - =================================================================================================================== - - ``` + Here is an example from the modular load manager: + + ``` + + =================================================================================================================== + ||SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.00 |48.33 |0.01 |0.00 |0.00 |48.33 || + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |4 |0 || + ||LATEST |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||SHORT |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + ||LONG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.00 |0.00 |0.00 || + =================================================================================================================== + + ``` + + Here is an example from the simple load manager: + + ``` + + =================================================================================================================== + ||COUNT |TOPIC |BUNDLE |PRODUCER |CONSUMER |BUNDLE + |BUNDLE - || + || |4 |4 |0 |2 |0 |0 || + ||RAW SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.25 |47.94 |0.01 |0.00 |0.00 |47.94 || + ||ALLOC SYSTEM |CPU % |MEMORY % |DIRECT % |BW IN % |BW OUT % |MAX % || + || |0.20 |1.89 | |1.27 |3.21 |3.21 || + ||RAW MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |0.00 |0.00 |0.00 |0.01 |0.01 |0.01 || + ||ALLOC MSG |MSG/S IN |MSG/S OUT |TOTAL |KB/S IN |KB/S OUT |TOTAL || + || |54.84 |134.48 |189.31 |126.54 |320.96 |447.50 || + =================================================================================================================== + + ``` It is important to note that the module load manager is _centralized_, meaning that all requests to assign a bundle---whether it's been seen before or whether this is the first time---only get handled by the _lead_ broker (which can change over time). To determine the current lead broker, examine the `/loadbalance/leader` node in ZooKeeper. diff --git a/site2/website-next/versioned_docs/version-2.8.0/developing-tools.md b/site2/website-next/versioned_docs/version-2.8.0/developing-tools.md index 8f068a291aecb..f560f9d79162a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/developing-tools.md +++ b/site2/website-next/versioned_docs/version-2.8.0/developing-tools.md @@ -1,7 +1,7 @@ --- id: develop-tools title: Simulation tools -sidebar_label: Simulation tools +sidebar_label: "Simulation tools" original_id: develop-tools --- @@ -24,6 +24,7 @@ send signals to clients to start incurring load. The client implementation is in To Start a simulation client, use the `pulsar-perf` script with the command `simulation-client` as follows: ``` + pulsar-perf simulation-client --port --service-url ``` @@ -39,6 +40,7 @@ command with. To start a simulation controller, use the `pulsar-perf` script with the command `simulation-controller` as follows: ``` + pulsar-perf simulation-controller --cluster --client-port --clients @@ -51,32 +53,32 @@ names. In all cases, the BASE name of the tenants, namespaces, and topics are us `my_namespace`, and the topic name is `my_topic`. The controller can perform the following actions: * Create a topic with a producer and a consumer - * `trade [--rate ] - [--rand-rate ,] - [--size ]` + * `trade [--rate ] + [--rand-rate ,] + [--size ]` * Create a group of topics with a producer and a consumer - * `trade_group [--rate ] - [--rand-rate ,] - [--separation ] [--size ] - [--topics-per-namespace ]` + * `trade_group [--rate ] + [--rand-rate ,] + [--separation ] [--size ] + [--topics-per-namespace ]` * Change the configuration of an existing topic - * `change [--rate ] - [--rand-rate ,] - [--size ]` + * `change [--rate ] + [--rand-rate ,] + [--size ]` * Change the configuration of a group of topics - * `change_group [--rate ] [--rand-rate ,] - [--size ] [--topics-per-namespace ]` + * `change_group [--rate ] [--rand-rate ,] + [--size ] [--topics-per-namespace ]` * Shutdown a previously created topic - * `stop ` + * `stop ` * Shutdown a previously created group of topics - * `stop_group ` + * `stop_group ` * Copy the historical data from one ZooKeeper to another and simulate based on the message rates and sizes in that history * `copy [--rate-multiplier value]` * Simulate the load of the historical data on the current ZooKeeper (should be same ZooKeeper being simulated on) - * `simulate [--rate-multiplier value]` + * `simulate [--rate-multiplier value]` * Stream the latest data from the given active ZooKeeper to simulate the real-time load of that ZooKeeper. - * `stream [--rate-multiplier value]` + * `stream [--rate-multiplier value]` The "group" arguments in these commands allow the user to create or affect multiple topics at once. Groups are created when calling the `trade_group` command, and all topics from these groups may be subsequently modified or stopped @@ -105,6 +107,7 @@ console as it is updated using watchers. To start a broker monitor, use the `monitor-brokers` command in the `pulsar-perf` script: ``` + pulsar-perf monitor-brokers --connect-string ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-debug.md b/site2/website-next/versioned_docs/version-2.8.0/functions-debug.md index 7d5c13e84fa5d..2587b48eff857 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-debug.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-debug.md @@ -226,7 +226,6 @@ With [Pulsar Functions CLI](reference-pulsar-admin.md#functions), you can debug :::tip - For complete commands of **Pulsar Functions CLI**, see [here](reference-pulsar-admin.md#functions)。 ::: @@ -254,7 +253,6 @@ $ pulsar-admin functions get options :::tip - `--fqfn` consists of `--name`, `--namespace` and `--tenant`, so you can specify either `--fqfn` or `--name`, `--namespace` and `--tenant`. ::: @@ -268,6 +266,7 @@ You can specify `--fqfn` to get information about a Pulsar Function. $ ./bin/pulsar-admin functions get public/default/ExclamationFunctio6 ``` + Optionally, you can specify `--name`, `--namespace` and `--tenant` to get information about a Pulsar Function. ```bash @@ -467,6 +466,7 @@ $ ./bin/pulsar-admin functions list \ --namespace default ``` + As shown below, the `list` command returns three functions running under the _public_ tenant and the _default_ namespace. ```text @@ -529,9 +529,9 @@ You must specify the [entire topic name](getting-started-pulsar.md#topic-names) ```text Function in trigger function has unidentified topic - Reason: Function in trigger function has unidentified topic ``` ::: + diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-deploy.md b/site2/website-next/versioned_docs/version-2.8.0/functions-deploy.md index fd7bd6205cd27..03eb08c8b3efb 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-deploy.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-deploy.md @@ -235,6 +235,7 @@ $ bin/pulsar-admin functions trigger \ The consumer listening on the output topic produces something as follows in the log. ``` + ----- got message ----- This function has been triggered with a value of hello world diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-develop.md b/site2/website-next/versioned_docs/version-2.8.0/functions-develop.md index d0607c643f486..b567078a92c09 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-develop.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-develop.md @@ -47,6 +47,7 @@ public class JavaNativeExclamationFunction implements Function { } ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/JavaNativeExclamationFunction.java). @@ -58,12 +59,12 @@ def process(input): return "{}!".format(input) ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/native_exclamation_function.py). :::note You can write Pulsar Functions in python2 or python3. However, Pulsar only looks for `python` as the interpreter. - If you're running Pulsar Functions on an Ubuntu system that only supports python3, you might fail to start the functions. In this case, you can create a symlink. Your system will fail if you subsequently install any other package that depends on Python 2.x. A solution is under development in [Issue 5518](https://github.com/apache/pulsar/issues/5518). @@ -112,6 +113,7 @@ public class ExclamationFunction implements Function { } ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/ExclamationFunction.java). @@ -129,6 +131,7 @@ class ExclamationFunction(Function): return input + '!' ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/exclamation_function.py). @@ -155,6 +158,7 @@ func main() { } ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/77cf09eafa4f1626a53a1fe2e65dd25f377c1127/pulsar-function-go/examples/inputFunc/inputFunc.go#L20-L36). @@ -184,6 +188,7 @@ SerDe stands for **Ser**ialization and **De**serialization. Pulsar Functions use } ]}> + When you write Pulsar Functions in Java, the following basic Java types are built in and supported by default: `String`, `Double`, `Integer`, `Float`, `Long`, `Short`, and `Byte`. To customize Java types, you need to implement the following interface. @@ -196,6 +201,7 @@ public interface SerDe { } ``` + SerDe works in the following ways in Java Functions. - If the input and output topics have schema, Pulsar Functions use schema for SerDe. - If the input or output topics do not exist, Pulsar Functions adopt the following rules to determine SerDe: @@ -205,6 +211,7 @@ SerDe works in the following ways in Java Functions. + In Python, the default SerDe is identity, meaning that the type is serialized as whatever type the producer function returns. You can specify the SerDe when [creating](functions-deploy.md#cluster-mode) or [running](functions-deploy.md#local-run-mode) functions. @@ -241,6 +248,7 @@ Custom SerDe | When you require explicit control over SerDe, potentially for per + Currently, the feature is not available in Go. @@ -341,19 +349,20 @@ In order to use this class in Pulsar Functions, you have two options: 2. You can create your own SerDe class. The following is an example. ```python - -from pulsar import SerDe - -class TweetSerDe(SerDe): - - def serialize(self, input): - return bytes("{0}|{1}".format(input.username, input.tweet_content)) - - def deserialize(self, input_bytes): - tweet_components = str(input_bytes).split('|') - return Tweet(tweet_components[0], tweet_componentsp[1]) - + + from pulsar import SerDe + + class TweetSerDe(SerDe): + + def serialize(self, input): + return bytes("{0}|{1}".format(input.username, input.tweet_content)) + + def deserialize(self, input_bytes): + tweet_components = str(input_bytes).split('|') + return Tweet(tweet_components[0], tweet_componentsp[1]) + ``` + For complete code, see [here](https://github.com/apache/pulsar/blob/master/pulsar-functions/python-examples/custom_object_function.py). @@ -399,6 +408,7 @@ Java, Python and Go SDKs provide access to a **context object** that can be used } ]}> + The [Context](https://github.com/apache/pulsar/blob/master/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Context.java) interface provides a number of methods that you can use to access the function [context](#context). The various method signatures for the `Context` interface are listed as follows. ```java @@ -473,6 +483,7 @@ public class ContextFunction implements Function { ``` + class ContextImpl(pulsar.Context): def get_message_id(self): ... @@ -540,6 +551,7 @@ class ContextImpl(pulsar.Context): ``` + func (c *FunctionContext) GetInstanceID() int { return c.instanceConf.instanceID } @@ -597,6 +609,7 @@ func (c *FunctionContext) NewOutputMessage(topic string) pulsar.Producer { The following example uses several methods available via the `Context` object. ``` + import ( "context" "fmt" @@ -648,6 +661,7 @@ $ bin/pulsar-admin functions create \ } ]}> + The Java SDK [`Context`](#context) object enables you to access key/value pairs provided to Pulsar Functions via the command line (as JSON). The following example passes a key/value pair. ```bash @@ -702,6 +716,7 @@ String wotd = context.getUserConfigValueOrDefault("word-of-the-day", "perspicaci + In Python function, you can access the configuration value like this. ```python @@ -748,6 +763,7 @@ class UserConfigFunction(Function): logger.info("The word of the day is {0}".format(wotd)) ``` + @@ -805,6 +821,7 @@ func contextFunc(ctx context.Context) { } ]}> + Pulsar Functions that use the Java SDK have access to an [SLF4j](https://www.slf4j.org/) [`Logger`](https://www.slf4j.org/api/org/apache/log4j/Logger.html) object that can be used to produce logs at the chosen log level. The following example logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```java @@ -974,6 +991,7 @@ The properties set like: ``` + propagate to places where they are referenced, such as: ```xml @@ -987,6 +1005,7 @@ propagate to places where they are referenced, such as: ``` + In the above example, debug level logging would be applied to ALL function logs. This may be more verbose than you desire. To be more selective, you can apply different log levels to different classes or modules. For example: @@ -1002,6 +1021,7 @@ This may be more verbose than you desire. To be more selective, you can apply di ``` + You can be more specific as well, such as applying a more verbose log level to a class in the module, such as: ```xml @@ -1023,9 +1043,11 @@ Additivity pertains to whether log messages will be duplicated if multiple Logge To disable additivity, specify ```xml + false ``` + as shown in examples above. Disabling additivity prevents duplication of log messages when one or more `` entries contain classes or modules that overlap. The `` is defined in the `` section, such as: @@ -1044,6 +1066,7 @@ The `` is defined in the `` section, such as: + Pulsar Functions that use the Python SDK have access to a logging object that can be used to produce logs at the chosen log level. The following example function that logs either a `WARNING`- or `INFO`-level log based on whether the incoming string contains the word `danger`. ```python @@ -1078,9 +1101,11 @@ Additionally, you can specify the function log level through the broker XML file + The following Go Function example shows different log levels based on the function input. ``` + import ( "context" @@ -1128,6 +1153,7 @@ Pulsar Functions using the Java SDK has access to the Pulsar admin client, which Below is an example of how to use the Pulsar admin client exposed from the Function `context`. ``` + import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; @@ -1161,6 +1187,7 @@ public class CursorManagementFunction implements Function { If you want your function to get access to the Pulsar admin client, you need to enable this feature by setting `exposeAdminClientEnabled=true` in the `functions_worker.yml` file. You can test whether this feature is enabled or not using the command `pulsar-admin functions localrun` with the flag `--web-service-url`. ``` + $ bin/pulsar-admin functions localrun \ --jar my-functions.jar \ --classname my.package.CursorManagementFunction \ @@ -1175,11 +1202,10 @@ $ bin/pulsar-admin functions localrun \ ## Metrics -Pulsar Functions allows you to deploy and manage processing functions that consume messages from and publish messages to Pulsar topics easily. It is important to ensure that the running functions are healthy at any time. Pulsar Functions can publish arbitrary metrics to the metrics interface which can be queried. +Pulsar Functions allows you to deploy and manage processing functions that consume messages from and publish messages to Pulsar topics easily. It is important to ensure that the running functions are healthy at any time. Pulsar Functions can publish arbitrary metrics to the metrics interface which can be queried. :::note - If a Pulsar Function uses the language-native interface for Java or Python, that function is not able to publish metrics and stats to Pulsar. ::: @@ -1215,6 +1241,7 @@ Here are examples of how to customize metrics for Java and Python functions. } ]}> + You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. ```java @@ -1241,6 +1268,7 @@ public class MetricRecorderFunction implements Function { + You can record metrics using the [`Context`](#context) object on a per-key basis. For example, you can set a metric for the `process-count` key and a different metric for the `elevens-count` key every time the function processes a message. The following is an example. ```python @@ -1255,8 +1283,10 @@ class MetricRecorderFunction(Function): context.record_metric('elevens-count', 1) ``` + + Currently, the feature is not available in Go. @@ -1293,6 +1323,7 @@ At the same time, Pulsar Functions provides two interfaces, **SecretsProvider** } ]}> + You can get secret provider using the [`Context`](#context) object. The following is an example: ```java @@ -1322,6 +1353,7 @@ public class GetSecretProviderFunction implements Function { + You can get secret provider using the [`Context`](#context) object. The following is an example: ```python @@ -1341,6 +1373,7 @@ class GetSecretProviderFunction(Function): + Currently, the feature is not available in Go. @@ -1377,6 +1410,7 @@ State storage is not available in Go. } ]}> + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](functions-develop.md#context) object when you are using Java SDK functions. #### incrCounter @@ -1523,6 +1557,7 @@ Counters and binary values share the same keyspace, so this deletes either type. + Currently Pulsar Functions expose the following APIs for mutating and accessing State. These APIs are available in the [Context](#context) object when you are using Python SDK functions. #### incr_counter @@ -1530,7 +1565,7 @@ Currently Pulsar Functions expose the following APIs for mutating and accessing ```python def incr_counter(self, key, amount): - """incr the counter of a given key in the managed state""" + ""incr the counter of a given key in the managed state"" ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-metrics.md b/site2/website-next/versioned_docs/version-2.8.0/functions-metrics.md index 6812ec7bb411c..2fe073599c626 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-metrics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-metrics.md @@ -1,7 +1,7 @@ --- id: functions-metrics title: Metrics for Pulsar Functions -sidebar_label: Metrics +sidebar_label: "Metrics" original_id: functions-metrics --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-overview.md b/site2/website-next/versioned_docs/version-2.8.0/functions-overview.md index 9cf5d186fa9d7..9e92f5ac71803 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-overview.md @@ -1,7 +1,7 @@ --- id: functions-overview title: Pulsar Functions overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: functions-overview --- @@ -210,3 +210,4 @@ $ bin/pulsar-admin functions update \ # Other function configs ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-package.md b/site2/website-next/versioned_docs/version-2.8.0/functions-package.md index c0e41880b1c42..42e53cb605eae 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-package.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-package.md @@ -29,156 +29,158 @@ To package a function in Java, complete the following steps. 1. Create a new maven project with a pom file. In the following code sample, the value of `mainClass` is your package name. - ```Java - - - - 4.0.0 - - java-function - java-function - 1.0-SNAPSHOT - - - - org.apache.pulsar - pulsar-functions-api - 2.6.0 - - - - - - - maven-assembly-plugin - - false - - jar-with-dependencies - - - - org.example.test.ExclamationFunction - - - - - - make-assembly - package - - assembly - - - - - - org.apache.maven.plugins - maven-compiler-plugin - - 8 - 8 - - - - - - - - ``` + ```Java + + + + 4.0.0 + + java-function + java-function + 1.0-SNAPSHOT + + + + org.apache.pulsar + pulsar-functions-api + 2.6.0 + + + + + + + maven-assembly-plugin + + false + + jar-with-dependencies + + + + org.example.test.ExclamationFunction + + + + + + make-assembly + package + + assembly + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + + + ``` 2. Write a Java function. - ``` - package org.example.test; - - import java.util.function.Function; + ``` + + package org.example.test; - public class ExclamationFunction implements Function { - @Override - public String apply(String s) { - return "This is my function!"; - } - } + import java.util.function.Function; - ``` - - For the imported package, you can use one of the following interfaces: - - Function interface provided by Java 8: `java.util.function.Function` - - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` - - The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. - - The following example uses `org.apache.pulsar.functions.api.Function` interface with context. - - ``` - package org.example.functions; - import org.apache.pulsar.functions.api.Context; - import org.apache.pulsar.functions.api.Function; - - import java.util.Arrays; - public class WordCountFunction implements Function { - // This function is invoked every time a message is published to the input topic - @Override - public Void process(String input, Context context) throws Exception { - Arrays.asList(input.split(" ")).forEach(word -> { - String counterKey = word.toLowerCase(); - context.incrCounter(counterKey, 1); - }); - return null; + public class ExclamationFunction implements Function { + @Override + public String apply(String s) { + return "This is my function!"; } - } - - ``` - -3. Package the Java function. - - ```bash - - mvn package - - ``` - - After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. + } + + ``` + For the imported package, you can use one of the following interfaces: + - Function interface provided by Java 8: `java.util.function.Function` + - Pulsar Function interface: `org.apache.pulsar.functions.api.Function` -4. Run the Java function. + The main difference between the two interfaces is that the `org.apache.pulsar.functions.api.Function` interface provides the context interface. When you write a function and want to interact with it, you can use context to obtain a wide variety of information and functionality for Pulsar Functions. - (1) Copy the packaged jar file to the Pulsar image. + The following example uses `org.apache.pulsar.functions.api.Function` interface with context. - ```bash + ``` + + package org.example.functions; + import org.apache.pulsar.functions.api.Context; + import org.apache.pulsar.functions.api.Function; + + import java.util.Arrays; + public class WordCountFunction implements Function { + // This function is invoked every time a message is published to the input topic + @Override + public Void process(String input, Context context) throws Exception { + Arrays.asList(input.split(" ")).forEach(word -> { + String counterKey = word.toLowerCase(); + context.incrCounter(counterKey, 1); + }); + return null; + } + } + + ``` - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar +3. Package the Java function. - ``` + ```bash + + mvn package + + ``` - (2) Run the Java function using the following command. + After the Java function is packaged, a `target` directory is created automatically. Open the `target` directory to check if there is a JAR package similar to `java-function-1.0-SNAPSHOT.jar`. - ```bash - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --jar java-function-1.0-SNAPSHOT.jar \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name JavaFunction +4. Run the Java function. - ``` + (1) Copy the packaged jar file to the Pulsar image. - The following log indicates that the Java function starts successfully. + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` - ```text + (2) Run the Java function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname org.example.test.ExclamationFunction \ + --jar java-function-1.0-SNAPSHOT.jar \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name JavaFunction + + ``` - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... + The following log indicates that the Java function starts successfully. - ``` + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Python @@ -194,73 +196,74 @@ To package a function with **one python file** in Python, complete the following 1. Write a Python function. - ``` - from pulsar import Function // import the Function module from Pulsar - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass + ``` + + from pulsar import Function // import the Function module from Pulsar - def process(self, input, context): - return input + '!' + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass - ``` + def process(self, input, context): + return input + '!' + + ``` - In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. + In this example, when you write a Python function, you need to inherit the Function class and implement the `process()` method. - `process()` mainly has two parameters: + `process()` mainly has two parameters: - - `input` represents your input. + - `input` represents your input. - - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. + - `context` represents an interface exposed by the Pulsar Function. You can get the attributes in the Python function based on the provided context object. 2. Install a Python client. - The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. + The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. - ```bash - - pip install python-client==2.6.0 - - ``` + ```bash + + pip install pulsar-client==2.6.0 + + ``` 3. Run the Python Function. - (1) Copy the Python function file to the Pulsar image. - - ```bash - - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar - - ``` - - (2) Run the Python function using the following command. - - ```bash - - ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ - --py \ - --inputs persistent://public/default/my-topic-1 \ - --output persistent://public/default/test-1 \ - --tenant public \ - --namespace default \ - --name PythonFunction + (1) Copy the Python function file to the Pulsar image. - ``` - - The following log indicates that the Python function starts successfully. + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` - ```text + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname . \ + --py \ + --inputs persistent://public/default/my-topic-1 \ + --output persistent://public/default/test-1 \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... + The following log indicates that the Python function starts successfully. - ``` + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### ZIP file @@ -268,63 +271,64 @@ To package a function with the **ZIP file** in Python, complete the following st 1. Prepare the ZIP file. - The following is required when packaging the ZIP file of the Python Function. - - ```text - - Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: - "func/src" - "func/requirements.txt" - "func/deps" - - ``` - Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. + The following is required when packaging the ZIP file of the Python Function. - ```text + ```text + + Assuming the zip file is named as `func.zip`, unzip the `func.zip` folder: + "func/src" + "func/requirements.txt" + "func/deps" + + ``` - . - ├── deps - │   └── sh-1.12.14-py2.py3-none-any.whl - └── src - └── exclamation.py + Take [exclamation.zip](https://github.com/apache/pulsar/tree/master/tests/docker-images/latest-version-image/python-examples) as an example. The internal structure of the example is as follows. - ``` + ```text + + . + ├── deps + │   └── sh-1.12.14-py2.py3-none-any.whl + └── src + └── exclamation.py + + ``` 2. Run the Python Function. - (1) Copy the ZIP file to the Pulsar image. - - ```bash - - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar - - ``` - - (2) Run the Python function using the following command. + (1) Copy the ZIP file to the Pulsar image. - ```bash - - ./bin/pulsar-admin functions localrun \ - --classname exclamation \ - --py \ - --inputs persistent://public/default/in-topic \ - --output persistent://public/default/out-topic \ - --tenant public \ - --namespace default \ - --name PythonFunction - - ``` - - The following log indicates that the Python function starts successfully. + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` - ```text + (2) Run the Python function using the following command. + + ```bash + + ./bin/pulsar-admin functions localrun \ + --classname exclamation \ + --py \ + --inputs persistent://public/default/in-topic \ + --output persistent://public/default/out-topic \ + --tenant public \ + --namespace default \ + --name PythonFunction + + ``` - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... + The following log indicates that the Python function starts successfully. - ``` + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ### PIP @@ -332,52 +336,54 @@ The PIP method is only supported in Kubernetes runtime. To package a function wi 1. Configure the `functions_worker.yml` file. - ```text - - #### Kubernetes Runtime #### - installUserCodeDependencies: true - - ``` + ```text + + #### Kubernetes Runtime #### + installUserCodeDependencies: true + + ``` 2. Write your Python Function. - ``` - from pulsar import Function - import js2xml - - # The classic ExclamationFunction that appends an exclamation at the end - # of the input - class ExclamationFunction(Function): - def __init__(self): - pass - - def process(self, input, context): - // add your logic - return input + '!' - - ``` + ``` + + from pulsar import Function + import js2xml + + # The classic ExclamationFunction that appends an exclamation at the end + # of the input + class ExclamationFunction(Function): + def __init__(self): + pass + + def process(self, input, context): + // add your logic + return input + '!' + + ``` - You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. + You can introduce additional dependencies. When Python Function detects that the file currently used is `whl` and the `installUserCodeDependencies` parameter is specified, the system uses the `pip install` command to install the dependencies required in Python Function. 3. Generate the `whl` file. - ```shell script - $ cd $PULSAR_HOME/pulsar-functions/scripts/python - $ chmod +x generate.sh - $ ./generate.sh - # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 - - ``` - - The output is written in `/path/to/python/output`: - - ```text + ```shell script + + $ cd $PULSAR_HOME/pulsar-functions/scripts/python + $ chmod +x generate.sh + $ ./generate.sh + # e.g: ./generate.sh /path/to/python /path/to/python/output 1.0.0 + + ``` - -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl - -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz - -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl + The output is written in `/path/to/python/output`: - ``` + ```text + + -rw-r--r-- 1 root staff 1.8K 8 27 14:29 pulsarfunction-1.0.0-py2-none-any.whl + -rw-r--r-- 1 root staff 1.4K 8 27 14:29 pulsarfunction-1.0.0.tar.gz + -rw-r--r-- 1 root staff 0B 8 27 14:29 pulsarfunction.whl + + ``` ## Go @@ -385,104 +391,107 @@ To package a function in Go, complete the following steps. 1. Write a Go function. - Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". - - ``` - import ( - "context" - "fmt" + Currently, Go function can be **only** implemented using SDK and the interface of the function is exposed in the form of SDK. Before using the Go function, you need to import "github.com/apache/pulsar/pulsar-function-go/pf". - "github.com/apache/pulsar/pulsar-function-go/pf" - ) + ``` + + import ( + "context" + "fmt" + + "github.com/apache/pulsar/pulsar-function-go/pf" + ) + + func HandleRequest(ctx context.Context, input []byte) error { + fmt.Println(string(input) + "!") + return nil + } + + func main() { + pf.Start(HandleRequest) + } + + ``` - func HandleRequest(ctx context.Context, input []byte) error { - fmt.Println(string(input) + "!") - return nil - } + You can use context to connect to the Go function. - func main() { - pf.Start(HandleRequest) + ``` + + if fc, ok := pf.FromContext(ctx); ok { + fmt.Printf("function ID is:%s, ", fc.GetFuncID()) + fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) } + + ``` - ``` - - You can use context to connect to the Go function. - - - ``` - if fc, ok := pf.FromContext(ctx); ok { - fmt.Printf("function ID is:%s, ", fc.GetFuncID()) - fmt.Printf("function version is:%s\n", fc.GetFuncVersion()) - } + When writing a Go function, remember that + - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. + - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: ``` - - When writing a Go function, remember that - - In `main()`, you **only** need to register the function name to `Start()`. **Only** one function name is received in `Start()`. - - Go function uses Go reflection, which is based on the received function name, to verify whether the parameter list and returned value list are correct. The parameter list and returned value list **must be** one of the following sample functions: - - - ``` - func () - func () error - func (input) error - func () (output, error) - func (input) (output, error) - func (context.Context) error - func (context.Context, input) error - func (context.Context) (output, error) - func (context.Context, input) (output, error) - - ``` + + func () + func () error + func (input) error + func () (output, error) + func (input) (output, error) + func (context.Context) error + func (context.Context, input) error + func (context.Context) (output, error) + func (context.Context, input) (output, error) + + ``` 2. Build the Go function. - ``` - go build .go - - ``` + ``` + + go build .go + + ``` 3. Run the Go Function. - (1) Copy the Go function file to the Pulsar image. - - ```bash - - docker exec -it [CONTAINER ID] /bin/bash - docker cp CONTAINER ID:/pulsar + (1) Copy the Go function file to the Pulsar image. - ``` - - (2) Run the Go function with the following command. - - ``` - ./bin/pulsar-admin functions localrun \ - --go [your go function path] - --inputs [input topics] \ - --output [output topic] \ - --tenant [default:public] \ - --namespace [default:default] \ - --name [custom unique go function name] - - ``` + ```bash + + docker exec -it [CONTAINER ID] /bin/bash + docker cp CONTAINER ID:/pulsar + + ``` - The following log indicates that the Go function starts successfully. + (2) Run the Go function with the following command. - ```text + ``` + + ./bin/pulsar-admin functions localrun \ + --go [your go function path] + --inputs [input topics] \ + --output [output topic] \ + --tenant [default:public] \ + --namespace [default:default] \ + --name [custom unique go function name] + + ``` - ... - 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully - ... + The following log indicates that the Go function starts successfully. - ``` + ```text + + ... + 07:55:03.724 [main] INFO org.apache.pulsar.functions.runtime.ProcessRuntime - Started process successfully + ... + + ``` ## Start Functions in cluster mode If you want to start a function in cluster mode, replace `localrun` with `create` in the commands above. The following log indicates that your function starts successfully. ```text - + "Created successfully" - - ``` + + ``` For information about parameters on `--classname`, `--jar`, `--py`, `--go`, `--inputs`, run the command `./bin/pulsar-admin functions` or see [here](reference-pulsar-admin.md#functions). \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-runtime.md b/site2/website-next/versioned_docs/version-2.8.0/functions-runtime.md index 9aeb676c7ca17..59c624de03f00 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-runtime.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-runtime.md @@ -68,7 +68,6 @@ The Kubernetes runtime supports secrets, so you can create a Kubernetes secret a :::tip - For the rules of translating Pulsar object names into Kubernetes resource labels, see [here](admin-api-overview.md#how-to-define-pulsar-resource-names-when-running-pulsar-in-kubernetes). ::: @@ -400,4 +399,5 @@ pulsarServiceUrl: pulsar://broker.pulsar:6650/ # or pulsar+ssl://pulsar-prod-bro pulsarWebServiceUrl: http://broker.pulsar:8080/ # or https://pulsar-prod-broker.pulsar:8443/ when using TLS useTls: true # when using TLS, critical! -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/functions-worker.md b/site2/website-next/versioned_docs/version-2.8.0/functions-worker.md index 81c76ce6533ca..bffcd10f2a840 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/functions-worker.md +++ b/site2/website-next/versioned_docs/version-2.8.0/functions-worker.md @@ -60,40 +60,41 @@ If you want to use Stateful-Functions related functions (for example, `putState Currently, the service uses the NAR package, so you need to set the configuration in `bookkeeper.conf`. - ```text - - extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent - - ``` + ```text + + extraServerComponents=org.apache.bookkeeper.stream.server.StreamStorageLifecycleComponent + + ``` After starting bookie, use the following methods to check whether the streamStorage service is started correctly. Input: - ```shell + ```shell + + telnet localhost 4181 + + ``` - telnet localhost 4181 - - ``` Output: - ```text - - Trying 127.0.0.1... - Connected to localhost. - Escape character is '^]'. - - ``` + ```text + + Trying 127.0.0.1... + Connected to localhost. + Escape character is '^]'. + + ``` 2. Turn on this function in `functions_worker.yml`. - ```text - - stateStorageServiceUrl: bk://:4181 + ```text + + stateStorageServiceUrl: bk://:4181 + + ``` - ``` - - `bk-service-url` is the service URL pointing to the BookKeeper table service. + `bk-service-url` is the service URL pointing to the BookKeeper table service. ### Start Functions-worker with broker @@ -109,7 +110,9 @@ curl :8080/admin/v2/worker/cluster After entering the command above, a list of active function workers in the cluster is returned. The output is similar to the following. -```json [{"workerId":"","workerHostname":"","port":8080}] +```json + +[{"workerId":"","workerHostname":"","port":8080}] ``` @@ -164,6 +167,7 @@ If you want to enable security on functions workers, you *should*: To enable TLS transport encryption, configure the following settings. ``` + useTLS: true pulsarServiceUrl: pulsar+ssl://localhost:6651/ pulsarWebServiceUrl: https://localhost:8443 @@ -191,6 +195,7 @@ Substitute the *providers list* with the providers you want to enable. ::: ``` + authenticationEnabled: true authenticationProviders: [ provider1, provider2 ] @@ -200,6 +205,7 @@ For *TLS Authentication* provider, follow the example below to add the necessary See [TLS Authentication](security-tls-authentication) for more details. ``` + brokerClientAuthenticationPlugin: org.apache.pulsar.client.impl.auth.AuthenticationTls brokerClientAuthenticationParameters: tlsCertFile:/path/to/admin.cert.pem,tlsKeyFile:/path/to/admin.key-pk8.pem @@ -212,6 +218,7 @@ For *SASL Authentication* provider, add `saslJaasClientAllowedIds` and `saslJaas under `properties` if needed. ``` + properties: saslJaasClientAllowedIds: .*pulsar.* saslJaasBrokerSectionName: Broker @@ -223,10 +230,11 @@ See [Token Authentication](security-jwt) for more details. Note: key files must be DER-encoded ``` + properties: tokenSecretKey: file://my/secret.key # If using public/private - # tokenPublicKey: file:///path/to/public.key + # tokenPublicKey: file:///path/to/public.key ``` @@ -345,6 +353,7 @@ Use the `Run-separately` mode in the following cases: **Error message: Namespace missing local cluster name in clusters list** ``` + Failed to get partitioned topic metadata: org.apache.pulsar.client.api.PulsarClientException$BrokerMetadataException: Namespace missing local cluster name in clusters list: local_cluster=xyz ns=public/functions clusters=[standalone] ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-clients.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-clients.md index 7860f0a136aa5..4194e347e2a09 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-clients.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-clients.md @@ -1,7 +1,7 @@ --- id: client-libraries title: Pulsar client libraries -sidebar_label: Overview +sidebar_label: "Overview" original_id: client-libraries --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-concepts-and-architecture.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-concepts-and-architecture.md index f6f5ff1aee4db..be300de5ab86c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-concepts-and-architecture.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-concepts-and-architecture.md @@ -1,7 +1,7 @@ --- id: concepts-architecture title: Pulsar concepts and architecture -sidebar_label: Concepts and architecture +sidebar_label: "Concepts and architecture" original_id: concepts-architecture --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-docker.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-docker.md index 087acb6124234..2d4dc043786d7 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-docker.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-docker.md @@ -1,7 +1,7 @@ --- id: standalone-docker title: Set up a standalone Pulsar in Docker -sidebar_label: Run Pulsar in Docker +sidebar_label: "Run Pulsar in Docker" original_id: standalone-docker --- @@ -20,15 +20,15 @@ and follow the instructions for your OS. * For MacOS, Linux, and Windows: ```shell - + $ docker run -it \ - -p 6650:6650 \ - -p 8080:8080 \ - --mount source=pulsardata,target=/pulsar/data \ - --mount source=pulsarconf,target=/pulsar/conf \ - apachepulsar/pulsar:@pulsar:version@ \ - bin/pulsar standalone - + -p 6650:6650 \ + -p 8080:8080 \ + --mount source=pulsardata,target=/pulsar/data \ + --mount source=pulsarconf,target=/pulsar/conf \ + apachepulsar/pulsar:@pulsar:version@ \ + bin/pulsar standalone + ``` A few things to note about this command: @@ -39,6 +39,7 @@ time the container is restarted. For details on the volumes you can use `docker If you start Pulsar successfully, you will see `INFO`-level log messages like this: ``` + 2017-08-09 22:34:04,030 - INFO - [main:WebService@213] - Web Service started at http://127.0.0.1:8080 2017-08-09 22:34:04,038 - INFO - [main:PulsarService@335] - messaging service is ready, bootstrap service on port=8080, broker url=pulsar://127.0.0.1:6650, cluster=standalone, configs=org.apache.pulsar.broker.ServiceConfiguration@4db60246 ... @@ -47,10 +48,10 @@ If you start Pulsar successfully, you will see `INFO`-level log messages like th :::tip - When you start a local standalone cluster, a `public/default` ::: + namespace is created automatically. The namespace is used for development purposes. All Pulsar topics are managed within namespaces. For more information, see [Topics](concepts-messaging.md#topics). @@ -179,3 +180,4 @@ The output is something like this: } ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-helm.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-helm.md index cc857d099642b..c5c428a1be4d2 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-helm.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-helm.md @@ -1,7 +1,7 @@ --- id: kubernetes-helm title: Get started in Kubernetes -sidebar_label: Run Pulsar in Kubernetes +sidebar_label: "Run Pulsar in Kubernetes" original_id: kubernetes-helm --- @@ -39,134 +39,134 @@ We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start 1. Create a Kubernetes cluster on Minikube. - ```bash + ```bash + + minikube start --memory=8192 --cpus=4 --kubernetes-version= + + ``` - minikube start --memory=8192 --cpus=4 --kubernetes-version= - - ``` - - The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. + The `` can be any [Kubernetes version supported by your Minikube installation](https://minikube.sigs.k8s.io/docs/reference/configuration/kubernetes/), such as `v1.16.1`. 2. Set `kubectl` to use Minikube. - ```bash - - kubectl config use-context minikube - - ``` + ```bash + + kubectl config use-context minikube + + ``` 3. To use the [Kubernetes Dashboard](https://kubernetes.io/docs/tasks/access-application-cluster/web-ui-dashboard/) with the local Kubernetes cluster on Minikube, enter the command below: - ```bash - - minikube dashboard + ```bash + + minikube dashboard + + ``` - ``` - The command automatically triggers opening a webpage in your browser. + The command automatically triggers opening a webpage in your browser. ## Step 1: Install Pulsar Helm chart 0. Add Pulsar charts repo. - ```bash - - helm repo add apache https://pulsar.apache.org/charts - - ``` - - ```bash - - helm repo update + ```bash + + helm repo add apache https://pulsar.apache.org/charts + + ``` - ``` + ```bash + + helm repo update + + ``` 1. Clone the Pulsar Helm chart repository. - ```bash - - git clone https://github.com/apache/pulsar-helm-chart - cd pulsar-helm-chart - - ``` + ```bash + + git clone https://github.com/apache/pulsar-helm-chart + cd pulsar-helm-chart + + ``` 2. Run the script `prepare_helm_release.sh` to create secrets required for installing the Apache Pulsar Helm chart. The username `pulsar` and password `pulsar` are used for logging into the Grafana dashboard and Pulsar Manager. - ```bash - - ./scripts/pulsar/prepare_helm_release.sh \ - -n pulsar \ - -k pulsar-mini \ - -c - - ``` + ```bash + + ./scripts/pulsar/prepare_helm_release.sh \ + -n pulsar \ + -k pulsar-mini \ + -c + + ``` 3. Use the Pulsar Helm chart to install a Pulsar cluster to Kubernetes. > **NOTE** > You need to specify `--set initialize=true` when installing Pulsar the first time. This command installs and starts Apache Pulsar. - ```bash - - helm install \ - --values examples/values-minikube.yaml \ - --set initialize=true \ - --namespace pulsar \ - pulsar-mini apache/pulsar - - ``` + ```bash + + helm install \ + --values examples/values-minikube.yaml \ + --set initialize=true \ + --namespace pulsar \ + pulsar-mini apache/pulsar + + ``` 4. Check the status of all pods. - ```bash - - kubectl get pods -n pulsar - - ``` - - If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. - - **Output** - - ```bash - - NAME READY STATUS RESTARTS AGE - pulsar-mini-bookie-0 1/1 Running 0 9m27s - pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s - pulsar-mini-broker-0 1/1 Running 0 9m27s - pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s - pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s - pulsar-mini-proxy-0 1/1 Running 0 9m27s - pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s - pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s - pulsar-mini-toolset-0 1/1 Running 0 9m27s - pulsar-mini-zookeeper-0 1/1 Running 0 9m27s - - ``` + ```bash + + kubectl get pods -n pulsar + + ``` + + If all pods start up successfully, you can see that the `STATUS` is changed to `Running` or `Completed`. + + **Output** + + ```bash + + NAME READY STATUS RESTARTS AGE + pulsar-mini-bookie-0 1/1 Running 0 9m27s + pulsar-mini-bookie-init-5gphs 0/1 Completed 0 9m27s + pulsar-mini-broker-0 1/1 Running 0 9m27s + pulsar-mini-grafana-6b7bcc64c7-4tkxd 1/1 Running 0 9m27s + pulsar-mini-prometheus-5fcf5dd84c-w8mgz 1/1 Running 0 9m27s + pulsar-mini-proxy-0 1/1 Running 0 9m27s + pulsar-mini-pulsar-init-t7cqt 0/1 Completed 0 9m27s + pulsar-mini-pulsar-manager-9bcbb4d9f-htpcs 1/1 Running 0 9m27s + pulsar-mini-toolset-0 1/1 Running 0 9m27s + pulsar-mini-zookeeper-0 1/1 Running 0 9m27s + + ``` 5. Check the status of all services in the namespace `pulsar`. - ```bash - - kubectl get services -n pulsar - - ``` - - **Output** - - - ```bash - - NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE - pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m - pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m - pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m - pulsar-mini-prometheus ClusterIP None 9090/TCP 11m - pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m - pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m - pulsar-mini-toolset ClusterIP None 11m - pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m - - ``` + ```bash + + kubectl get services -n pulsar + + ``` + + **Output** + + ```bash + + NAME TYPE CLUSTER-IP EXTERNAL-IP PORT(S) AGE + pulsar-mini-bookie ClusterIP None 3181/TCP,8000/TCP 11m + pulsar-mini-broker ClusterIP None 8080/TCP,6650/TCP 11m + pulsar-mini-grafana LoadBalancer 10.106.141.246 3000:31905/TCP 11m + pulsar-mini-prometheus ClusterIP None 9090/TCP 11m + pulsar-mini-proxy LoadBalancer 10.97.240.109 80:32305/TCP,6650:31816/TCP 11m + pulsar-mini-pulsar-manager LoadBalancer 10.103.192.175 9527:30190/TCP 11m + pulsar-mini-toolset ClusterIP None 11m + pulsar-mini-zookeeper ClusterIP None 2888/TCP,3888/TCP,2181/TCP 11m + + ``` ## Step 2: Use pulsar-admin to create Pulsar tenants/namespaces/topics @@ -174,85 +174,85 @@ We use [Minikube](https://minikube.sigs.k8s.io/docs/start/) in this quick start 1. Enter the `toolset` container. - ```bash - - kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash - - ``` + ```bash + + kubectl exec -it -n pulsar pulsar-mini-toolset-0 -- /bin/bash + + ``` 2. In the `toolset` container, create a tenant named `apache`. - ```bash - - bin/pulsar-admin tenants create apache - - ``` - - Then you can list the tenants to see if the tenant is created successfully. - - ```bash - - bin/pulsar-admin tenants list - - ``` + ```bash + + bin/pulsar-admin tenants create apache + + ``` - You should see a similar output as below. The tenant `apache` has been successfully created. + Then you can list the tenants to see if the tenant is created successfully. - ```bash + ```bash + + bin/pulsar-admin tenants list + + ``` - "apache" - "public" - "pulsar" + You should see a similar output as below. The tenant `apache` has been successfully created. - ``` + ```bash + + "apache" + "public" + "pulsar" + + ``` 3. In the `toolset` container, create a namespace named `pulsar` in the tenant `apache`. - ```bash + ```bash + + bin/pulsar-admin namespaces create apache/pulsar + + ``` - bin/pulsar-admin namespaces create apache/pulsar + Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. - ``` + ```bash + + bin/pulsar-admin namespaces list apache + + ``` - Then you can list the namespaces of tenant `apache` to see if the namespace is created successfully. + You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. - ```bash - - bin/pulsar-admin namespaces list apache - - ``` - - You should see a similar output as below. The namespace `apache/pulsar` has been successfully created. - - ```bash - - "apache/pulsar" - - ``` + ```bash + + "apache/pulsar" + + ``` 4. In the `toolset` container, create a topic `test-topic` with `4` partitions in the namespace `apache/pulsar`. - ```bash - - bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 - - ``` + ```bash + + bin/pulsar-admin topics create-partitioned-topic apache/pulsar/test-topic -p 4 + + ``` 5. In the `toolset` container, list all the partitioned topics in the namespace `apache/pulsar`. - ```bash - - bin/pulsar-admin topics list-partitioned-topics apache/pulsar + ```bash + + bin/pulsar-admin topics list-partitioned-topics apache/pulsar + + ``` - ``` + Then you can see all the partitioned topics in the namespace `apache/pulsar`. - Then you can see all the partitioned topics in the namespace `apache/pulsar`. - - ```bash - - "persistent://apache/pulsar/test-topic" - - ``` + ```bash + + "persistent://apache/pulsar/test-topic" + + ``` ## Step 3: Use Pulsar client to produce and consume messages @@ -307,6 +307,7 @@ minikube service pulsar-mini-proxy -n pulsar At this point, you can get the service URLs to connect to your Pulsar client. Here are URL examples: ``` + webServiceUrl=http://127.0.0.1:61853/ brokerServiceUrl=pulsar://127.0.0.1:61854/ @@ -318,88 +319,88 @@ Then you can proceed with the following steps: 2. Decompress the tarball based on your download file. - ```bash - - tar -xf .tar.gz - - ``` + ```bash + + tar -xf .tar.gz + + ``` 3. Expose `PULSAR_HOME`. - (1) Enter the directory of the decompressed download file. - - (2) Expose `PULSAR_HOME` as the environment variable. + (1) Enter the directory of the decompressed download file. - ```bash + (2) Expose `PULSAR_HOME` as the environment variable. - export PULSAR_HOME=$(pwd) - - ``` + ```bash + + export PULSAR_HOME=$(pwd) + + ``` 4. Configure the Pulsar client. - In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. + In the `${PULSAR_HOME}/conf/client.conf` file, replace `webServiceUrl` and `brokerServiceUrl` with the service URLs you get from the above steps. 5. Create a subscription to consume messages from `apache/pulsar/test-topic`. - ```bash - - bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 - - ``` + ```bash + + bin/pulsar-client consume -s sub apache/pulsar/test-topic -n 0 + + ``` 6. Open a new terminal. In the new terminal, create a producer and send 10 messages to the `test-topic` topic. - ```bash - - bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 - - ``` + ```bash + + bin/pulsar-client produce apache/pulsar/test-topic -m "---------hello apache pulsar-------" -n 10 + + ``` 7. Verify the results. - - From the producer side - - **Output** - - The messages have been produced successfully. - - ```bash - - 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced - - ``` - - - From the consumer side - - **Output** - - At the same time, you can receive the messages as below. - - ```bash - - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - ----- got message ----- - ---------hello apache pulsar------- - - ``` + - From the producer side + + **Output** + + The messages have been produced successfully. + + ```bash + + 18:15:15.489 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 10 messages successfully produced + + ``` + + - From the consumer side + + **Output** + + At the same time, you can receive the messages as below. + + ```bash + + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + ----- got message ----- + ---------hello apache pulsar------- + + ``` ## Step 4: Use Pulsar Manager to manage the cluster @@ -407,20 +408,20 @@ Then you can proceed with the following steps: 1. By default, the `Pulsar Manager` is exposed as a separate `LoadBalancer`. You can open the Pulsar Manager UI using the following command: - ```bash - - minikube service -n pulsar pulsar-mini-pulsar-manager - - ``` + ```bash + + minikube service -n pulsar pulsar-mini-pulsar-manager + + ``` 2. The Pulsar Manager UI will be open in your browser. You can use the username `pulsar` and password `pulsar` to log into Pulsar Manager. 3. In Pulsar Manager UI, you can create an environment. - - Click `New Environment` button in the top-left corner. - - Type `pulsar-mini` for the field `Environment Name` in the popup window. - - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. - - Click `Confirm` button in the popup window. + - Click `New Environment` button in the top-left corner. + - Type `pulsar-mini` for the field `Environment Name` in the popup window. + - Type `http://pulsar-mini-broker:8080` for the field `Service URL` in the popup window. + - Click `Confirm` button in the popup window. 4. After successfully created an environment, you are redirected to the `tenants` page of that environment. Then you can create `tenants`, `namespaces` and `topics` using the Pulsar Manager. @@ -430,11 +431,11 @@ Grafana is an open-source visualization tool, which can be used for visualizing 1. By default, the Grafana is exposed as a separate `LoadBalancer`. You can open the Grafana UI using the following command: - ```bash - - minikube service pulsar-mini-grafana -n pulsar - - ``` + ```bash + + minikube service pulsar-mini-grafana -n pulsar + + ``` 2. The Grafana UI is open in your browser. You can use the username `pulsar` and password `pulsar` to log into the Grafana Dashboard. diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-pulsar.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-pulsar.md index 5fbc1e23a877d..d1dd8bf605992 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-pulsar.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-pulsar.md @@ -1,7 +1,7 @@ --- id: pulsar-2.0 title: Pulsar 2.0 -sidebar_label: Pulsar 2.0 +sidebar_label: "Pulsar 2.0" original_id: pulsar-2.0 --- @@ -34,6 +34,7 @@ Prior to version 2.0, *all* Pulsar topics had the following form: {persistent|non-persistent}://property/cluster/namespace/topic ``` + Two important changes have been made in Pulsar 2.0: * There is no longer a [cluster component](#no-cluster) diff --git a/site2/website-next/versioned_docs/version-2.8.0/getting-started-standalone.md b/site2/website-next/versioned_docs/version-2.8.0/getting-started-standalone.md index 5e84f37144897..cb8c99745a2cc 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/getting-started-standalone.md +++ b/site2/website-next/versioned_docs/version-2.8.0/getting-started-standalone.md @@ -2,7 +2,7 @@ slug: / id: standalone title: Set up a standalone Pulsar locally -sidebar_label: Run Pulsar locally +sidebar_label: "Run Pulsar locally" original_id: standalone --- @@ -31,7 +31,6 @@ By default, Pulsar allocates 2G JVM heap memory to start. It can be changed in ` :::note - Broker is only supported on 64-bit JVM. ::: @@ -49,9 +48,9 @@ To get started with Pulsar, download a binary tarball release in one of the foll * use [wget](https://www.gnu.org/software/wget): ```shell - + $ wget pulsar:binary_release_url - + ``` After you download the tarball, untar it and use the `cd` command to navigate to the resulting directory: @@ -86,10 +85,8 @@ Directory | Contains :::tip If you want to use builtin connectors and tiered storage offloaders, you can install them according to the following instructions: - * [Install builtin connectors (optional)](#install-builtin-connectors-optional) * [Install tiered storage offloaders (optional)](#install-tiered-storage-offloaders-optional) - Otherwise, skip this step and perform the next step [Start Pulsar standalone](#start-pulsar-standalone). Pulsar can be successfully installed without installing bulitin connectors and tiered storage offloaders. ::: @@ -108,9 +105,9 @@ To enable those `builtin` connectors, you can download the connectors tarball re * use [wget](https://www.gnu.org/software/wget): ```shell - + $ wget pulsar:connector_release_url/{connector}-@pulsar:version@.nar - + ``` After you download the nar file, copy the file to the `connectors` directory in the pulsar directory. @@ -129,10 +126,8 @@ pulsar-io-aerospike-@pulsar:version@.nar :::note - * If you are running Pulsar in a bare metal cluster, make sure `connectors` tarball is unzipped in every pulsar directory of the broker (or in every pulsar directory of function-worker if you are running a separate worker cluster for Pulsar Functions). - * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled [all builtin connectors](io-overview.md#working-with-connectors). @@ -142,7 +137,6 @@ you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pul :::tip - Since `2.2.0` release, Pulsar releases a separate binary distribution, containing the tiered storage offloaders. To enable tiered storage feature, follow the instructions below; otherwise skip this section. @@ -159,9 +153,9 @@ To get started with [tiered storage offloaders](concepts-tiered-storage), you ne * use [wget](https://www.gnu.org/software/wget): ```shell - + $ wget pulsar:offloader_release_url - + ``` After you download the tarball, untar the offloaders package and copy the offloaders as `offloaders` @@ -185,9 +179,7 @@ For more information on how to configure tiered storage, see [Tiered storage coo :::note - * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's pulsar directory. - * If you are [running Pulsar in Docker](getting-started-docker.md) or deploying Pulsar using a docker image (e.g. [K8S](deploy-kubernetes.md) or [DCOS](deploy-dcos)), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. @@ -215,10 +207,10 @@ If you have started Pulsar successfully, you will see `INFO`-level log messages :::tip - -* The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. +* The service is running on your terminal, which is under your direct control. If you need to run other commands, open a new terminal window. ::: + You can also run the service as a background process using the `pulsar-daemon start standalone` command. For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). > > * By default, there is no encryption, authentication, or authorization configured. Apache Pulsar can be accessed from remote server without any authorization. Please do check [Security Overview](security-overview) document to secure your deployment. @@ -242,13 +234,13 @@ $ bin/pulsar-client consume my-topic -s "first-subscription" If the message has been successfully consumed, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 09:56:55.566 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.MultiTopicsConsumerImpl - [TopicsConsumerFakeTopicNamee2df9] [first-subscription] Success subscribe new topic my-topic in topics consumer, partitions: 4, allTopicPartitionsNumber: 4 ``` :::tip - As you have noticed that we do not explicitly create the `my-topic` topic, to which we consume the message. When you consume a message to a topic that does not yet exist, Pulsar creates that topic for you automatically. Producing a message to a topic that does not exist will automatically create that topic for you as well. ::: @@ -266,6 +258,7 @@ $ bin/pulsar-client produce my-topic --messages "hello-pulsar" If the message has been successfully published to the topic, you will see a confirmation like the following in the `pulsar-client` logs: ``` + 13:09:39.356 [main] INFO org.apache.pulsar.client.cli.PulsarClientTool - 1 messages successfully produced ``` @@ -276,9 +269,8 @@ Press `Ctrl+C` to stop a local standalone Pulsar. :::tip - If the service runs as a background process using the `pulsar-daemon start standalone` command, then use the `pulsar-daemon stop standalone` command to stop the service. - For more information, see [pulsar-daemon](https://pulsar.apache.org/docs/en/reference-cli-tools/#pulsar-daemon). ::: + diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-deploy.md b/site2/website-next/versioned_docs/version-2.8.0/helm-deploy.md index 907affd58a490..e64ca2de9b1cf 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-deploy.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-deploy.md @@ -1,7 +1,7 @@ --- id: helm-deploy title: Deploy Pulsar cluster using Helm -sidebar_label: Deployment +sidebar_label: "Deployment" original_id: helm-deploy --- @@ -74,7 +74,6 @@ To use local persistent volumes as the persistent storage for Helm release, you :::note - Before installing the production instance of Pulsar, ensure to plan the storage settings to avoid extra storage migration work. Because after initial installation, you must edit Kubernetes objects manually if you want to change storage settings. ::: @@ -212,6 +211,7 @@ certs: type: selfsigning ``` + You can also customize the generated TLS certificates by configuring the fields as the following. ```yaml @@ -342,6 +342,7 @@ To use local persistent volumes as the persistent storage, you need to install a One of the easiest way to get started is to use the local storage provisioner provided along with the Pulsar Helm chart. ``` + helm repo add streamnative https://charts.streamnative.io helm repo update helm install pulsar-storage-provisioner streamnative/local-storage-provisioner @@ -380,9 +381,9 @@ The `prepare_helm_release` creates the following resources: - A Kubernetes namespace for installing the Pulsar release - JWT secret keys and tokens for three super users: `broker-admin`, `proxy-admin`, and `admin`. By default, it generates an asymmetric pubic/private key pair. You can choose to generate a symmetric secret key by specifying `--symmetric`. - - `proxy-admin` role is used for proxies to communicate to brokers. - - `broker-admin` role is used for inter-broker communications. - - `admin` role is used by the admin tools. + - `proxy-admin` role is used for proxies to communicate to brokers. + - `broker-admin` role is used for inter-broker communications. + - `admin` role is used by the admin tools. ## Deploy Pulsar cluster using Helm @@ -407,7 +408,6 @@ helm install pulsar apache/pulsar \ :::note - For the first deployment, add `--set initialize=true` option to initialize bookie and Pulsar cluster metadata. ::: @@ -435,3 +435,4 @@ To find the IP addresses of those components, run the following command: kubectl get service -n ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-install.md b/site2/website-next/versioned_docs/version-2.8.0/helm-install.md index 538e198c61681..7db5d915fe216 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-install.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-install.md @@ -1,7 +1,7 @@ --- id: helm-install title: Install Apache Pulsar using Helm -sidebar_label: Install +sidebar_label: "Install" original_id: helm-install --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-overview.md b/site2/website-next/versioned_docs/version-2.8.0/helm-overview.md index 18997006ff988..1f9d2b066205f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-overview.md @@ -1,7 +1,7 @@ --- id: helm-overview title: Apache Pulsar Helm Chart -sidebar_label: Overview +sidebar_label: "Overview" original_id: helm-overview --- @@ -18,41 +18,41 @@ The Apache Pulsar Helm chart is one of the most convenient ways to operate Pulsa This chart includes all the components for a complete experience, but each part can be configured to be installed separately. - Pulsar core components: - - ZooKeeper - - Bookies - - Brokers - - Function workers - - Proxies + - ZooKeeper + - Bookies + - Brokers + - Function workers + - Proxies - Control Center: - - Pulsar Manager - - Prometheus - - Grafana + - Pulsar Manager + - Prometheus + - Grafana It includes support for: - Security - - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) - - self-signed - - [Let's Encrypt](https://letsencrypt.org/) - - TLS Encryption - - Proxy - - Broker - - Toolset - - Bookie - - ZooKeeper - - Authentication - - JWT - - Authorization + - Automatically provisioned TLS certificates, using [Jetstack](https://www.jetstack.io/)'s [cert-manager](https://cert-manager.io/docs/) + - self-signed + - [Let's Encrypt](https://letsencrypt.org/) + - TLS Encryption + - Proxy + - Broker + - Toolset + - Bookie + - ZooKeeper + - Authentication + - JWT + - Authorization - Storage - - Non-persistence storage - - Persistence volume - - Local persistent volumes + - Non-persistence storage + - Persistence volume + - Local persistent volumes - Functions - - Kubernetes Runtime - - Process Runtime - - Thread Runtime + - Kubernetes Runtime + - Process Runtime + - Thread Runtime - Operations - - Independent image versions for all components, enabling controlled upgrades + - Independent image versions for all components, enabling controlled upgrades ## Pulsar Helm chart quick start @@ -102,7 +102,7 @@ For the purposes of continuity, these charts have some Kubernetes objects that c It is recommended to *consciously* remove these items, as they affect re-deployment. * PVCs for stateful data: *consciously* remove these items. - - ZooKeeper: This is your metadata. - - BookKeeper: This is your data. - - Prometheus: This is your metrics data, which can be safely removed. + - ZooKeeper: This is your metadata. + - BookKeeper: This is your data. + - Prometheus: This is your metrics data, which can be safely removed. * Secrets: if the secrets are generated by the [prepare release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/prepare_helm_release.sh), they contain secret keys and tokens. You can use the [cleanup release script](https://github.com/apache/pulsar-helm-chart/blob/master/scripts/pulsar/cleanup_helm_release.sh) to remove these secrets and tokens as needed. diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-prepare.md b/site2/website-next/versioned_docs/version-2.8.0/helm-prepare.md index 3584480a4aaa9..705e35794a39f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-prepare.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-prepare.md @@ -1,7 +1,7 @@ --- id: helm-prepare title: Prepare Kubernetes resources -sidebar_label: Prepare +sidebar_label: "Prepare" original_id: helm-prepare --- @@ -14,7 +14,7 @@ For a fully functional Pulsar cluster, you need a few resources before deploying - [Google Kubernetes Engine](#google-kubernetes-engine) - [Manual cluster creation](#manual-cluster-creation) - [Scripted cluster creation](#scripted-cluster-creation) - - [Create cluster with local SSDs](#create-cluster-with-local-ssds) + - [Create cluster with local SSDs](#create-cluster-with-local-ssds) - [Next Steps](#next-steps) ## Google Kubernetes Engine @@ -24,7 +24,7 @@ To get started easier, a script is provided to create the cluster automatically. - [Google Kubernetes Engine](#google-kubernetes-engine) - [Manual cluster creation](#manual-cluster-creation) - [Scripted cluster creation](#scripted-cluster-creation) - - [Create cluster with local SSDs](#create-cluster-with-local-ssds) + - [Create cluster with local SSDs](#create-cluster-with-local-ssds) - [Next Steps](#next-steps) ### Manual cluster creation @@ -86,9 +86,11 @@ PROJECT= scripts/pulsar/gke_bootstrap_script.sh down To install a Pulsar Helm chart using local persistent volumes, you need to create a GKE cluster with local SSDs. You can do so Specifying the `USE_LOCAL_SSD` to be `true` in the following command to create a Pulsar cluster with local SSDs. ``` + PROJECT= USE_LOCAL_SSD=true LOCAL_SSD_COUNT= scripts/pulsar/gke_bootstrap_script.sh up ``` + ## Next Steps Continue with the [installation of the chart](helm-deploy) once you have the cluster up and running. diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-tools.md b/site2/website-next/versioned_docs/version-2.8.0/helm-tools.md index 1a60e4b791b69..773c681f7c331 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-tools.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-tools.md @@ -1,7 +1,7 @@ --- id: helm-tools title: Required tools for deploying Pulsar Helm Chart -sidebar_label: Required Tools +sidebar_label: "Required Tools" original_id: helm-tools --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/helm-upgrade.md b/site2/website-next/versioned_docs/version-2.8.0/helm-upgrade.md index 077045e6062ab..8b00b8ed37553 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/helm-upgrade.md +++ b/site2/website-next/versioned_docs/version-2.8.0/helm-upgrade.md @@ -1,7 +1,7 @@ --- id: helm-upgrade title: Upgrade Pulsar Helm release -sidebar_label: Upgrade +sidebar_label: "Upgrade" original_id: helm-upgrade --- @@ -15,10 +15,10 @@ We also recommend that you need to provide all values using the `helm upgrade -- :::note - You can retrieve your previous `--set` arguments cleanly, with `helm get values `. If you direct this into a file (`helm get values > pulsar.yml`), you can safely ::: + pass this file through `-f`. Thus `helm upgrade apache/pulsar -f pulsar.yaml`. This safely replaces the behavior of `--reuse-values`. ## Steps @@ -29,19 +29,21 @@ To upgrade Apache Pulsar to a newer version, follow these steps: 2. Go through [deployment documentation](helm-deploy) step by step. 3. Extract your previous `--set` arguments with the following command. - ```bash + ```bash + + helm get values > pulsar.yaml + + ``` - helm get values > pulsar.yaml - - ``` 4. Decide all the values you need to set. 5. Perform the upgrade, with all `--set` arguments extracted in step 4. - ```bash - - helm upgrade apache/pulsar \ - --version \ - -f pulsar.yaml \ - --set ... + ```bash + + helm upgrade apache/pulsar \ + --version \ + -f pulsar.yaml \ + --set ... + + ``` - ``` \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-aerospike-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-aerospike-sink.md index 5b4eda521564e..7ff980521a489 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-aerospike-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-aerospike-sink.md @@ -1,7 +1,7 @@ --- id: io-aerospike-sink title: Aerospike sink connector -sidebar_label: Aerospike sink connector +sidebar_label: "Aerospike sink connector" original_id: io-aerospike-sink --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-canal-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-canal-source.md index 367dc8e7aecde..853b387164d65 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-canal-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-canal-source.md @@ -1,7 +1,7 @@ --- id: io-canal-source title: Canal source connector -sidebar_label: Canal source connector +sidebar_label: "Canal source connector" original_id: io-canal-source --- @@ -34,38 +34,38 @@ Before using the Canal connector, you can create a configuration file through on * JSON - ```json - - { - "zkServers": "127.0.0.1:2181", - "batchSize": "5120", - "destination": "example", - "username": "", - "password": "", - "cluster": false, - "singleHostname": "127.0.0.1", - "singlePort": "11111", - } - - ``` + ```json + + { + "zkServers": "127.0.0.1:2181", + "batchSize": "5120", + "destination": "example", + "username": "", + "password": "", + "cluster": false, + "singleHostname": "127.0.0.1", + "singlePort": "11111", + } + + ``` * YAML - You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. - - ```yaml - - configs: - zkServers: "127.0.0.1:2181" - batchSize: 5120 - destination: "example" - username: "" - password: "" - cluster: false - singleHostname: "127.0.0.1" - singlePort: 11111 - - ``` + You can create a YAML file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/canal/src/main/resources/canal-mysql-source-config.yaml) below to your YAML file. + + ```yaml + + configs: + zkServers: "127.0.0.1:2181" + batchSize: 5120 + destination: "example" + username: "" + password: "" + cluster: false + singleHostname: "127.0.0.1" + singlePort: 11111 + + ``` ## Usage @@ -73,171 +73,167 @@ Here is an example of storing MySQL data using the configuration file as above. 1. Start a MySQL server. - ```bash - - $ docker pull mysql:5.7 - $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 - - ``` + ```bash + + $ docker pull mysql:5.7 + $ docker run -d -it --rm --name pulsar-mysql -p 3306:3306 -e MYSQL_ROOT_PASSWORD=canal -e MYSQL_USER=mysqluser -e MYSQL_PASSWORD=mysqlpw mysql:5.7 + + ``` 2. Create a configuration file `mysqld.cnf`. - ```bash - - [mysqld] - pid-file = /var/run/mysqld/mysqld.pid - socket = /var/run/mysqld/mysqld.sock - datadir = /var/lib/mysql - #log-error = /var/log/mysql/error.log - # By default we only accept connections from localhost - #bind-address = 127.0.0.1 - # Disabling symbolic-links is recommended to prevent assorted security risks - symbolic-links=0 - log-bin=mysql-bin - binlog-format=ROW - server_id=1 - - ``` - -3. Copy the configuration file `mysqld.cnf` to MySQL server. + ```bash - - ```bash - - $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ - - ``` - -4. Restart the MySQL server. + [mysqld] + pid-file = /var/run/mysqld/mysqld.pid + socket = /var/run/mysqld/mysqld.sock + datadir = /var/lib/mysql + #log-error = /var/log/mysql/error.log + # By default we only accept connections from localhost + #bind-address = 127.0.0.1 + # Disabling symbolic-links is recommended to prevent assorted security risks + symbolic-links=0 + log-bin=mysql-bin + binlog-format=ROW + server_id=1 + ``` - ```bash +3. Copy the configuration file `mysqld.cnf` to MySQL server. - $ docker restart pulsar-mysql + ```bash + + $ docker cp mysqld.cnf pulsar-mysql:/etc/mysql/mysql.conf.d/ + + ``` - ``` +4. Restart the MySQL server. -5. Create a test database in MySQL server. + ```bash + $ docker restart pulsar-mysql + + ``` - ```bash - - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' +5. Create a test database in MySQL server. - ``` + ```bash + + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal -e 'create database test;' + + ``` 6. Start a Canal server and connect to MySQL server. - ``` - $ docker pull canal/canal-server:v1.1.2 - $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 - - ``` + ``` + + $ docker pull canal/canal-server:v1.1.2 + $ docker run -d -it --link pulsar-mysql -e canal.auto.scan=false -e canal.destinations=test -e canal.instance.master.address=pulsar-mysql:3306 -e canal.instance.dbUsername=root -e canal.instance.dbPassword=canal -e canal.instance.connectionCharset=UTF-8 -e canal.instance.tsdb.enable=true -e canal.instance.gtidon=false --name=pulsar-canal-server -p 8000:8000 -p 2222:2222 -p 11111:11111 -p 11112:11112 -m 4096m canal/canal-server:v1.1.2 + + ``` 7. Start Pulsar standalone. - ```bash - - $ docker pull apachepulsar/pulsar:2.3.0 - $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone - - ``` + ```bash + + $ docker pull apachepulsar/pulsar:2.3.0 + $ docker run -d -it --link pulsar-canal-server -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:2.3.0 bin/pulsar standalone + + ``` 8. Modify the configuration file `canal-mysql-source-config.yaml`. - ```yaml - - configs: - zkServers: "" - batchSize: "5120" - destination: "test" - username: "" - password: "" - cluster: false - singleHostname: "pulsar-canal-server" - singlePort: "11111" - - ``` + ```yaml + + configs: + zkServers: "" + batchSize: "5120" + destination: "test" + username: "" + password: "" + cluster: false + singleHostname: "pulsar-canal-server" + singlePort: "11111" + + ``` 9. Create a consumer file `pulsar-client.py`. - ```python - - import pulsar - - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', - subscription_name='my-sub') + ```python + + import pulsar - while True: - msg = consumer.receive() - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', + subscription_name='my-sub') - client.close() + while True: + msg = consumer.receive() + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) - ``` + client.close() + + ``` 10. Copy the configuration file `canal-mysql-source-config.yaml` and the consumer file `pulsar-client.py` to Pulsar server. - ```bash - - $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ - $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ - - ``` + ```bash + + $ docker cp canal-mysql-source-config.yaml pulsar-standalone:/pulsar/conf/ + $ docker cp pulsar-client.py pulsar-standalone:/pulsar/ + + ``` 11. Download a Canal connector and start it. - - - ```bash - $ docker exec -it pulsar-standalone /bin/bash - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors - $ ./bin/pulsar-admin source localrun \ - --archive ./connectors/pulsar-io-canal-2.3.0.nar \ - --classname org.apache.pulsar.io.canal.CanalStringSource \ - --tenant public \ - --namespace default \ - --name canal \ - --destination-topic-name my-topic \ - --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ - --parallelism 1 - - ``` + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.3.0/connectors/pulsar-io-canal-2.3.0.nar -P connectors + $ ./bin/pulsar-admin source localrun \ + --archive ./connectors/pulsar-io-canal-2.3.0.nar \ + --classname org.apache.pulsar.io.canal.CanalStringSource \ + --tenant public \ + --namespace default \ + --name canal \ + --destination-topic-name my-topic \ + --source-config-file /pulsar/conf/canal-mysql-source-config.yaml \ + --parallelism 1 + + ``` 12. Consume data from MySQL. - ```bash - - $ docker exec -it pulsar-standalone /bin/bash - $ python pulsar-client.py - - ``` + ```bash + + $ docker exec -it pulsar-standalone /bin/bash + $ python pulsar-client.py + + ``` 13. Open another window to log in MySQL server. - ```bash - - $ docker exec -it pulsar-mysql /bin/bash - $ mysql -h 127.0.0.1 -uroot -pcanal - - ``` + ```bash + + $ docker exec -it pulsar-mysql /bin/bash + $ mysql -h 127.0.0.1 -uroot -pcanal + + ``` 14. Create a table, and insert, delete, and update data in MySQL server. - - - ```bash - mysql> use test; - mysql> show tables; - mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, - `test_author` VARCHAR(40) NOT NULL, - `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; - mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); - mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; - mysql> DELETE FROM test_table WHERE test_title='c'; - - ``` + ```bash + + mysql> use test; + mysql> show tables; + mysql> CREATE TABLE IF NOT EXISTS `test_table`(`test_id` INT UNSIGNED AUTO_INCREMENT,`test_title` VARCHAR(100) NOT NULL, + `test_author` VARCHAR(40) NOT NULL, + `test_date` DATE,PRIMARY KEY ( `test_id` ))ENGINE=InnoDB DEFAULT CHARSET=utf8; + mysql> INSERT INTO test_table (test_title, test_author, test_date) VALUES("a", "b", NOW()); + mysql> UPDATE test_table SET test_title='c' WHERE test_title='a'; + mysql> DELETE FROM test_table WHERE test_title='c'; + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-cassandra-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-cassandra-sink.md index 699c7f1d5ff7e..c79917ca80eec 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-cassandra-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-cassandra-sink.md @@ -1,7 +1,7 @@ --- id: io-cassandra-sink title: Cassandra sink connector -sidebar_label: Cassandra sink connector +sidebar_label: "Cassandra sink connector" original_id: io-cassandra-sink --- @@ -31,30 +31,30 @@ Before using the Cassandra sink connector, you need to create a configuration fi * JSON - ```json - - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - - ``` - -* YAML + ```json + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` - ``` - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" +* YAML - ``` + ``` + + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" + + ``` ## Usage diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-cdc-debezium.md b/site2/website-next/versioned_docs/version-2.8.0/io-cdc-debezium.md index 53770e16b676d..9b00ccfb2e390 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-cdc-debezium.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-cdc-debezium.md @@ -1,7 +1,7 @@ --- id: io-cdc-debezium title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" original_id: io-cdc-debezium --- @@ -51,65 +51,65 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "pulsar.service.url": "pulsar://127.0.0.1:6650", - "offset.storage.topic": "offset-topic" - } - - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -117,86 +117,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone - - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. - - ```bash - - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - - ``` + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - * Use the **YAML** configuration file as shown previously. - - - ```bash - - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml + * Use the **YAML** configuration file as shown previously. - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -208,49 +208,49 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "postgres", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "schema.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "postgres", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "schema.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" - parallelism: 1 - - configs: + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - ## config for pg, docker image: debezium/example-postgress:0.8 - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "postgres" - database.dbname: "postgres" - database.server.name: "dbserver1" - schema.whitelist: "inventory" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.8 + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "postgres" + database.dbname: "postgres" + database.server.name: "dbserver1" + schema.whitelist: "inventory" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -259,150 +259,150 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - - $ docker pull debezium/example-postgres:0.8 - $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 - - ``` + ```bash + + $ docker pull debezium/example-postgres:0.8 + $ docker run -d -it --rm --name pulsar-postgresql -p 5432:5432 debezium/example-postgres:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone - - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. - - ```bash - - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "postgres","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` - ``` - * Use the **YAML** configuration file as shown previously. - - - ```bash - - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - - ``` - -5. Start a PostgreSQL client in docker. + ``` + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` - ```bash - - $ docker exec -it pulsar-postgresql /bin/bash +5. Start a PostgreSQL client in docker. - ``` + ```bash + + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to change the data of the table _products_. - ``` - psql -U postgres postgres - postgres=# \c postgres; - You are now connected to database "postgres" as user "postgres". - postgres=# SET search_path TO inventory; - SET - postgres=# select * from products; - id | name | description | weight - -----+--------------------+---------------------------------------------------------+-------- - 102 | car battery | 12V car battery | 8.1 - 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 - 104 | hammer | 12oz carpenter's hammer | 0.75 - 105 | hammer | 14oz carpenter's hammer | 0.875 - 106 | hammer | 16oz carpenter's hammer | 1 - 107 | rocks | box of assorted rocks | 5.3 - 108 | jacket | water resistent black wind breaker | 0.1 - 109 | spare tire | 24 inch spare tire | 22.2 - 101 | 1111111111 | Small 2-wheel scooter | 3.14 - (9 rows) - - postgres=# UPDATE products SET name='1111111111' WHERE id=107; - UPDATE 1 - - ``` - - In the terminal window of subscribing topic, you can receive the following messages. - + ``` + + psql -U postgres postgres + postgres=# \c postgres; + You are now connected to database "postgres" as user "postgres". + postgres=# SET search_path TO inventory; + SET + postgres=# select * from products; + id | name | description | weight + -----+--------------------+---------------------------------------------------------+-------- + 102 | car battery | 12V car battery | 8.1 + 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 + 104 | hammer | 12oz carpenter's hammer | 0.75 + 105 | hammer | 14oz carpenter's hammer | 0.875 + 106 | hammer | 16oz carpenter's hammer | 1 + 107 | rocks | box of assorted rocks | 5.3 + 108 | jacket | water resistent black wind breaker | 0.1 + 109 | spare tire | 24 inch spare tire | 22.2 + 101 | 1111111111 | Small 2-wheel scooter | 3.14 + (9 rows) + + postgres=# UPDATE products SET name='1111111111' WHERE id=107; + UPDATE 1 + + ``` - ```bash + In the terminal window of subscribing topic, you can receive the following messages. - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} + ```bash + + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":107}}�{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"before"},{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":true,"field":"version"},{"type":"string","optional":true,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":false,"field":"db"},{"type":"int64","optional":true,"field":"ts_usec"},{"type":"int64","optional":true,"field":"txId"},{"type":"int64","optional":true,"field":"lsn"},{"type":"string","optional":true,"field":"schema"},{"type":"string","optional":true,"field":"table"},{"type":"boolean","optional":true,"default":false,"field":"snapshot"},{"type":"boolean","optional":true,"field":"last_snapshot_record"}],"optional":false,"name":"io.debezium.connector.postgresql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"after":{"id":107,"name":"1111111111","description":"box of assorted rocks","weight":5.3},"source":{"version":"0.9.2.Final","connector":"postgresql","name":"dbserver1","db":"postgres","ts_usec":1559208957661080,"txId":577,"lsn":23862872,"schema":"inventory","table":"products","snapshot":false,"last_snapshot_record":null},"op":"u","ts_ms":1559208957692}} + + ``` - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" - parallelism: 1 + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - configs: - - ## config for pg, docker image: debezium/example-postgress:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-postgress:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -411,98 +411,106 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + Use the following commands to initialize the data. - ``` - Use the following commands to initialize the data. + ``` bash + ./usr/local/bin/init-inventory.sh + + ``` - ``` bash - ./usr/local/bin/init-inventory.sh + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash + ``` - $ bin/pulsar standalone - - ``` + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. - ```bash + ``` - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` - ``` - * Use the **YAML** configuration file as shown previously. - - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml + ``` - ``` + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - - ``` + ``` -5. Start a MongoDB client in docker. + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` - ```bash +5. Start a MongoDB client in docker. - $ docker exec -it pulsar-mongodb /bin/bash + ``` - ``` - -6. A MongoDB client pops out. + $ docker exec -it pulsar-mongodb /bin/bash + + ``` - ```bash - - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) +6. A MongoDB client pops out. - ``` + ``` - In the terminal window of subscribing topic, you can receive the following messages. - + + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` - ```bash + In the terminal window of subscribing topic, you can receive the following messages. - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + ``` - ``` + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + + ``` + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -512,15 +520,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -534,11 +542,13 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-cdc.md b/site2/website-next/versioned_docs/version-2.8.0/io-cdc.md index f78686012d45e..9ce8f7fc39101 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-cdc.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-cdc.md @@ -1,7 +1,7 @@ --- id: io-cdc title: CDC connector -sidebar_label: CDC connector +sidebar_label: "CDC connector" original_id: io-cdc --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-cli.md b/site2/website-next/versioned_docs/version-2.8.0/io-cli.md index 96576693b768d..886affed7bfcb 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-cli.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-cli.md @@ -1,7 +1,7 @@ --- id: io-cli title: Connector Admin CLI -sidebar_label: CLI +sidebar_label: "CLI" original_id: io-cli --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-connectors.md b/site2/website-next/versioned_docs/version-2.8.0/io-connectors.md index d3a702e6795e4..945fbfe1a0539 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-connectors.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-connectors.md @@ -1,7 +1,7 @@ --- id: io-connectors title: Built-in connector -sidebar_label: Built-in connector +sidebar_label: "Built-in connector" original_id: io-connectors --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-debezium-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-debezium-source.md index ac069fe29af5a..692d1d64e9ead 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-debezium-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-debezium-source.md @@ -1,7 +1,7 @@ --- id: io-debezium-source title: Debezium source connector -sidebar_label: Debezium source connector +sidebar_label: "Debezium source connector" original_id: io-debezium-source --- @@ -72,65 +72,65 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - - { - "database.hostname": "localhost", - "database.port": "3306", - "database.user": "debezium", - "database.password": "dbz", - "database.server.id": "184054", - "database.server.name": "dbserver1", - "database.whitelist": "inventory", - "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", - "database.history.pulsar.topic": "history-topic", - "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", - "key.converter": "org.apache.kafka.connect.json.JsonConverter", - "value.converter": "org.apache.kafka.connect.json.JsonConverter", - "pulsar.service.url": "pulsar://127.0.0.1:6650", - "offset.storage.topic": "offset-topic" - } - - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "3306", + "database.user": "debezium", + "database.password": "dbz", + "database.server.id": "184054", + "database.server.name": "dbserver1", + "database.whitelist": "inventory", + "database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory", + "database.history.pulsar.topic": "history-topic", + "database.history.pulsar.service.url": "pulsar://127.0.0.1:6650", + "key.converter": "org.apache.kafka.connect.json.JsonConverter", + "value.converter": "org.apache.kafka.connect.json.JsonConverter", + "pulsar.service.url": "pulsar://127.0.0.1:6650", + "offset.storage.topic": "offset-topic" + } + + ``` * YAML - You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-mysql-source" - topicName: "debezium-mysql-topic" - archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" - parallelism: 1 - - configs: - - ## config for mysql, docker image: debezium/example-mysql:0.8 - database.hostname: "localhost" - database.port: "3306" - database.user: "debezium" - database.password: "dbz" - database.server.id: "184054" - database.server.name: "dbserver1" - database.whitelist: "inventory" - database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" - database.history.pulsar.topic: "history-topic" - database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" - - ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG - key.converter: "org.apache.kafka.connect.json.JsonConverter" - value.converter: "org.apache.kafka.connect.json.JsonConverter" - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" + You can create a `debezium-mysql-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mysql/src/main/resources/debezium-mysql-source-config.yaml) below to the `debezium-mysql-source-config.yaml` file. - ## OFFSET_STORAGE_TOPIC_CONFIG - offset.storage.topic: "offset-topic" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mysql-source" + topicName: "debezium-mysql-topic" + archive: "connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for mysql, docker image: debezium/example-mysql:0.8 + database.hostname: "localhost" + database.port: "3306" + database.user: "debezium" + database.password: "dbz" + database.server.id: "184054" + database.server.name: "dbserver1" + database.whitelist: "inventory" + database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory" + database.history.pulsar.topic: "history-topic" + database.history.pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG + key.converter: "org.apache.kafka.connect.json.JsonConverter" + value.converter: "org.apache.kafka.connect.json.JsonConverter" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ## OFFSET_STORAGE_TOPIC_CONFIG + offset.storage.topic: "offset-topic" + + ``` ### Usage @@ -138,86 +138,86 @@ This example shows how to change the data of a MySQL table using the Pulsar Debe 1. Start a MySQL server with a database from which Debezium can capture changes. - ```bash - - $ docker run -it --rm \ - --name mysql \ - -p 3306:3306 \ - -e MYSQL_ROOT_PASSWORD=debezium \ - -e MYSQL_USER=mysqluser \ - -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 - - ``` + ```bash + + $ docker run -it --rm \ + --name mysql \ + -p 3306:3306 \ + -e MYSQL_ROOT_PASSWORD=debezium \ + -e MYSQL_USER=mysqluser \ + -e MYSQL_PASSWORD=mysqlpw debezium/example-mysql:0.8 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone - - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. - * Use the **JSON** configuration file as shown previously. + * Use the **JSON** configuration file as shown previously. - Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. - - ```bash - - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ - --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' - - ``` - - * Use the **YAML** configuration file as shown previously. - - - ```bash + Make sure the nar file is available at `connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mysql-@pulsar:version@.nar \ + --name debezium-mysql-source --destination-topic-name debezium-mysql-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "3306","database.user": "debezium","database.password": "dbz","database.server.id": "184054","database.server.name": "dbserver1","database.whitelist": "inventory","database.history": "org.apache.pulsar.io.debezium.PulsarDatabaseHistory","database.history.pulsar.topic": "history-topic","database.history.pulsar.service.url": "pulsar://127.0.0.1:6650","key.converter": "org.apache.kafka.connect.json.JsonConverter","value.converter": "org.apache.kafka.connect.json.JsonConverter","pulsar.service.url": "pulsar://127.0.0.1:6650","offset.storage.topic": "offset-topic"}' + + ``` - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mysql-source-config.yaml + * Use the **YAML** configuration file as shown previously. - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mysql-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the table _inventory.products_. - ```bash - - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 - - ``` + ```bash + + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` 5. Start a MySQL client in docker. - ```bash - - $ docker run -it --rm \ - --name mysqlterm \ - --link mysql \ - --rm mysql:5.7 sh \ - -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' - - ``` + ```bash + + $ docker run -it --rm \ + --name mysqlterm \ + --link mysql \ + --rm mysql:5.7 sh \ + -c 'exec mysql -h"$MYSQL_PORT_3306_TCP_ADDR" -P"$MYSQL_PORT_3306_TCP_PORT" -uroot -p"$MYSQL_ENV_MYSQL_ROOT_PASSWORD"' + + ``` 6. A MySQL client pops out. Use the following commands to change the data of the table _products_. - ``` - mysql> use inventory; - mysql> show tables; - mysql> SELECT * FROM products; - mysql> UPDATE products SET name='1111111111' WHERE id=101; - mysql> UPDATE products SET name='1111111111' WHERE id=107; - - ``` + ``` + + mysql> use inventory; + mysql> show tables; + mysql> SELECT * FROM products; + mysql> UPDATE products SET name='1111111111' WHERE id=101; + mysql> UPDATE products SET name='1111111111' WHERE id=107; + + ``` - In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. + In the terminal window of subscribing topic, you can find the data changes have been kept in the _sub-products_ topic. ## Example of PostgreSQL @@ -229,54 +229,54 @@ You can use one of the following methods to create a configuration file. * JSON - ```json - - { - "database.hostname": "localhost", - "database.port": "5432", - "database.user": "postgres", - "database.password": "changeme", - "database.dbname": "postgres", - "database.server.name": "dbserver1", - "plugin.name": "pgoutput", - "schema.whitelist": "public", - "table.whitelist": "public.users", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - - ``` + ```json + + { + "database.hostname": "localhost", + "database.port": "5432", + "database.user": "postgres", + "database.password": "changeme", + "database.dbname": "postgres", + "database.server.name": "dbserver1", + "plugin.name": "pgoutput", + "schema.whitelist": "public", + "table.whitelist": "public.users", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-postgres-source" - topicName: "debezium-postgres-topic" - archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" - parallelism: 1 - - configs: - - ## config for postgres version 10+, official docker image: postgres:<10+> - database.hostname: "localhost" - database.port: "5432" - database.user: "postgres" - database.password: "changeme" - database.dbname: "postgres" - database.server.name: "dbserver1" - plugin.name: "pgoutput" - schema.whitelist: "public" - table.whitelist: "public.users" + You can create a `debezium-postgres-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/postgres/src/main/resources/debezium-postgres-source-config.yaml) below to the `debezium-postgres-source-config.yaml` file. - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-postgres-source" + topicName: "debezium-postgres-topic" + archive: "connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for postgres version 10+, official docker image: postgres:<10+> + database.hostname: "localhost" + database.port: "5432" + database.user: "postgres" + database.password: "changeme" + database.dbname: "postgres" + database.server.name: "dbserver1" + plugin.name: "pgoutput" + schema.whitelist: "public" + table.whitelist: "public.users" + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` - ``` - Notice that `pgoutput` is a standard plugin of Postgres introduced in version 10 - [see Postgres architecture docu](https://www.postgresql.org/docs/10/logical-replication-architecture.html). You don't need to install anything, just make sure the WAL level is set to `logical` (see docker command below and [Postgres docu](https://www.postgresql.org/docs/current/runtime-config-wal.html)). ### Usage @@ -286,158 +286,158 @@ This example shows how to change the data of a PostgreSQL table using the Pulsar 1. Start a PostgreSQL server with a database from which Debezium can capture changes. - ```bash - - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=changeme \ - postgres:13.3 -c wal_level=logical - - ``` + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=changeme \ + postgres:13.3 -c wal_level=logical + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone - - ``` + ```bash + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. - - ```bash - - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ - --name debezium-postgres-source \ - --destination-topic-name debezium-postgres-topic \ - --tenant public \ - --namespace default \ - --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + Make sure the nar file is available at `connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar`. + + ```bash + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-postgres-@pulsar:version@.nar \ + --name debezium-postgres-source \ + --destination-topic-name debezium-postgres-topic \ + --tenant public \ + --namespace default \ + --source-config '{"database.hostname": "localhost","database.port": "5432","database.user": "postgres","database.password": "changeme","database.dbname": "postgres","database.server.name": "dbserver1","schema.whitelist": "public","table.whitelist": "public.users","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` - ``` - * Use the **YAML** configuration file as shown previously. - - - ```bash - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-postgres-source-config.yaml - - ``` + ```bash + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-postgres-source-config.yaml + + ``` 4. Subscribe the topic _sub-users_ for the _public.users_ table. - ``` - $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 - - ``` - -5. Start a PostgreSQL client in docker. + ``` + $ bin/pulsar-client consume -s "sub-users" public/default/dbserver1.public.users -n 0 + + ``` - ```bash - - $ docker exec -it pulsar-postgresql /bin/bash +5. Start a PostgreSQL client in docker. - ``` + ```bash + + $ docker exec -it pulsar-postgresql /bin/bash + + ``` 6. A PostgreSQL client pops out. Use the following commands to create sample data in the table _users_. - ``` - psql -U postgres -h localhost -p 5432 - Password for user postgres: - - CREATE TABLE users( - id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), - hash_firstname TEXT NOT NULL, - hash_lastname TEXT NOT NULL, - gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) - ); - - INSERT INTO users(hash_firstname, hash_lastname, gender) - SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); - - postgres=# select * from users; - - id | hash_firstname | hash_lastname | gender - -------+----------------------------------+----------------------------------+-------- - 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female - 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male - 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female - 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female - - - postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; - UPDATE 1 - - ``` + ``` + + psql -U postgres -h localhost -p 5432 + Password for user postgres: + + CREATE TABLE users( + id BIGINT GENERATED ALWAYS AS IDENTITY, PRIMARY KEY(id), + hash_firstname TEXT NOT NULL, + hash_lastname TEXT NOT NULL, + gender VARCHAR(6) NOT NULL CHECK (gender IN ('male', 'female')) + ); + + INSERT INTO users(hash_firstname, hash_lastname, gender) + SELECT md5(RANDOM()::TEXT), md5(RANDOM()::TEXT), CASE WHEN RANDOM() < 0.5 THEN 'male' ELSE 'female' END FROM generate_series(1, 100); + + postgres=# select * from users; + + id | hash_firstname | hash_lastname | gender + -------+----------------------------------+----------------------------------+-------- + 1 | 02bf7880eb489edc624ba637f5ab42bd | 3e742c2cc4217d8e3382cc251415b2fb | female + 2 | dd07064326bb9119189032316158f064 | 9c0e938f9eddbd5200ba348965afbc61 | male + 3 | 2c5316fdd9d6595c1cceb70eed12e80c | 8a93d7d8f9d76acfaaa625c82a03ea8b | female + 4 | 3dfa3b4f70d8cd2155567210e5043d2b | 32c156bc28f7f03ab5d28e2588a3dc19 | female - In the terminal window of subscribing topic, you can receive the following messages. - + + postgres=# UPDATE users SET hash_firstname='maxim' WHERE id=1; + UPDATE 1 + + ``` - ```bash + In the terminal window of subscribing topic, you can receive the following messages. - ----- got message ----- - {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} - ...many more + ```bash + + ----- got message ----- + {"before":null,"after":{"id":1,"hash_firstname":"maxim","hash_lastname":"292113d30a3ccee0e19733dd7f88b258","gender":"male"},"source:{"version":"1.0.0.Final","connector":"postgresql","name":"foobar","ts_ms":1624045862644,"snapshot":"false","db":"postgres","schema":"public","table":"users","txId":595,"lsn":24419784,"xmin":null},"op":"u","ts_ms":1624045862648} + ...many more + + ``` - ``` ## Example of MongoDB You need to create a configuration file before using the Pulsar Debezium connector. * JSON - ```json - - { - "mongodb.hosts": "rs0/mongodb:27017", - "mongodb.name": "dbserver1", - "mongodb.user": "debezium", - "mongodb.password": "dbz", - "mongodb.task.id": "1", - "database.whitelist": "inventory", - "pulsar.service.url": "pulsar://127.0.0.1:6650" - } - - ``` + ```json + + { + "mongodb.hosts": "rs0/mongodb:27017", + "mongodb.name": "dbserver1", + "mongodb.user": "debezium", + "mongodb.password": "dbz", + "mongodb.task.id": "1", + "database.whitelist": "inventory", + "pulsar.service.url": "pulsar://127.0.0.1:6650" + } + + ``` * YAML - You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - - ```yaml - - tenant: "public" - namespace: "default" - name: "debezium-mongodb-source" - topicName: "debezium-mongodb-topic" - archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" - parallelism: 1 + You can create a `debezium-mongodb-source-config.yaml` file and copy the [contents](https://github.com/apache/pulsar/blob/master/pulsar-io/debezium/mongodb/src/main/resources/debezium-mongodb-source-config.yaml) below to the `debezium-mongodb-source-config.yaml` file. - configs: - - ## config for pg, docker image: debezium/example-mongodb:0.10 - mongodb.hosts: "rs0/mongodb:27017", - mongodb.name: "dbserver1", - mongodb.user: "debezium", - mongodb.password: "dbz", - mongodb.task.id: "1", - database.whitelist: "inventory", - - ## PULSAR_SERVICE_URL_CONFIG - pulsar.service.url: "pulsar://127.0.0.1:6650" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "debezium-mongodb-source" + topicName: "debezium-mongodb-topic" + archive: "connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar" + parallelism: 1 + + configs: + + ## config for pg, docker image: debezium/example-mongodb:0.10 + mongodb.hosts: "rs0/mongodb:27017", + mongodb.name: "dbserver1", + mongodb.user: "debezium", + mongodb.password: "dbz", + mongodb.task.id: "1", + database.whitelist: "inventory", + + ## PULSAR_SERVICE_URL_CONFIG + pulsar.service.url: "pulsar://127.0.0.1:6650" + + ``` ### Usage @@ -446,98 +446,106 @@ This example shows how to change the data of a MongoDB table using the Pulsar De 1. Start a MongoDB server with a database from which Debezium can capture changes. - ```bash + ```bash + + $ docker pull debezium/example-mongodb:0.10 + $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + + ``` - $ docker pull debezium/example-mongodb:0.10 - $ docker run -d -it --rm --name pulsar-mongodb -e MONGODB_USER=mongodb -e MONGODB_PASSWORD=mongodb -p 27017:27017 debezium/example-mongodb:0.10 + Use the following commands to initialize the data. - ``` - Use the following commands to initialize the data. + ``` bash + ./usr/local/bin/init-inventory.sh + + ``` - ``` bash - ./usr/local/bin/init-inventory.sh + If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` - ``` - If the local host cannot access the container network, you can update the file ```/etc/hosts``` and add a rule ```127.0.0.1 6 f114527a95f```. f114527a95f is container id, you can try to get by ```docker ps -a``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone + ``` - ``` + + $ bin/pulsar standalone + + ``` 3. Start the Pulsar Debezium connector in local run mode using one of the following methods. * Use the **JSON** configuration file as shown previously. - - Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. + + Make sure the nar file is available at `connectors/pulsar-io-mongodb-@pulsar:version@.nar`. - ```bash + ``` - $ bin/pulsar-admin source localrun \ - --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ - --name debezium-mongodb-source \ - --destination-topic-name debezium-mongodb-topic \ - --tenant public \ - --namespace default \ - --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + $ bin/pulsar-admin source localrun \ + --archive connectors/pulsar-io-debezium-mongodb-@pulsar:version@.nar \ + --name debezium-mongodb-source \ + --destination-topic-name debezium-mongodb-topic \ + --tenant public \ + --namespace default \ + --source-config '{"mongodb.hosts": "rs0/mongodb:27017","mongodb.name": "dbserver1","mongodb.user": "debezium","mongodb.password": "dbz","mongodb.task.id": "1","database.whitelist": "inventory","pulsar.service.url": "pulsar://127.0.0.1:6650"}' + + ``` - ``` - * Use the **YAML** configuration file as shown previously. - - ```bash + ``` - $ bin/pulsar-admin source localrun \ - --source-config-file debezium-mongodb-source-config.yaml - - ``` + + $ bin/pulsar-admin source localrun \ + --source-config-file debezium-mongodb-source-config.yaml + + ``` 4. Subscribe the topic _sub-products_ for the _inventory.products_ table. - ``` - $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + ``` - ``` - -5. Start a MongoDB client in docker. + $ bin/pulsar-client consume -s "sub-products" public/default/dbserver1.inventory.products -n 0 + + ``` - ```bash - - $ docker exec -it pulsar-mongodb /bin/bash +5. Start a MongoDB client in docker. - ``` + ``` -6. A MongoDB client pops out. + $ docker exec -it pulsar-mongodb /bin/bash + + ``` - ```bash - - mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory - db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) +6. A MongoDB client pops out. - ``` + ``` - In the terminal window of subscribing topic, you can receive the following messages. - + + mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory + db.products.update({"_id":NumberLong(104)},{$set:{weight:1.25}}) + + ``` - ```bash + In the terminal window of subscribing topic, you can receive the following messages. - ----- got message ----- - {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + ``` - ``` + ----- got message ----- + {"schema":{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"}],"optional":false,"name":"dbserver1.inventory.products.Key"},"payload":{"id":"104"}}, value = {"schema":{"type":"struct","fields":[{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"after"},{"type":"string","optional":true,"name":"io.debezium.data.Json","version":1,"field":"patch"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":false,"field":"rs"},{"type":"string","optional":false,"field":"collection"},{"type":"int32","optional":false,"field":"ord"},{"type":"int64","optional":true,"field":"h"}],"optional":false,"name":"io.debezium.connector.mongo.Source","field":"source"},{"type":"string","optional":true,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"},"payload":{"after":"{\"_id\": {\"$numberLong\": \"104\"},\"name\": \"hammer\",\"description\": \"12oz carpenter's hammer\",\"weight\": 1.25,\"quantity\": 4}","patch":null,"source":{"version":"0.10.0.Final","connector":"mongodb","name":"dbserver1","ts_ms":1573541905000,"snapshot":"true","db":"inventory","rs":"rs0","collection":"products","ord":1,"h":4983083486544392763},"op":"r","ts_ms":1573541909761}}. + + ``` + ## FAQ ### Debezium postgres connector will hang when create snap -```$xslt +``` + #18 prio=5 os_prio=31 tid=0x00007fd83096f800 nid=0xa403 waiting on condition [0x000070000f534000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) @@ -547,15 +555,15 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at java.util.concurrent.LinkedBlockingDeque.putLast(LinkedBlockingDeque.java:396) at java.util.concurrent.LinkedBlockingDeque.put(LinkedBlockingDeque.java:649) at io.debezium.connector.base.ChangeEventQueue.enqueue(ChangeEventQueue.java:132) - at io.debezium.connector.postgresql.PostgresConnectorTask$$Lambda$203/385424085.accept(Unknown Source) + at io.debezium.connector.postgresql.PostgresConnectorTask$Lambda$203/385424085.accept(Unknown Source) at io.debezium.connector.postgresql.RecordsSnapshotProducer.sendCurrentRecord(RecordsSnapshotProducer.java:402) at io.debezium.connector.postgresql.RecordsSnapshotProducer.readTable(RecordsSnapshotProducer.java:321) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$takeSnapshot$6(RecordsSnapshotProducer.java:226) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$240/1347039967.accept(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$240/1347039967.accept(Unknown Source) at io.debezium.jdbc.JdbcConnection.queryWithBlockingConsumer(JdbcConnection.java:535) at io.debezium.connector.postgresql.RecordsSnapshotProducer.takeSnapshot(RecordsSnapshotProducer.java:224) at io.debezium.connector.postgresql.RecordsSnapshotProducer.lambda$start$0(RecordsSnapshotProducer.java:87) - at io.debezium.connector.postgresql.RecordsSnapshotProducer$$Lambda$206/589332928.run(Unknown Source) + at io.debezium.connector.postgresql.RecordsSnapshotProducer$Lambda$206/589332928.run(Unknown Source) at java.util.concurrent.CompletableFuture.uniRun(CompletableFuture.java:705) at java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:717) at java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2010) @@ -569,11 +577,13 @@ This example shows how to change the data of a MongoDB table using the Pulsar De at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:230) at java.lang.Thread.run(Thread.java:748) -``` +``` If you encounter the above problems in synchronizing data, please refer to [this](https://github.com/apache/pulsar/issues/4075) and add the following configuration to the configuration file: -```$xslt +``` + max.queue.size= ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-debug.md b/site2/website-next/versioned_docs/version-2.8.0/io-debug.md index fada2555acf71..f815e862cae42 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-debug.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-debug.md @@ -1,7 +1,7 @@ --- id: io-debug title: How to debug Pulsar connectors -sidebar_label: Debug +sidebar_label: "Debug" original_id: io-debug --- @@ -14,61 +14,66 @@ To better demonstrate how to debug Pulsar connectors, here takes a Mongo sink co **Deploy a Mongo sink environment** 1. Start a Mongo service. - ```bash - - docker pull mongo:4 - docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 + ```bash + + docker pull mongo:4 + docker run -d -p 27017:27017 --name pulsar-mongo -v $PWD/data:/data/db mongo:4 + + ``` - ``` 2. Create a DB and a collection. - ```bash - - docker exec -it pulsar-mongo /bin/bash - mongo - > use pulsar - > db.createCollection('messages') - > exit + ```bash + + docker exec -it pulsar-mongo /bin/bash + mongo + > use pulsar + > db.createCollection('messages') + > exit + + ``` - ``` 3. Start Pulsar standalone. - ```bash - - docker pull apachepulsar/pulsar:2.4.0 - docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + ```bash + + docker pull apachepulsar/pulsar:2.4.0 + docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --link pulsar-mongo --name pulsar-mongo-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone + + ``` - ``` 4. Configure the Mongo sink with the `mongo-sink-config.yaml` file. - ```bash - - configs: - mongoUri: "mongodb://pulsar-mongo:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - - ``` - ```bash + ```bash + + configs: + mongoUri: "mongodb://pulsar-mongo:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + + ``` - docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ + ```bash + + docker cp mongo-sink-config.yaml pulsar-mongo-standalone:/pulsar/ + + ``` - ``` 5. Download the Mongo sink nar package. - ```bash - - docker exec -it pulsar-mongo-standalone /bin/bash - curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar + ```bash + + docker exec -it pulsar-mongo-standalone /bin/bash + curl -O http://apache.01link.hk/pulsar/pulsar-2.4.0/connectors/pulsar-io-mongo-2.4.0.nar + + ``` - ``` ## Debug in localrun mode Start the Mongo sink in localrun mode using the `localrun` command. :::tip - For more information about the `localrun` command, see [`localrun`](reference-connector-admin.md/#localrun-1). ::: @@ -84,138 +89,144 @@ For more information about the `localrun` command, see [`localrun`](reference-co --parallelism 1 ``` + ### Use connector log Use one of the following methods to get a connector log in localrun mode: * After executing the `localrun` command, the **log is automatically printed on the console**. * The log is located at: - - - ```bash - logs/functions/tenant/namespace/function-name/function-name-instance-id.log - - ``` - - **Example** - - The path of the Mongo sink connector is: + ```bash + + logs/functions/tenant/namespace/function-name/function-name-instance-id.log + + ``` - ```bash + **Example** + + The path of the Mongo sink connector is: - logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log + ```bash + + logs/functions/public/default/pulsar-mongo-sink/pulsar-mongo-sink-0.log + + ``` - ``` To clearly explain the log information, here breaks down the large block of information into small blocks and add descriptions for each block. * This piece of log information shows the storage path of the nar package after decompression. - ``` - 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, + ``` + + 08:21:54.132 [main] INFO org.apache.pulsar.common.nar.NarClassLoader - Created class loader with paths: [file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/, file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/, + + ``` + + :::tip - ``` - -:::tip + If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. -If `class cannot be found` exception is thrown, check whether the nar file is decompressed in the folder `file:/tmp/pulsar-nar/pulsar-io-mongo-2.4.0.nar-unpacked/META-INF/bundled-dependencies/` or not. + ::: -::: * This piece of log information illustrates the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, resources, and so on, which can be used to **check whether the Mongo sink connector is configured correctly or not**. - ```bash - - 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" - namespace: "default" - name: "pulsar-mongo-sink" - className: "org.apache.pulsar.functions.api.utils.IdentityFunction" - autoAck: true - parallelism: 1 - source { - typeClassName: "[B" - inputSpecs { - key: "test-mongo" - value { - } - } - cleanupSubscription: true - } - sink { - className: "org.apache.pulsar.io.mongodb.MongoSink" - configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" - typeClassName: "[B" - } - resources { - cpu: 1.0 - ram: 1073741824 - disk: 10737418240 - } - componentType: SINK - , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) + ```bash + + 08:21:55.390 [main] INFO org.apache.pulsar.functions.runtime.ThreadRuntime - ThreadContainer starting function with instance config InstanceConfig(instanceId=0, functionId=853d60a1-0c48-44d5-9a5c-6917386476b2, functionVersion=c2ce1458-b69e-4175-88c0-a0a856a2be8c, functionDetails=tenant: "public" + namespace: "default" + name: "pulsar-mongo-sink" + className: "org.apache.pulsar.functions.api.utils.IdentityFunction" + autoAck: true + parallelism: 1 + source { + typeClassName: "[B" + inputSpecs { + key: "test-mongo" + value { + } + } + cleanupSubscription: true + } + sink { + className: "org.apache.pulsar.io.mongodb.MongoSink" + configs: "{\"mongoUri\":\"mongodb://pulsar-mongo:27017\",\"database\":\"pulsar\",\"collection\":\"messages\",\"batchSize\":2,\"batchTimeMs\":500}" + typeClassName: "[B" + } + resources { + cpu: 1.0 + ram: 1073741824 + disk: 10737418240 + } + componentType: SINK + , maxBufferedTuples=1024, functionAuthenticationSpec=null, port=38459, clusterName=local) + + ``` - ``` * This piece of log information demonstrates the status of the connections to Mongo and configuration information. - ```bash - - 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 - 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} + ```bash + + 08:21:56.231 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.connection - Opened connection [connectionId{localValue:1, serverValue:8}] to pulsar-mongo:27017 + 08:21:56.326 [cluster-ClusterId{value='5d6396a3c9e77c0569ff00eb', description='null'}-pulsar-mongo:27017] INFO org.mongodb.driver.cluster - Monitor thread successfully connected to server with description ServerDescription{address=pulsar-mongo:27017, type=STANDALONE, state=CONNECTED, ok=true, version=ServerVersion{versionList=[4, 2, 0]}, minWireVersion=0, maxWireVersion=8, maxDocumentSize=16777216, logicalSessionTimeoutMinutes=30, roundTripTimeNanos=89058800} + + ``` - ``` * This piece of log information explains the configuration of consumers and clients, including the topic name, subscription name, subscription type, and so on. - ```bash - - 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { - "topicNames" : [ "test-mongo" ], - "topicsPattern" : null, - "subscriptionName" : "public/default/pulsar-mongo-sink", - "subscriptionType" : "Shared", - "receiverQueueSize" : 1000, - "acknowledgementsGroupTimeMicros" : 100000, - "negativeAckRedeliveryDelayMicros" : 60000000, - "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, - "consumerName" : null, - "ackTimeoutMillis" : 0, - "tickDurationMillis" : 1000, - "priorityLevel" : 0, - "cryptoFailureAction" : "CONSUME", - "properties" : { - "application" : "pulsar-sink", - "id" : "public/default/pulsar-mongo-sink", - "instance_id" : "0" - }, - "readCompacted" : false, - "subscriptionInitialPosition" : "Latest", - "patternAutoDiscoveryPeriod" : 1, - "regexSubscriptionMode" : "PersistentOnly", - "deadLetterPolicy" : null, - "autoUpdatePartitions" : true, - "replicateSubscriptionState" : false, - "resetIncludeHead" : false - } - 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { - "serviceUrl" : "pulsar://localhost:6650", - "authPluginClassName" : null, - "authParams" : null, - "operationTimeoutMs" : 30000, - "statsIntervalSeconds" : 60, - "numIoThreads" : 1, - "numListenerThreads" : 1, - "connectionsPerBroker" : 1, - "useTcpNoDelay" : true, - "useTls" : false, - "tlsTrustCertsFilePath" : null, - "tlsAllowInsecureConnection" : false, - "tlsHostnameVerificationEnable" : false, - "concurrentLookupRequest" : 5000, - "maxLookupRequest" : 50000, - "maxNumberOfRejectedRequestPerConnection" : 50, - "keepAliveIntervalSeconds" : 30, - "connectionTimeoutMs" : 10000, - "requestTimeoutMs" : 60000, - "defaultBackoffIntervalNanos" : 100000000, - "maxBackoffIntervalNanos" : 30000000000 - } + ```bash + + 08:21:56.719 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Starting Pulsar consumer status recorder with config: { + "topicNames" : [ "test-mongo" ], + "topicsPattern" : null, + "subscriptionName" : "public/default/pulsar-mongo-sink", + "subscriptionType" : "Shared", + "receiverQueueSize" : 1000, + "acknowledgementsGroupTimeMicros" : 100000, + "negativeAckRedeliveryDelayMicros" : 60000000, + "maxTotalReceiverQueueSizeAcrossPartitions" : 50000, + "consumerName" : null, + "ackTimeoutMillis" : 0, + "tickDurationMillis" : 1000, + "priorityLevel" : 0, + "cryptoFailureAction" : "CONSUME", + "properties" : { + "application" : "pulsar-sink", + "id" : "public/default/pulsar-mongo-sink", + "instance_id" : "0" + }, + "readCompacted" : false, + "subscriptionInitialPosition" : "Latest", + "patternAutoDiscoveryPeriod" : 1, + "regexSubscriptionMode" : "PersistentOnly", + "deadLetterPolicy" : null, + "autoUpdatePartitions" : true, + "replicateSubscriptionState" : false, + "resetIncludeHead" : false + } + 08:21:56.726 [pulsar-client-io-1-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - Pulsar client config: { + "serviceUrl" : "pulsar://localhost:6650", + "authPluginClassName" : null, + "authParams" : null, + "operationTimeoutMs" : 30000, + "statsIntervalSeconds" : 60, + "numIoThreads" : 1, + "numListenerThreads" : 1, + "connectionsPerBroker" : 1, + "useTcpNoDelay" : true, + "useTls" : false, + "tlsTrustCertsFilePath" : null, + "tlsAllowInsecureConnection" : false, + "tlsHostnameVerificationEnable" : false, + "concurrentLookupRequest" : 5000, + "maxLookupRequest" : 50000, + "maxNumberOfRejectedRequestPerConnection" : 50, + "keepAliveIntervalSeconds" : 30, + "connectionTimeoutMs" : 10000, + "requestTimeoutMs" : 60000, + "defaultBackoffIntervalNanos" : 100000000, + "maxBackoffIntervalNanos" : 30000000000 + } + + ``` - ``` ## Debug in cluster mode You can use the following methods to debug a connector in cluster mode: * [Use connector log](#use-connector-log) @@ -243,6 +254,7 @@ Pulsar admin CLI helps you debug Pulsar connectors with the following subcommand --parallelism 1 ``` + ### `get` Use the `get` command to get the basic information about the Mongo sink connector, such as tenant, namespace, name, parallelism, and so on. @@ -276,10 +288,10 @@ Use the `get` command to get the basic information about the Mongo sink connecto :::tip - For more information about the `get` command, see [`get`](reference-connector-admin.md/#get-1). ::: + ### `status` Use the `status` command to get the current status about the Mongo sink connector, such as the number of instance, the number of running instance, instanceId, workerId and so on. @@ -311,14 +323,14 @@ Use the `status` command to get the current status about the Mongo sink connecto } ``` -:::tip +:::tip For more information about the `status` command, see [`status`](reference-connector-admin.md/#stauts-1). - If there are multiple connectors running on a worker, `workerId` can locate the worker on which the specified connector is running. ::: + ### `topics stats` Use the `topics stats` command to get the stats for a topic and its connected producer and consumer, such as whether the topic has received messages or not, whether there is a backlog of messages or not, the available permits and other key information. All rates are computed over a 1-minute window and are relative to the last completed 1-minute period. @@ -369,12 +381,13 @@ Use the `topics stats` command to get the stats for a topic and its connected pr } ``` -:::tip +:::tip For more information about the `topic stats` command, see [`topic stats`](http://pulsar.apache.org/docs/en/pulsar-admin/#stats-1). ::: + ## Checklist This checklist indicates the major areas to check when you debug connectors. It is a reminder of what to look for to ensure a thorough review and an evaluation tool to get the status of connectors. * Does Pulsar start successfully? diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-develop.md b/site2/website-next/versioned_docs/version-2.8.0/io-develop.md index 8bb7667d22501..1512fcb5613a5 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-develop.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-develop.md @@ -1,7 +1,7 @@ --- id: io-develop title: How to develop Pulsar connectors -sidebar_label: Develop +sidebar_label: "Develop" original_id: io-develop --- @@ -33,66 +33,66 @@ interface, which means you need to implement the {@inject: github:open:/pulsar-i 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sourceContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SourceContext sourceContext) throws Exception; + + ``` - /** - * Open connector with configuration - * - * @param config initialization config - * @param sourceContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SourceContext sourceContext) throws Exception; + This method is called when the source connector is initialized. - ``` + In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. + + For example, a Kafka connector can create a Kafka client in this `open` method. - This method is called when the source connector is initialized. - - In this method, you can retrieve all connector specific settings through the passed-in `config` parameter and initialize all necessary resources. - - For example, a Kafka connector can create a Kafka client in this `open` method. - - Besides, Pulsar runtime also provides a `SourceContext` for the - connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. + Besides, Pulsar runtime also provides a `SourceContext` for the + connector to access runtime resources for tasks like collecting metrics. The implementation can save the `SourceContext` for future use. 2. Implement the {@inject: github:read:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Source.java} method. - ```java - - /** - * Reads the next message from source. - * If source does not have any new messages, this call should block. - * @return next message from source. The return result should never be null - * @throws Exception - */ - Record read() throws Exception; - - ``` - - If nothing to return, the implementation should be blocking rather than returning `null`. - - The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: - - |Variable|Required|Description - |---|---|--- - `TopicName`|No|Pulsar topic name from which the record is originated from. - `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| - `Value`|Yes|Actual data of the record. - `EventTime`|No|Event time of the record from the source. - `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. - `Properties` |No| If the record carries user-defined properties, it returns those properties. - `DestinationTopic`|No|Topic to which message should be written. - `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| - - * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: - - Method|Description - |---|--- - `ack` |Acknowledge that the record is fully processed. - `fail`|Indicate that the record fails to be processed. + ```java + + /** + * Reads the next message from source. + * If source does not have any new messages, this call should block. + * @return next message from source. The return result should never be null + * @throws Exception + */ + Record read() throws Exception; + + ``` + + If nothing to return, the implementation should be blocking rather than returning `null`. + + The returned {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should encapsulate the following information, which is needed by Pulsar IO runtime. + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following variables: + + |Variable|Required|Description + |---|---|--- + `TopicName`|No|Pulsar topic name from which the record is originated from. + `Key`|No| Messages can optionally be tagged with keys.

    For more information, see [Routing modes](concepts-messaging.md#routing-modes).| + `Value`|Yes|Actual data of the record. + `EventTime`|No|Event time of the record from the source. + `PartitionId`|No| If the record is originated from a partitioned source, it returns its `PartitionId`.

    `PartitionId` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `RecordSequence`|No|If the record is originated from a sequential source, it returns its `RecordSequence`.

    `RecordSequence` is used as a part of the unique identifier by Pulsar IO runtime to deduplicate messages and achieve exactly-once processing guarantee. + `Properties` |No| If the record carries user-defined properties, it returns those properties. + `DestinationTopic`|No|Topic to which message should be written. + `Message`|No|A class which carries data sent by users.

    For more information, see [Message.java](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Message.java).| + + * {@inject: github:Record:/pulsar-functions/api-java/src/main/java/org/apache/pulsar/functions/api/Record.java} should provide the following methods: + + Method|Description + |---|--- + `ack` |Acknowledge that the record is fully processed. + `fail`|Indicate that the record fails to be processed. ## Handle schema information @@ -100,14 +100,17 @@ Pulsar IO automatically handles the schema and provides a strongly typed API bas If you know the schema type that you are producing, you can declare the Java class relative to that type in your sink declaration. ``` + public class MySource implements Source { public Record read() {} } ``` + If you want to implement a source that works with any schema, you can go with `byte[]` (of `ByteBuffer`) and use Schema.AUTO_PRODUCE_BYTES(). ``` + public class MySource implements Source { public Record read() { @@ -138,7 +141,6 @@ When Pulsar IO runtime encounters a `KVRecord`, it brings the following changes :::tip - For more information about **how to create a source connector**, see {@inject: github:KafkaSource:/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java}. ::: @@ -149,37 +151,37 @@ Developing a sink connector **is similar to** developing a source connector, tha 1. Implement the {@inject: github:open:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - - /** - * Open connector with configuration - * - * @param config initialization config - * @param sinkContext - * @throws Exception IO type exceptions when opening a connector - */ - void open(final Map config, SinkContext sinkContext) throws Exception; - - ``` + ```java + + /** + * Open connector with configuration + * + * @param config initialization config + * @param sinkContext + * @throws Exception IO type exceptions when opening a connector + */ + void open(final Map config, SinkContext sinkContext) throws Exception; + + ``` 2. Implement the {@inject: github:write:/pulsar-io/core/src/main/java/org/apache/pulsar/io/core/Sink.java} method. - ```java - - /** - * Write a message to Sink - * @param record record to write to sink - * @throws Exception - */ - void write(Record record) throws Exception; - - ``` + ```java + + /** + * Write a message to Sink + * @param record record to write to sink + * @throws Exception + */ + void write(Record record) throws Exception; + + ``` - During the implementation, you can decide how to write the `Value` and - the `Key` to the actual source, and leverage all the provided information such as - `PartitionId` and `RecordSequence` to achieve different processing guarantees. + During the implementation, you can decide how to write the `Value` and + the `Key` to the actual source, and leverage all the provided information such as + `PartitionId` and `RecordSequence` to achieve different processing guarantees. - You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). + You also need to ack records (if messages are sent successfully) or fail records (if messages fail to send). ## Handling Schema information @@ -187,6 +189,7 @@ Pulsar IO handles automatically the Schema and provides a strongly typed API bas If you know the Schema type that you are consuming from you can declare the Java class relative to that type in your Sink declaration. ``` + public class MySink implements Sink { public void write(Record record) {} } @@ -196,6 +199,7 @@ public class MySink implements Sink { If you want to implement a sink that works with any schema, you can you go with the special GenericObject interface. ``` + public class MySink implements Sink { public void write(Record record) { Schema schema = record.getSchema(); @@ -218,6 +222,7 @@ You are able to access the native AVRO record using `genericObject.getNativeObj In the case of KeyValue type, you can access both the schema for the key and the schema for the value using this code. ``` + public class MySink implements Sink { public void write(Record record) { Schema schema = record.getSchema(); @@ -261,7 +266,6 @@ Pulsar uses [testcontainers](https://www.testcontainers.org/) **for all integrat :::tip - For more information about **how to create integration tests for Pulsar connectors**, see {@inject: github:IntegrationTests:/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io}. ::: @@ -276,10 +280,10 @@ work with Pulsar Functions' runtime, that is, [NAR](#nar) and [uber JAR](#uber-j :::note - If you plan to package and distribute your connector for others to use, you are obligated to ::: + license and copyright your own code properly. Remember to add the license and copyright to all libraries your code uses and to your distribution. > @@ -294,7 +298,6 @@ a bit of Java ClassLoader isolation. :::tip - For more information about **how NAR works**, see [here](https://medium.com/hashmapinc/nifi-nar-files-explained-14113f7796fd). ::: @@ -332,7 +335,6 @@ For Gradle users, there is a [Gradle Nar plugin available on the Gradle Plugin P :::tip - For more information about an **how to use NAR for Pulsar connectors**, see {@inject: github:TwitterFirehose:/pulsar-io/twitter/pom.xml}. ::: @@ -394,6 +396,7 @@ Here is an example of how to customize metrics for a Java connector. ``` + public class TestMetricSink implements Sink { @Override diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-dynamodb-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-dynamodb-source.md index cc071c8d44a58..4a93683eb4a5b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-dynamodb-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-dynamodb-source.md @@ -1,7 +1,7 @@ --- id: io-dynamodb-source title: AWS DynamoDB source connector -sidebar_label: AWS DynamoDB source connector +sidebar_label: "AWS DynamoDB source connector" original_id: io-dynamodb-source --- @@ -45,40 +45,40 @@ Before using the DynamoDB source connector, you need to create a configuration f * JSON - ```json - - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsDynamodbStreamArn": "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsDynamodbStreamArn: "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-elasticsearch-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-elasticsearch-sink.md index fe055fab789a7..a2624bdca129d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-elasticsearch-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-elasticsearch-sink.md @@ -1,7 +1,7 @@ --- id: io-elasticsearch-sink title: ElasticSearch sink connector -sidebar_label: ElasticSearch sink connector +sidebar_label: "ElasticSearch sink connector" original_id: io-elasticsearch-sink --- @@ -37,136 +37,141 @@ Before using the ElasticSearch sink connector, you need to create a configuratio * JSON - ```json - - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "username": "scooby", - "password": "doobie" - } - - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - username: "scooby" - password: "doobie" - - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + username: "scooby" + password: "doobie" + + ``` #### For Elasticsearch Before 6.2 * JSON - ```json - - { - "elasticSearchUrl": "http://localhost:9200", - "indexName": "my_index", - "typeName": "doc", - "username": "scooby", - "password": "doobie" - } - - ``` + ```json + + { + "elasticSearchUrl": "http://localhost:9200", + "indexName": "my_index", + "typeName": "doc", + "username": "scooby", + "password": "doobie" + } + + ``` * YAML - ```yaml - - configs: - elasticSearchUrl: "http://localhost:9200" - indexName: "my_index" - typeName: "doc" - username: "scooby" - password: "doobie" - - ``` + ```yaml + + configs: + elasticSearchUrl: "http://localhost:9200" + indexName: "my_index" + typeName: "doc" + username: "scooby" + password: "doobie" + + ``` ### Usage 1. Start a single node Elasticsearch cluster. - ```bash - - $ docker run -p 9200:9200 -p 9300:9300 \ - -e "discovery.type=single-node" \ - docker.elastic.co/elasticsearch/elasticsearch:7.5.1 - - ``` + ```bash + + $ docker run -p 9200:9200 -p 9300:9300 \ + -e "discovery.type=single-node" \ + docker.elastic.co/elasticsearch/elasticsearch:7.5.1 + + ``` 2. Start a Pulsar service locally in standalone mode. - ```bash - - $ bin/pulsar standalone + ```bash + + $ bin/pulsar standalone + + ``` - ``` - Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-@pulsar:version@.nar`. + Make sure the NAR file is available at `connectors/pulsar-io-elastic-search-@pulsar:version@.nar`. 3. Start the Pulsar Elasticsearch connector in local run mode using one of the following methods. - * Use the **JSON** configuration as shown previously. - - ```bash - - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ - --inputs elasticsearch_test - - ``` - * Use the **YAML** configuration file as shown previously. - - - ```bash - - $ bin/pulsar-admin sinks localrun \ - --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ - --tenant public \ - --namespace default \ - --name elasticsearch-test-sink \ - --sink-config-file elasticsearch-sink.yml \ - --inputs elasticsearch_test - - ``` + * Use the **JSON** configuration as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config '{"elasticSearchUrl":"http://localhost:9200","indexName": "my_index","username": "scooby","password": "doobie"}' \ + --inputs elasticsearch_test + + ``` + + * Use the **YAML** configuration file as shown previously. + + ```bash + + $ bin/pulsar-admin sinks localrun \ + --archive connectors/pulsar-io-elastic-search-@pulsar:version@.nar \ + --tenant public \ + --namespace default \ + --name elasticsearch-test-sink \ + --sink-config-file elasticsearch-sink.yml \ + --inputs elasticsearch_test + + ``` 4. Publish records to the topic. - ```bash - - $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" - - ``` + ```bash + + $ bin/pulsar-client produce elasticsearch_test --messages "{\"a\":1}" + + ``` 5. Check documents in Elasticsearch. - - * refresh the index - - ```bash - - $ curl -s http://localhost:9200/my_index/_refresh - - ``` - * search documents + + * refresh the index - ```bash + ```bash + + $ curl -s http://localhost:9200/my_index/_refresh + + ``` - $ curl -s http://localhost:9200/my_index/_search + + * search documents - ``` - You can see the record that published earlier has been successfully written into Elasticsearch. + ```bash + + $ curl -s http://localhost:9200/my_index/_search + + ``` - ```json + You can see the record that published earlier has been successfully written into Elasticsearch. - {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} + ```json + + {"took":2,"timed_out":false,"_shards":{"total":1,"successful":1,"skipped":0,"failed":0},"hits":{"total":{"value":1,"relation":"eq"},"max_score":1.0,"hits":[{"_index":"my_index","_type":"_doc","_id":"FSxemm8BLjG_iC0EeTYJ","_score":1.0,"_source":{"a":1}}]}} + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-file-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-file-source.md index e4b427c33615b..66ae7bb31e575 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-file-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-file-source.md @@ -1,7 +1,7 @@ --- id: io-file-source title: File source connector -sidebar_label: File source connector +sidebar_label: "File source connector" original_id: io-file-source --- @@ -38,44 +38,44 @@ Before using the File source connector, you need to create a configuration file * JSON - ```json - - { - "inputDirectory": "/Users/david", - "recurse": true, - "keepFile": true, - "fileFilter": "[^\\.].*", - "pathFilter": "*", - "minimumFileAge": 0, - "maximumFileAge": 9999999999, - "minimumSize": 1, - "maximumSize": 5000000, - "ignoreHiddenFiles": true, - "pollingInterval": 5000, - "numWorkers": 1 - } - - ``` + ```json + + { + "inputDirectory": "/Users/david", + "recurse": true, + "keepFile": true, + "fileFilter": "[^\\.].*", + "pathFilter": "*", + "minimumFileAge": 0, + "maximumFileAge": 9999999999, + "minimumSize": 1, + "maximumSize": 5000000, + "ignoreHiddenFiles": true, + "pollingInterval": 5000, + "numWorkers": 1 + } + + ``` * YAML - ```yaml - - configs: - inputDirectory: "/Users/david" - recurse: true - keepFile: true - fileFilter: "[^\\.].*" - pathFilter: "*" - minimumFileAge: 0 - maximumFileAge: 9999999999 - minimumSize: 1 - maximumSize: 5000000 - ignoreHiddenFiles: true - pollingInterval: 5000 - numWorkers: 1 - - ``` + ```yaml + + configs: + inputDirectory: "/Users/david" + recurse: true + keepFile: true + fileFilter: "[^\\.].*" + pathFilter: "*" + minimumFileAge: 0 + maximumFileAge: 9999999999 + minimumSize: 1 + maximumSize: 5000000 + ignoreHiddenFiles: true + pollingInterval: 5000 + numWorkers: 1 + + ``` ## Usage @@ -83,84 +83,82 @@ Here is an example of using the File source connecter. 1. Pull a Pulsar image. - ```bash - - $ docker pull apachepulsar/pulsar:{version} - - ``` - -2. Start Pulsar standalone. + ```bash + $ docker pull apachepulsar/pulsar:{version} + + ``` - ```bash - - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone +2. Start Pulsar standalone. - ``` + ```bash + + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` 3. Create a configuration file _file-connector.yaml_. - ```yaml - - configs: - inputDirectory: "/opt" - - ``` + ```yaml + + configs: + inputDirectory: "/opt" + + ``` 4. Copy the configuration file _file-connector.yaml_ to the container. - ```bash - - $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ - - ``` + ```bash + + $ docker cp connectors/file-connector.yaml pulsar-standalone:/pulsar/ + + ``` 5. Download the File source connector. - ```bash - - $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar - - ``` + ```bash + + $ curl -O https://mirrors.tuna.tsinghua.edu.cn/apache/pulsar/pulsar-{version}/connectors/pulsar-io-file-{version}.nar + + ``` 6. Start the File source connector. - ```bash - - $ docker exec -it pulsar-standalone /bin/bash - - $ ./bin/pulsar-admin sources localrun \ - --archive /pulsar/pulsar-io-file-{version}.nar \ - --name file-test \ - --destination-topic-name pulsar-file-test \ - --source-config-file /pulsar/file-connector.yaml + ```bash + + $ docker exec -it pulsar-standalone /bin/bash - ``` + $ ./bin/pulsar-admin sources localrun \ + --archive /pulsar/pulsar-io-file-{version}.nar \ + --name file-test \ + --destination-topic-name pulsar-file-test \ + --source-config-file /pulsar/file-connector.yaml + + ``` 7. Start a consumer. - ```bash - - ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test - - ``` - -8. Write the message to the file _test.txt_. + ```bash + ./bin/pulsar-client consume -s file-test -n 0 pulsar-file-test + + ``` - ```bash - - echo "hello world!" > /opt/test.txt - - ``` - - The following information appears on the consumer terminal window. +8. Write the message to the file _test.txt_. - ```bash + ```bash + + echo "hello world!" > /opt/test.txt + + ``` - ----- got message ----- - hello world! + The following information appears on the consumer terminal window. - ``` + ```bash + + ----- got message ----- + hello world! + + ``` - \ No newline at end of file + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-flume-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-flume-sink.md index b6b7e39879d5c..ded05d80726f1 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-flume-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-flume-sink.md @@ -1,7 +1,7 @@ --- id: io-flume-sink title: Flume sink connector -sidebar_label: Flume sink connector +sidebar_label: "Flume sink connector" original_id: io-flume-sink --- @@ -33,28 +33,28 @@ Before using the Flume sink connector, you need to create a configuration file t * JSON - ```json - - { - "name": "a1", - "confFile": "sink.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - - ``` + ```json + + { + "name": "a1", + "confFile": "sink.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - - configs: - name: a1 - confFile: sink.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - - ``` + ```yaml + + configs: + name: a1 + confFile: sink.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-flume-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-flume-source.md index a046aafcaad20..42f35e7ea499d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-flume-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-flume-source.md @@ -1,7 +1,7 @@ --- id: io-flume-source title: Flume source connector -sidebar_label: Flume source connector +sidebar_label: "Flume source connector" original_id: io-flume-source --- @@ -33,28 +33,28 @@ Before using the Flume source connector, you need to create a configuration file * JSON - ```json - - { - "name": "a1", - "confFile": "source.conf", - "noReloadConf": "false", - "zkConnString": "", - "zkBasePath": "" - } - - ``` + ```json + + { + "name": "a1", + "confFile": "source.conf", + "noReloadConf": "false", + "zkConnString": "", + "zkBasePath": "" + } + + ``` * YAML - ```yaml - - configs: - name: a1 - confFile: source.conf - noReloadConf: false - zkConnString: "" - zkBasePath: "" - - ``` + ```yaml + + configs: + name: a1 + confFile: source.conf + noReloadConf: false + zkConnString: "" + zkBasePath: "" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-hbase-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-hbase-sink.md index 9f5b36cff1776..ed1587eb53b1e 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-hbase-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-hbase-sink.md @@ -1,7 +1,7 @@ --- id: io-hbase-sink title: HBase sink connector -sidebar_label: HBase sink connector +sidebar_label: "HBase sink connector" original_id: io-hbase-sink --- @@ -37,35 +37,35 @@ Before using the HBase sink connector, you need to create a configuration file t * JSON - ```json - - { - "hbaseConfigResources": "hbase-site.xml", - "zookeeperQuorum": "localhost", - "zookeeperClientPort": "2181", - "zookeeperZnodeParent": "/hbase", - "tableName": "pulsar_hbase", - "rowKeyName": "rowKey", - "familyName": "info", - "qualifierNames": [ 'name', 'address', 'age'] - } - - ``` + ```json + + { + "hbaseConfigResources": "hbase-site.xml", + "zookeeperQuorum": "localhost", + "zookeeperClientPort": "2181", + "zookeeperZnodeParent": "/hbase", + "tableName": "pulsar_hbase", + "rowKeyName": "rowKey", + "familyName": "info", + "qualifierNames": [ 'name', 'address', 'age'] + } + + ``` * YAML - ```yaml - - configs: - hbaseConfigResources: "hbase-site.xml" - zookeeperQuorum: "localhost" - zookeeperClientPort: "2181" - zookeeperZnodeParent: "/hbase" - tableName: "pulsar_hbase" - rowKeyName: "rowKey" - familyName: "info" - qualifierNames: [ 'name', 'address', 'age'] - - ``` - - \ No newline at end of file + ```yaml + + configs: + hbaseConfigResources: "hbase-site.xml" + zookeeperQuorum: "localhost" + zookeeperClientPort: "2181" + zookeeperZnodeParent: "/hbase" + tableName: "pulsar_hbase" + rowKeyName: "rowKey" + familyName: "info" + qualifierNames: [ 'name', 'address', 'age'] + + ``` + + \ No newline at end of file diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-hdfs2-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-hdfs2-sink.md index 51c5597978e89..10369997588d6 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-hdfs2-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-hdfs2-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs2-sink title: HDFS2 sink connector -sidebar_label: HDFS2 sink connector +sidebar_label: "HDFS2 sink connector" original_id: io-hdfs2-sink --- @@ -39,29 +39,30 @@ Before using the HDFS2 sink connector, you need to create a configuration file t * JSON - ```json - - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "fileExtension": ".log", - "compression": "SNAPPY", - "subdirectoryPattern": "yyyy-MM-dd" - } - - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "fileExtension": ".log", + "compression": "SNAPPY", + "subdirectoryPattern": "yyyy-MM-dd" + } + + ``` * YAML - ```yaml - - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - fileExtension: ".log" - compression: "SNAPPY" - subdirectoryPattern: "yyyy-MM-dd" + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + fileExtension: ".log" + compression: "SNAPPY" + subdirectoryPattern: "yyyy-MM-dd" + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-hdfs3-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-hdfs3-sink.md index 3f21357f2f22c..6bd77ee58d277 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-hdfs3-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-hdfs3-sink.md @@ -1,7 +1,7 @@ --- id: io-hdfs3-sink title: HDFS3 sink connector -sidebar_label: HDFS3 sink connector +sidebar_label: "HDFS3 sink connector" original_id: io-hdfs3-sink --- @@ -38,25 +38,26 @@ Before using the HDFS3 sink connector, you need to create a configuration file t * JSON - ```json - - { - "hdfsConfigResources": "core-site.xml", - "directory": "/foo/bar", - "filenamePrefix": "prefix", - "compression": "SNAPPY" - } - - ``` + ```json + + { + "hdfsConfigResources": "core-site.xml", + "directory": "/foo/bar", + "filenamePrefix": "prefix", + "compression": "SNAPPY" + } + + ``` * YAML - ```yaml - - configs: - hdfsConfigResources: "core-site.xml" - directory: "/foo/bar" - filenamePrefix: "prefix" - compression: "SNAPPY" + ```yaml + + configs: + hdfsConfigResources: "core-site.xml" + directory: "/foo/bar" + filenamePrefix: "prefix" + compression: "SNAPPY" + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-influxdb-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-influxdb-sink.md index f7fa3f6e20056..0a0593ee3eaba 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-influxdb-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-influxdb-sink.md @@ -1,7 +1,7 @@ --- id: io-influxdb-sink title: InfluxDB sink connector -sidebar_label: InfluxDB sink connector +sidebar_label: "InfluxDB sink connector" original_id: io-influxdb-sink --- @@ -51,72 +51,75 @@ Before using the InfluxDB sink connector, you need to create a configuration fil #### InfluxDBv2 * JSON - ```json - - { - "influxdbUrl": "http://localhost:9999", - "organization": "example-org", - "bucket": "example-bucket", - "token": "xxxx", - "precision": "ns", - "logLevel": "NONE", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - - ``` + ```json + + { + "influxdbUrl": "http://localhost:9999", + "organization": "example-org", + "bucket": "example-bucket", + "token": "xxxx", + "precision": "ns", + "logLevel": "NONE", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` + * YAML - ```yaml - - { - influxdbUrl: "http://localhost:9999" - organization: "example-org" - bucket: "example-bucket" - token: "xxxx" - precision: "ns" - logLevel: "NONE" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } - - ``` + ```yaml + + { + influxdbUrl: "http://localhost:9999" + organization: "example-org" + bucket: "example-bucket" + token: "xxxx" + precision: "ns" + logLevel: "NONE" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + } + + ``` + #### InfluxDBv1 * JSON - ```json - - { - "influxdbUrl": "http://localhost:8086", - "database": "test_db", - "consistencyLevel": "ONE", - "logLevel": "NONE", - "retentionPolicy": "autogen", - "gzipEnable": false, - "batchTimeMs": 1000, - "batchSize": 100 - } - - ``` + ```json + + { + "influxdbUrl": "http://localhost:8086", + "database": "test_db", + "consistencyLevel": "ONE", + "logLevel": "NONE", + "retentionPolicy": "autogen", + "gzipEnable": false, + "batchTimeMs": 1000, + "batchSize": 100 + } + + ``` * YAML - ```yaml - - { - influxdbUrl: "http://localhost:8086" - database: "test_db" - consistencyLevel: "ONE" - logLevel: "NONE" - retentionPolicy: "autogen" - gzipEnable: false - batchTimeMs: 1000 - batchSize: 100 - } + ```yaml + + { + influxdbUrl: "http://localhost:8086" + database: "test_db" + consistencyLevel: "ONE" + logLevel: "NONE" + retentionPolicy: "autogen" + gzipEnable: false + batchTimeMs: 1000 + batchSize: 100 + } + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-jdbc-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-jdbc-sink.md index 8700dfc4a57fd..e992e724e534d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-jdbc-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-jdbc-sink.md @@ -1,7 +1,7 @@ --- id: io-jdbc-sink title: JDBC sink connector -sidebar_label: JDBC sink connector +sidebar_label: "JDBC sink connector" original_id: io-jdbc-sink --- @@ -35,65 +35,65 @@ The configuration of all JDBC sink connectors has the following properties. * JSON - ```json - - { - "userName": "clickhouse", - "password": "password", - "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", - "tableName": "pulsar_clickhouse_jdbc_sink" - } - - ``` + ```json + + { + "userName": "clickhouse", + "password": "password", + "jdbcUrl": "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink", + "tableName": "pulsar_clickhouse_jdbc_sink" + } + + ``` * YAML - ```yaml - - tenant: "public" - namespace: "default" - name: "jdbc-clickhouse-sink" - topicName: "persistent://public/default/jdbc-clickhouse-topic" - sinkType: "jdbc-clickhouse" - configs: - userName: "clickhouse" - password: "password" - jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" - tableName: "pulsar_clickhouse_jdbc_sink" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-clickhouse-sink" + topicName: "persistent://public/default/jdbc-clickhouse-topic" + sinkType: "jdbc-clickhouse" + configs: + userName: "clickhouse" + password: "password" + jdbcUrl: "jdbc:clickhouse://localhost:8123/pulsar_clickhouse_jdbc_sink" + tableName: "pulsar_clickhouse_jdbc_sink" + + ``` ### Example for MariaDB * JSON - ```json - - { - "userName": "mariadb", - "password": "password", - "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", - "tableName": "pulsar_mariadb_jdbc_sink" - } - - ``` + ```json + + { + "userName": "mariadb", + "password": "password", + "jdbcUrl": "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink", + "tableName": "pulsar_mariadb_jdbc_sink" + } + + ``` * YAML - ```yaml - - tenant: "public" - namespace: "default" - name: "jdbc-mariadb-sink" - topicName: "persistent://public/default/jdbc-mariadb-topic" - sinkType: "jdbc-mariadb" - configs: - userName: "mariadb" - password: "password" - jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" - tableName: "pulsar_mariadb_jdbc_sink" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-mariadb-sink" + topicName: "persistent://public/default/jdbc-mariadb-topic" + sinkType: "jdbc-mariadb" + configs: + userName: "mariadb" + password: "password" + jdbcUrl: "jdbc:mariadb://localhost:3306/pulsar_mariadb_jdbc_sink" + tableName: "pulsar_mariadb_jdbc_sink" + + ``` ### Example for PostgreSQL @@ -101,33 +101,33 @@ Before using the JDBC PostgreSQL sink connector, you need to create a configurat * JSON - ```json - - { - "userName": "postgres", - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "tableName": "pulsar_postgres_jdbc_sink" - } - - ``` + ```json + + { + "userName": "postgres", + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "tableName": "pulsar_postgres_jdbc_sink" + } + + ``` * YAML - ```yaml - - tenant: "public" - namespace: "default" - name: "jdbc-postgres-sink" - topicName: "persistent://public/default/jdbc-postgres-topic" - sinkType: "jdbc-postgres" - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" - - ``` + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-postgres-sink" + topicName: "persistent://public/default/jdbc-postgres-topic" + sinkType: "jdbc-postgres" + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" + tableName: "pulsar_postgres_jdbc_sink" + + ``` For more information on **how to use this JDBC sink connector**, see [connect Pulsar to PostgreSQL](io-quickstart.md#connect-pulsar-to-postgresql). @@ -135,26 +135,27 @@ For more information on **how to use this JDBC sink connector**, see [connect Pu * JSON - ```json - - { - "jdbcUrl": "jdbc:sqlite:db.sqlite", - "tableName": "pulsar_sqlite_jdbc_sink" - } - - ``` + ```json + + { + "jdbcUrl": "jdbc:sqlite:db.sqlite", + "tableName": "pulsar_sqlite_jdbc_sink" + } + + ``` * YAML - ```yaml - - tenant: "public" - namespace: "default" - name: "jdbc-sqlite-sink" - topicName: "persistent://public/default/jdbc-sqlite-topic" - sinkType: "jdbc-sqlite" - configs: - jdbcUrl: "jdbc:sqlite:db.sqlite" - tableName: "pulsar_sqlite_jdbc_sink" + ```yaml + + tenant: "public" + namespace: "default" + name: "jdbc-sqlite-sink" + topicName: "persistent://public/default/jdbc-sqlite-topic" + sinkType: "jdbc-sqlite" + configs: + jdbcUrl: "jdbc:sqlite:db.sqlite" + tableName: "pulsar_sqlite_jdbc_sink" + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-kafka-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-kafka-sink.md index c1669c1a95a5e..743f78514e068 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-kafka-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-kafka-sink.md @@ -1,7 +1,7 @@ --- id: io-kafka-sink title: Kafka sink connector -sidebar_label: Kafka sink connector +sidebar_label: "Kafka sink connector" original_id: io-kafka-sink --- @@ -38,40 +38,39 @@ Before using the Kafka sink connector, you need to create a configuration file t * JSON - ```json - - { - "bootstrapServers": "localhost:6667", - "topic": "test", - "acks": "1", - "batchSize": "16384", - "maxRequestSize": "1048576", - "producerConfigProperties": - { - "client.id": "test-pulsar-producer", - "security.protocol": "SASL_PLAINTEXT", - "sasl.mechanism": "GSSAPI", - "sasl.kerberos.service.name": "kafka", - "acks": "all" - } - } + ```json + + { + "bootstrapServers": "localhost:6667", + "topic": "test", + "acks": "1", + "batchSize": "16384", + "maxRequestSize": "1048576", + "producerConfigProperties": + { + "client.id": "test-pulsar-producer", + "security.protocol": "SASL_PLAINTEXT", + "sasl.mechanism": "GSSAPI", + "sasl.kerberos.service.name": "kafka", + "acks": "all" + } + } * YAML - - ```yaml - - configs: - bootstrapServers: "localhost:6667" - topic: "test" - acks: "1" - batchSize: "16384" - maxRequestSize: "1048576" - producerConfigProperties: - client.id: "test-pulsar-producer" - security.protocol: "SASL_PLAINTEXT" - sasl.mechanism: "GSSAPI" - sasl.kerberos.service.name: "kafka" - acks: "all" - - ``` + ``` + +yaml + configs: + bootstrapServers: "localhost:6667" + topic: "test" + acks: "1" + batchSize: "16384" + maxRequestSize: "1048576" + producerConfigProperties: + client.id: "test-pulsar-producer" + security.protocol: "SASL_PLAINTEXT" + sasl.mechanism: "GSSAPI" + sasl.kerberos.service.name: "kafka" + acks: "all" + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-kafka-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-kafka-source.md index fb852c9d085f2..737136c070197 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-kafka-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-kafka-source.md @@ -1,7 +1,7 @@ --- id: io-kafka-source title: Kafka source connector -sidebar_label: Kafka source connector +sidebar_label: "Kafka source connector" original_id: io-kafka-source --- @@ -71,30 +71,30 @@ Before using the Kafka source connector, you need to create a configuration file * JSON - ```json - - { - "bootstrapServers": "pulsar-kafka:9092", - "groupId": "test-pulsar-io", - "topic": "my-topic", - "sessionTimeoutMs": "10000", - "autoCommitEnabled": false - } - - ``` + ```json + + { + "bootstrapServers": "pulsar-kafka:9092", + "groupId": "test-pulsar-io", + "topic": "my-topic", + "sessionTimeoutMs": "10000", + "autoCommitEnabled": false + } + + ``` * YAML - ```yaml - - configs: - bootstrapServers: "pulsar-kafka:9092" - groupId: "test-pulsar-io" - topic: "my-topic" - sessionTimeoutMs: "10000" - autoCommitEnabled: false - - ``` + ```yaml + + configs: + bootstrapServers: "pulsar-kafka:9092" + groupId: "test-pulsar-io" + topic: "my-topic" + sessionTimeoutMs: "10000" + autoCommitEnabled: false + + ``` ## Usage @@ -102,134 +102,129 @@ Here is an example of using the Kafka source connector with the configuration fi 1. Download a Kafka client and a Kafka connector. - ```bash - - $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar - - $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar + ```bash + + $ wget https://repo1.maven.org/maven2/org/apache/kafka/kafka-clients/0.10.2.1/kafka-clients-0.10.2.1.jar - ``` + $ wget https://archive.apache.org/dist/pulsar/pulsar-2.4.0/connectors/pulsar-io-kafka-2.4.0.nar + + ``` 2. Create a network. - ```bash - + $ docker network create kafka-pulsar - + ``` 3. Pull a ZooKeeper image and start ZooKeeper. - ```bash - + $ docker pull wurstmeister/zookeeper $ docker run -d -it -p 2181:2181 --name pulsar-kafka-zookeeper --network kafka-pulsar wurstmeister/zookeeper - + ``` 4. Pull a Kafka image and start Kafka. - ```bash - + $ docker pull wurstmeister/kafka:2.11-1.0.2 $ docker run -d -it --network kafka-pulsar -p 6667:6667 -p 9092:9092 -e KAFKA_ADVERTISED_HOST_NAME=pulsar-kafka -e KAFKA_ZOOKEEPER_CONNECT=pulsar-kafka-zookeeper:2181 --name pulsar-kafka wurstmeister/kafka:2.11-1.0.2 - + ``` 5. Pull a Pulsar image and start Pulsar standalone. - ```bash - + $ docker pull apachepulsar/pulsar:@pulsar:version@ $ docker run -d -it --network kafka-pulsar -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-kafka-standalone apachepulsar/pulsar:2.4.0 bin/pulsar standalone - + ``` 6. Create a producer file _kafka-producer.py_. - ```python - + from kafka import KafkaProducer producer = KafkaProducer(bootstrap_servers='pulsar-kafka:9092') future = producer.send('my-topic', b'hello world') future.get() - + ``` 7. Create a consumer file _pulsar-client.py_. - ```python - - import pulsar - - client = pulsar.Client('pulsar://localhost:6650') - consumer = client.subscribe('my-topic', subscription_name='my-aa') - - while True: - msg = consumer.receive() - print msg - print dir(msg) - print("Received message: '%s'" % msg.data()) - consumer.acknowledge(msg) + ```python + + import pulsar - client.close() + client = pulsar.Client('pulsar://localhost:6650') + consumer = client.subscribe('my-topic', subscription_name='my-aa') - ``` + while True: + msg = consumer.receive() + print msg + print dir(msg) + print("Received message: '%s'" % msg.data()) + consumer.acknowledge(msg) -8. Copy the following files to Pulsar. + client.close() + ``` - ```bash - - $ docker cp pulsar-io-kafka-@pulsar:version@.nar pulsar-kafka-standalone:/pulsar - $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf - $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ - $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ +8. Copy the following files to Pulsar. - ``` + ```bash + + $ docker cp pulsar-io-kafka-@pulsar:version@.nar pulsar-kafka-standalone:/pulsar + $ docker cp kafkaSourceConfig.yaml pulsar-kafka-standalone:/pulsar/conf + $ docker cp pulsar-client.py pulsar-kafka-standalone:/pulsar/ + $ docker cp kafka-producer.py pulsar-kafka-standalone:/pulsar/ + + ``` 9. Open a new terminal window and start the Kafka source connector in local run mode. - ```bash - - $ docker exec -it pulsar-kafka-standalone /bin/bash - - $ ./bin/pulsar-admin source localrun \ - --archive ./pulsar-io-kafka-@pulsar:version@.nar \ - --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ - --tenant public \ - --namespace default \ - --name kafka \ - --destination-topic-name my-topic \ - --source-config-file ./conf/kafkaSourceConfig.yaml \ - --parallelism 1 - - ``` + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash + + $ ./bin/pulsar-admin source localrun \ + --archive ./pulsar-io-kafka-@pulsar:version@.nar \ + --classname org.apache.pulsar.io.kafka.KafkaBytesSource \ + --tenant public \ + --namespace default \ + --name kafka \ + --destination-topic-name my-topic \ + --source-config-file ./conf/kafkaSourceConfig.yaml \ + --parallelism 1 + + ``` 10. Open a new terminal window and run the consumer. - ```bash - - $ docker exec -it pulsar-kafka-standalone /bin/bash - - $ pip install kafka-python - - $ python3 kafka-producer.py + ```bash + + $ docker exec -it pulsar-kafka-standalone /bin/bash - ``` + $ pip install kafka-python - The following information appears on the consumer terminal window. + $ python3 kafka-producer.py + + ``` - ```bash + The following information appears on the consumer terminal window. - Received message: 'hello world' + ```bash + + Received message: 'hello world' + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-sink.md index e84743c0c45c9..483c861db284d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-sink.md @@ -1,7 +1,7 @@ --- id: io-kinesis-sink title: Kinesis sink connector -sidebar_label: Kinesis sink connector +sidebar_label: "Kinesis sink connector" original_id: io-kinesis-sink --- @@ -33,21 +33,21 @@ The following are built-in `AwsCredentialProviderPlugin` plugins: * `org.apache.pulsar.io.aws.AwsDefaultProviderChainPlugin` - This plugin takes no configuration, it uses the default AWS provider chain. - - For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). + This plugin takes no configuration, it uses the default AWS provider chain. + + For more information, see [AWS documentation](https://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). * `org.apache.pulsar.io.aws.STSAssumeRoleProviderPlugin` - This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. - - This configuration takes the form of a small json document like: - - ```json + This plugin takes a configuration (via the `awsCredentialPluginParam`) that describes a role to assume when running the KCL. - {"roleArn": "arn...", "roleSessionName": "name"} + This configuration takes the form of a small json document like: - ``` + ```json + + {"roleArn": "arn...", "roleSessionName": "name"} + + ``` ### Example @@ -55,29 +55,30 @@ Before using the Kinesis sink connector, you need to create a configuration file * JSON - ```json - - { - "awsEndpoint": "some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "messageFormat": "ONLY_RAW_PAYLOAD", - "retainOrdering": "true" - } - - ``` + ```json + + { + "awsEndpoint": "some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "messageFormat": "ONLY_RAW_PAYLOAD", + "retainOrdering": "true" + } + + ``` * YAML - ```yaml - - configs: - awsEndpoint: "some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - messageFormat: "ONLY_RAW_PAYLOAD" - retainOrdering: "true" + ```yaml + + configs: + awsEndpoint: "some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + messageFormat: "ONLY_RAW_PAYLOAD" + retainOrdering: "true" + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-source.md index f38a6237ec24a..3e593a31161ad 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-kinesis-source.md @@ -1,7 +1,7 @@ --- id: io-kinesis-source title: Kinesis source connector -sidebar_label: Kinesis source connector +sidebar_label: "Kinesis source connector" original_id: io-kinesis-source --- @@ -46,40 +46,40 @@ Before using the Kinesis source connector, you need to create a configuration fi * JSON - ```json - - { - "awsEndpoint": "https://some.endpoint.aws", - "awsRegion": "us-east-1", - "awsKinesisStreamName": "my-stream", - "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", - "applicationName": "My test application", - "checkpointInterval": "30000", - "backoffTime": "4000", - "numRetries": "3", - "receiveQueueSize": 2000, - "initialPositionInStream": "TRIM_HORIZON", - "startAtTime": "2019-03-05T19:28:58.000Z" - } - - ``` + ```json + + { + "awsEndpoint": "https://some.endpoint.aws", + "awsRegion": "us-east-1", + "awsKinesisStreamName": "my-stream", + "awsCredentialPluginParam": "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}", + "applicationName": "My test application", + "checkpointInterval": "30000", + "backoffTime": "4000", + "numRetries": "3", + "receiveQueueSize": 2000, + "initialPositionInStream": "TRIM_HORIZON", + "startAtTime": "2019-03-05T19:28:58.000Z" + } + + ``` * YAML - ```yaml - - configs: - awsEndpoint: "https://some.endpoint.aws" - awsRegion: "us-east-1" - awsKinesisStreamName: "my-stream" - awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" - applicationName: "My test application" - checkpointInterval: 30000 - backoffTime: 4000 - numRetries: 3 - receiveQueueSize: 2000 - initialPositionInStream: "TRIM_HORIZON" - startAtTime: "2019-03-05T19:28:58.000Z" - - ``` + ```yaml + + configs: + awsEndpoint: "https://some.endpoint.aws" + awsRegion: "us-east-1" + awsKinesisStreamName: "my-stream" + awsCredentialPluginParam: "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}" + applicationName: "My test application" + checkpointInterval: 30000 + backoffTime: 4000 + numRetries: 3 + receiveQueueSize: 2000 + initialPositionInStream: "TRIM_HORIZON" + startAtTime: "2019-03-05T19:28:58.000Z" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-mongo-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-mongo-sink.md index e1a03a3fab607..b370464c18f4b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-mongo-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-mongo-sink.md @@ -1,7 +1,7 @@ --- id: io-mongo-sink title: MongoDB sink connector -sidebar_label: MongoDB sink connector +sidebar_label: "MongoDB sink connector" original_id: io-mongo-sink --- @@ -32,31 +32,30 @@ The configuration of the MongoDB sink connector has the following properties. Before using the Mongo sink connector, you need to create a configuration file through one of the following methods. * JSON - - - ```json - - { - "mongoUri": "mongodb://localhost:27017", - "database": "pulsar", - "collection": "messages", - "batchSize": "2", - "batchTimeMs": "500" - } - ``` - -* YAML + ```json + { + "mongoUri": "mongodb://localhost:27017", + "database": "pulsar", + "collection": "messages", + "batchSize": "2", + "batchTimeMs": "500" + } + + ``` - ```yaml +* YAML - { - mongoUri: "mongodb://localhost:27017" - database: "pulsar" - collection: "messages" - batchSize: 2 - batchTimeMs: 500 - } + ```yaml + + { + mongoUri: "mongodb://localhost:27017" + database: "pulsar" + collection: "messages" + batchSize: 2 + batchTimeMs: 500 + } + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-netty-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-netty-source.md index ec21729d550d0..d41265fa4fa28 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-netty-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-netty-source.md @@ -1,7 +1,7 @@ --- id: io-netty-source title: Netty source connector -sidebar_label: Netty source connector +sidebar_label: "Netty source connector" original_id: io-netty-source --- @@ -34,28 +34,28 @@ Before using the Netty source connector, you need to create a configuration file * JSON - ```json - - { - "type": "tcp", - "host": "127.0.0.1", - "port": "10911", - "numberOfThreads": "1" - } - - ``` + ```json + + { + "type": "tcp", + "host": "127.0.0.1", + "port": "10911", + "numberOfThreads": "1" + } + + ``` * YAML - ```yaml - - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 - - ``` + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` ## Usage @@ -65,49 +65,47 @@ The following examples show how to use the Netty source connector with TCP and H 1. Start Pulsar standalone. - ```bash - - $ docker pull apachepulsar/pulsar:{version} - - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - - ``` - -2. Create a configuration file _netty-source-config.yaml_. + ```bash + $ docker pull apachepulsar/pulsar:{version} - ```yaml + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` - configs: - type: "tcp" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 +2. Create a configuration file _netty-source-config.yaml_. - ``` + ```yaml + + configs: + type: "tcp" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` 3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. - ```bash - - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ - - ``` + ```bash + + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + + ``` - ``` - 5. Start the Netty source connector. - ```bash - + $ ./bin/pulsar-admin sources localrun \ --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ @@ -116,98 +114,95 @@ The following examples show how to use the Netty source connector with TCP and H --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 - + ``` 6. Consume data. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ apt-get update - - $ apt-get -y install telnet - - $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 - Trying 127.0.0.1... - Connected to 127.0.0.1. - Escape character is '^]'. - hello - world - - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ apt-get update + + $ apt-get -y install telnet + + $ root@1d19327b2c67:/pulsar# telnet 127.0.0.1 10999 + Trying 127.0.0.1... + Connected to 127.0.0.1. + Escape character is '^]'. + hello + world + + ``` 8. The following information appears on the consumer terminal window. - ```bash - - ----- got message ----- - hello - - ----- got message ----- - world + ```bash + + ----- got message ----- + hello - ``` + ----- got message ----- + world + + ``` ### HTTP 1. Start Pulsar standalone. - ```bash - - $ docker pull apachepulsar/pulsar:{version} - - $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone - - ``` - -2. Create a configuration file _netty-source-config.yaml_. + ```bash + $ docker pull apachepulsar/pulsar:{version} - ```yaml - - configs: - type: "http" - host: "127.0.0.1" - port: 10999 - numberOfThreads: 1 + $ docker run -d -it -p 6650:6650 -p 8080:8080 -v $PWD/data:/pulsar/data --name pulsar-netty-standalone apachepulsar/pulsar:{version} bin/pulsar standalone + + ``` - ``` +2. Create a configuration file _netty-source-config.yaml_. -3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. + ```yaml + configs: + type: "http" + host: "127.0.0.1" + port: 10999 + numberOfThreads: 1 + + ``` - ```bash - - $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ +3. Copy the configuration file _netty-source-config.yaml_ to Pulsar server. - ``` + ```bash + + $ docker cp netty-source-config.yaml pulsar-netty-standalone:/pulsar/conf/ + + ``` 4. Download the Netty source connector. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + curl -O http://mirror-hk.koddos.net/apache/pulsar/pulsar-{version}/connectors/pulsar-io-netty-{version}.nar + + ``` - ``` - 5. Start the Netty source connector. - ```bash - + $ ./bin/pulsar-admin sources localrun \ --archive pulsar-io-@pulsar:version@.nar \ --tenant public \ @@ -216,34 +211,35 @@ The following examples show how to use the Netty source connector with TCP and H --destination-topic-name netty-topic \ --source-config-file netty-source-config.yaml \ --parallelism 1 - + ``` 6. Consume data. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 - - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ ./bin/pulsar-client consume -t Exclusive -s netty-sub netty-topic -n 0 + + ``` 7. Open another terminal window to send data to the Netty source. - ```bash - - $ docker exec -it pulsar-netty-standalone /bin/bash - - $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ - - ``` + ```bash + + $ docker exec -it pulsar-netty-standalone /bin/bash + + $ curl -X POST --data 'hello, world!' http://127.0.0.1:10999/ + + ``` 8. The following information appears on the consumer terminal window. - ```bash - - ----- got message ----- - hello, world! + ```bash + + ----- got message ----- + hello, world! + + ``` - ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-nsq-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-nsq-source.md index c1a7073056f5a..4a89e916a8a2b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-nsq-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-nsq-source.md @@ -1,7 +1,7 @@ --- id: io-nsq-source title: NSQ source connector -sidebar_label: NSQ source connector +sidebar_label: "NSQ source connector" original_id: io-nsq-source --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-overview.md b/site2/website-next/versioned_docs/version-2.8.0/io-overview.md index 816e8aa5106a7..d56b9c03be35f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-overview.md @@ -1,7 +1,7 @@ --- id: io-overview title: Pulsar connector overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: io-overview --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-quickstart.md b/site2/website-next/versioned_docs/version-2.8.0/io-quickstart.md index 6df4de8a66903..67ca9710a10a8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-quickstart.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-quickstart.md @@ -1,7 +1,7 @@ --- id: io-quickstart title: How to connect Pulsar to database -sidebar_label: Get started +sidebar_label: "Get started" original_id: io-quickstart --- @@ -21,10 +21,8 @@ At the end of this tutorial, you are able to: :::tip - * These instructions assume you are running Pulsar in [standalone mode](getting-started-standalone). However, all the commands used in this tutorial can be used in a multi-nodes Pulsar cluster without any changes. - * All the instructions are assumed to run at the root directory of a Pulsar binary distribution. ::: @@ -39,74 +37,74 @@ For more information about **how to install a standalone Pulsar and built-in con 1. Start Pulsar locally. - ```bash - - bin/pulsar standalone - - ``` + ```bash + + bin/pulsar standalone + + ``` - All the components of a Pulsar service are start in order. - - You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. + All the components of a Pulsar service are start in order. + + You can curl those pulsar service endpoints to make sure Pulsar service is up running correctly. 2. Check Pulsar binary protocol port. - ```bash - - telnet localhost 6650 - - ``` + ```bash + + telnet localhost 6650 + + ``` 3. Check Pulsar Function cluster. - ```bash - - curl -s http://localhost:8080/admin/v2/worker/cluster - - ``` - - **Example output** - - ```json + ```bash + + curl -s http://localhost:8080/admin/v2/worker/cluster + + ``` - [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] + **Example output** - ``` + ```json + + [{"workerId":"c-standalone-fw-localhost-6750","workerHostname":"localhost","port":6750}] + + ``` 4. Make sure a public tenant and a default namespace exist. - ```bash - - curl -s http://localhost:8080/admin/v2/namespaces/public - - ``` - - **Example output** - - ```json + ```bash + + curl -s http://localhost:8080/admin/v2/namespaces/public + + ``` - ["public/default","public/functions"] + **Example output** - ``` + ```json + + ["public/default","public/functions"] + + ``` 5. All built-in connectors should be listed as available. - ```bash - - curl -s http://localhost:8080/admin/v2/functions/connectors - - ``` - - **Example output** - - ```json + ```bash + + curl -s http://localhost:8080/admin/v2/functions/connectors + + ``` - [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] + **Example output** - ``` + ```json + + [{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaBytesSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}] + + ``` - If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, - or you can navigate to the `logs` directory under the Pulsar directory to view the logs. + If an error occurs when starting Pulsar service, you may see an exception at the terminal running `pulsar/standalone`, + or you can navigate to the `logs` directory under the Pulsar directory to view the logs. ## Connect Pulsar to Cassandra @@ -114,9 +112,7 @@ This section demonstrates how to connect Pulsar to Cassandra. :::tip - * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - * The Cassandra sink connector reads messages from Pulsar topics and writes the messages into Cassandra tables. For more information, see [Cassandra sink connector](io-cassandra-sink). ::: @@ -127,13 +123,13 @@ This example uses `cassandra` Docker image to start a single-node Cassandra clus 1. Start a Cassandra cluster. -```bash - -docker run -d --rm --name=cassandra -p 9042:9042 cassandra - -``` + ```bash + + docker run -d --rm --name=cassandra -p 9042:9042 cassandra + + ``` -:::note + :::note Before moving to the next steps, make sure the Cassandra cluster is running. @@ -141,68 +137,69 @@ docker run -d --rm --name=cassandra -p 9042:9042 cassandra 2. Make sure the Docker process is running. - ```bash - - docker ps - - ``` + ```bash + + docker ps + + ``` 3. Check the Cassandra logs to make sure the Cassandra process is running as expected. - ```bash - - docker logs cassandra - - ``` + ```bash + + docker logs cassandra + + ``` 4. Check the status of the Cassandra cluster. - ```bash - - docker exec cassandra nodetool status - - ``` - - **Example output** + ```bash + + docker exec cassandra nodetool status + + ``` - ``` - Datacenter: datacenter1 - ======================= - Status=Up/Down - |/ State=Normal/Leaving/Joining/Moving - -- Address Load Tokens Owns (effective) Host ID Rack - UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 + **Example output** - ``` + ``` + + Datacenter: datacenter1 + ======================= + Status=Up/Down + |/ State=Normal/Leaving/Joining/Moving + -- Address Load Tokens Owns (effective) Host ID Rack + UN 172.17.0.2 103.67 KiB 256 100.0% af0e4b2f-84e0-4f0b-bb14-bd5f9070ff26 rack1 + + ``` 5. Use `cqlsh` to connect to the Cassandra cluster. - ```bash - - $ docker exec -ti cassandra cqlsh localhost - Connected to Test Cluster at localhost:9042. - [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] - Use HELP for help. - cqlsh> - - ``` + ```bash + + $ docker exec -ti cassandra cqlsh localhost + Connected to Test Cluster at localhost:9042. + [cqlsh 5.0.1 | Cassandra 3.11.2 | CQL spec 3.4.4 | Native protocol v4] + Use HELP for help. + cqlsh> + + ``` 6. Create a keyspace `pulsar_test_keyspace`. - ```bash - - cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; - - ``` + ```bash + + cqlsh> CREATE KEYSPACE pulsar_test_keyspace WITH replication = {'class':'SimpleStrategy', 'replication_factor':1}; + + ``` 7. Create a table `pulsar_test_table`. - ```bash - - cqlsh> USE pulsar_test_keyspace; - cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); - - ``` + ```bash + + cqlsh> USE pulsar_test_keyspace; + cqlsh:pulsar_test_keyspace> CREATE TABLE pulsar_test_table (key text PRIMARY KEY, col text); + + ``` ### Configure a Cassandra sink @@ -218,32 +215,31 @@ You can create a configuration file through one of the following methods. * JSON - ```json - - { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - } - - ``` - -* YAML + ```json + { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + } + + ``` - ```yaml - - configs: - roots: "localhost:9042" - keyspace: "pulsar_test_keyspace" - columnFamily: "pulsar_test_table" - keyname: "key" - columnName: "col" +* YAML - ``` + ```yaml + + configs: + roots: "localhost:9042" + keyspace: "pulsar_test_keyspace" + columnFamily: "pulsar_test_table" + keyname: "key" + columnName: "col" + ``` + For more information, see [Cassandra sink connector](io-cassandra-sink). ### Create a Cassandra sink @@ -281,160 +277,160 @@ to monitor a connector and perform other operations on it. * Get the information of a Cassandra sink. ```bash - + bin/pulsar-admin sinks get \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink - + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json - + { - "tenant": "public", - "namespace": "default", - "name": "cassandra-test-sink", - "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", - "inputSpecs": { - "test_cassandra": { - "isRegexPattern": false - } - }, - "configs": { - "roots": "localhost:9042", - "keyspace": "pulsar_test_keyspace", - "columnFamily": "pulsar_test_table", - "keyname": "key", - "columnName": "col" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true, - "archive": "builtin://cassandra" + "tenant": "public", + "namespace": "default", + "name": "cassandra-test-sink", + "className": "org.apache.pulsar.io.cassandra.CassandraStringSink", + "inputSpecs": { + "test_cassandra": { + "isRegexPattern": false + } + }, + "configs": { + "roots": "localhost:9042", + "keyspace": "pulsar_test_keyspace", + "columnFamily": "pulsar_test_table", + "keyname": "key", + "columnName": "col" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true, + "archive": "builtin://cassandra" } - + ``` * Check the status of a Cassandra sink. ```bash - + bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink - + --tenant public \ + --namespace default \ + --name cassandra-test-sink + ``` **Example output** ```json - + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] } - + ``` ### Verify a Cassandra sink 1. Produce some messages to the input topic of the Cassandra sink _test_cassandra_. - ```bash + ```bash + + for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done + + ``` - for i in {0..9}; do bin/pulsar-client produce -m "key-$i" -n 1 test_cassandra; done +2. Inspect the status of the Cassandra sink _test_cassandra_. - ``` + ```bash + + bin/pulsar-admin sinks status \ + --tenant public \ + --namespace default \ + --name cassandra-test-sink + + ``` -2. Inspect the status of the Cassandra sink _test_cassandra_. + You can see 10 messages are processed by the Cassandra sink _test_cassandra_. - ```bash - - bin/pulsar-admin sinks status \ - --tenant public \ - --namespace default \ - --name cassandra-test-sink - - ``` - - You can see 10 messages are processed by the Cassandra sink _test_cassandra_. - - **Example output** - - ```json - - { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 10, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 10, - "lastReceivedTime" : 1551685489136, - "workerId" : "c-standalone-fw-localhost-8080" - } - } ] - } + **Example output** - ``` + ```json + + { + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 10, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 10, + "lastReceivedTime" : 1551685489136, + "workerId" : "c-standalone-fw-localhost-8080" + } + } ] + } + + ``` 3. Use `cqlsh` to connect to the Cassandra cluster. ```bash - + docker exec -ti cassandra cqlsh localhost - + ``` 4. Check the data of the Cassandra table _pulsar_test_table_. ```bash - + cqlsh> use pulsar_test_keyspace; cqlsh:pulsar_test_keyspace> select * from pulsar_test_table; key | col --------+-------- - key-5 | key-5 - key-0 | key-0 - key-9 | key-9 - key-2 | key-2 - key-1 | key-1 - key-3 | key-3 - key-6 | key-6 - key-7 | key-7 - key-4 | key-4 - key-8 | key-8 - + key-5 | key-5 + key-0 | key-0 + key-9 | key-9 + key-2 | key-2 + key-1 | key-1 + key-3 | key-3 + key-6 | key-6 + key-7 | key-7 + key-4 | key-4 + key-8 | key-8 + ``` ### Delete a Cassandra Sink @@ -457,12 +453,11 @@ This section demonstrates how to connect Pulsar to PostgreSQL. :::tip - * Make sure you have Docker installed. If you do not have one, see [install Docker](https://docs.docker.com/docker-for-mac/install/). - * The JDBC sink connector pulls messages from Pulsar topics ::: + and persists the messages to ClickHouse, MariaDB, PostgreSQL, or SQlite. >For more information, see [JDBC sink connector](io-jdbc-sink). @@ -473,35 +468,35 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS 1. Pull the PostgreSQL 12 image from Docker. - ```bash - - $ docker pull postgres:12 - - ``` + ```bash + + $ docker pull postgres:12 + + ``` 2. Start PostgreSQL. - ```bash - - $ docker run -d -it --rm \ - --name pulsar-postgres \ - -p 5432:5432 \ - -e POSTGRES_PASSWORD=password \ - -e POSTGRES_USER=postgres \ - postgres:12 - - ``` + ```bash + + $ docker run -d -it --rm \ + --name pulsar-postgres \ + -p 5432:5432 \ + -e POSTGRES_PASSWORD=password \ + -e POSTGRES_USER=postgres \ + postgres:12 + + ``` - #### Tip - - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + #### Tip + + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. :::tip @@ -511,46 +506,46 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS 3. Check if PostgreSQL has been started successfully. - ```bash - - $ docker logs -f pulsar-postgres - - ``` - - PostgreSQL has been started successfully if the following message appears. - - ```text + ```bash + + $ docker logs -f pulsar-postgres + + ``` - 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 - 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 - 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" - 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC - 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections + PostgreSQL has been started successfully if the following message appears. - ``` + ```text + + 2020-05-11 20:09:24.492 UTC [1] LOG: starting PostgreSQL 12.2 (Debian 12.2-2.pgdg100+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 8.3.0-6) 8.3.0, 64-bit + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv4 address "0.0.0.0", port 5432 + 2020-05-11 20:09:24.492 UTC [1] LOG: listening on IPv6 address "::", port 5432 + 2020-05-11 20:09:24.499 UTC [1] LOG: listening on Unix socket "/var/run/postgresql/.s.PGSQL.5432" + 2020-05-11 20:09:24.523 UTC [55] LOG: database system was shut down at 2020-05-11 20:09:24 UTC + 2020-05-11 20:09:24.533 UTC [1] LOG: database system is ready to accept connections + + ``` 4. Access to PostgreSQL. - ```bash - - $ docker exec -it pulsar-postgres /bin/bash - - ``` + ```bash + + $ docker exec -it pulsar-postgres /bin/bash + + ``` 5. Create a PostgreSQL table _pulsar_postgres_jdbc_sink_. - ```bash - - $ psql -U postgres postgres - - postgres=# create table if not exists pulsar_postgres_jdbc_sink - ( - id serial PRIMARY KEY, - name VARCHAR(255) NOT NULL - ); - - ``` + ```bash + + $ psql -U postgres postgres + + postgres=# create table if not exists pulsar_postgres_jdbc_sink + ( + id serial PRIMARY KEY, + name VARCHAR(255) NOT NULL + ); + + ``` ### Configure a JDBC sink @@ -560,35 +555,35 @@ In this section, you need to configure a JDBC sink connector. 1. Add a configuration file. - To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. - - For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. - - Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - - ```yaml + To run a JDBC sink connector, you need to prepare a YAML configuration file including the information that Pulsar connector runtime needs to know. + + For example, how Pulsar connector can find the PostgreSQL cluster, what is the JDBC URL and the table that Pulsar connector uses for writing messages to. - configs: - userName: "postgres" - password: "password" - jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" - tableName: "pulsar_postgres_jdbc_sink" + Create a _pulsar-postgres-jdbc-sink.yaml_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - ``` + ```yaml + + configs: + userName: "postgres" + password: "password" + jdbcUrl: "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink" + tableName: "pulsar_postgres_jdbc_sink" + + ``` 2. Create a schema. - Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - - ```json + Create a _avro-schema_ file, copy the following contents to this file, and place the file in the `pulsar/connectors` folder. - { - "type": "AVRO", - "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", - "properties": {} - } - - ``` + ```json + + { + "type": "AVRO", + "schema": "{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}", + "properties": {} + } + + ``` :::tip @@ -598,29 +593,29 @@ In this section, you need to configure a JDBC sink connector. 3. Upload a schema to a topic. - This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. - - ```bash - - $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema + This example uploads the _avro-schema_ schema to the _pulsar-postgres-jdbc-sink-topic_ topic. - ``` + ```bash + + $ bin/pulsar-admin schemas upload pulsar-postgres-jdbc-sink-topic -f ./connectors/avro-schema + + ``` 4. Check if the schema has been uploaded successfully. - ```bash - - $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic - - ``` - - The schema has been uploaded successfully if the following message appears. - - ```json + ```bash + + $ bin/pulsar-admin schemas get pulsar-postgres-jdbc-sink-topic + + ``` - {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} + The schema has been uploaded successfully if the following message appears. - ``` + ```json + + {"name":"pulsar-postgres-jdbc-sink-topic","schema":"{\"type\":\"record\",\"name\":\"Test\",\"fields\":[{\"name\":\"id\",\"type\":[\"null\",\"int\"]},{\"name\":\"name\",\"type\":[\"null\",\"string\"]}]}","type":"AVRO","properties":{}} + + ``` ### Create a JDBC sink @@ -656,7 +651,6 @@ This sink connector runs as a Pulsar Function and writes the messages produced i :::tip - For more information about `pulsar-admin sinks create options`, see [here](io-cli.md#sinks). ::: @@ -677,119 +671,116 @@ to monitor a connector and perform other operations on it. * List all running JDBC sink(s). ```bash - + $ bin/pulsar-admin sinks list \ --tenant public \ --namespace default - + ``` -:::tip - + :::tip -For more information about `pulsar-admin sinks list options`, see [here](io-cli.md/#list-1). + For more information about `pulsar-admin sinks list options`, see [here](io-cli.md/#list-1). -::: + ::: The result shows that only the _postgres-jdbc-sink_ sink is running. ```json - + [ "pulsar-postgres-jdbc-sink" ] - + ``` * Get the information of a JDBC sink. ```bash - + $ bin/pulsar-admin sinks get \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink - + ``` -:::tip + :::tip + For more information about `pulsar-admin sinks get options`, see [here](io-cli.md/#get-1). -For more information about `pulsar-admin sinks get options`, see [here](io-cli.md/#get-1). - -::: + ::: The result shows the information of the sink connector, including tenant, namespace, topic and so on. ```json - + { - "tenant": "public", - "namespace": "default", - "name": "pulsar-postgres-jdbc-sink", - "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", - "inputSpecs": { - "pulsar-postgres-jdbc-sink-topic": { - "isRegexPattern": false - } - }, - "configs": { - "password": "password", - "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", - "userName": "postgres", - "tableName": "pulsar_postgres_jdbc_sink" - }, - "parallelism": 1, - "processingGuarantees": "ATLEAST_ONCE", - "retainOrdering": false, - "autoAck": true + "tenant": "public", + "namespace": "default", + "name": "pulsar-postgres-jdbc-sink", + "className": "org.apache.pulsar.io.jdbc.PostgresJdbcAutoSchemaSink", + "inputSpecs": { + "pulsar-postgres-jdbc-sink-topic": { + "isRegexPattern": false + } + }, + "configs": { + "password": "password", + "jdbcUrl": "jdbc:postgresql://localhost:5432/pulsar_postgres_jdbc_sink", + "userName": "postgres", + "tableName": "pulsar_postgres_jdbc_sink" + }, + "parallelism": 1, + "processingGuarantees": "ATLEAST_ONCE", + "retainOrdering": false, + "autoAck": true } - + ``` * Get the status of a JDBC sink ```bash - + $ bin/pulsar-admin sinks status \ --tenant public \ --namespace default \ --name pulsar-postgres-jdbc-sink - + ``` -:::tip + :::tip + For more information about `pulsar-admin sinks status options`, see [here](io-cli.md/#status-1). -For more information about `pulsar-admin sinks status options`, see [here](io-cli.md/#status-1). - -::: + ::: The result shows the current status of sink connector, including the number of instance, running status, worker ID and so on. ```json - + { - "numInstances" : 1, - "numRunning" : 1, - "instances" : [ { - "instanceId" : 0, - "status" : { - "running" : true, - "error" : "", - "numRestarts" : 0, - "numReadFromPulsar" : 0, - "numSystemExceptions" : 0, - "latestSystemExceptions" : [ ], - "numSinkExceptions" : 0, - "latestSinkExceptions" : [ ], - "numWrittenToSink" : 0, - "lastReceivedTime" : 0, - "workerId" : "c-standalone-fw-192.168.2.52-8080" - } - } ] + "numInstances" : 1, + "numRunning" : 1, + "instances" : [ { + "instanceId" : 0, + "status" : { + "running" : true, + "error" : "", + "numRestarts" : 0, + "numReadFromPulsar" : 0, + "numSystemExceptions" : 0, + "latestSystemExceptions" : [ ], + "numSinkExceptions" : 0, + "latestSinkExceptions" : [ ], + "numWrittenToSink" : 0, + "lastReceivedTime" : 0, + "workerId" : "c-standalone-fw-192.168.2.52-8080" + } + } ] } - + ``` ### Stop a JDBC sink @@ -808,7 +799,6 @@ $ bin/pulsar-admin sinks stop \ :::tip - For more information about `pulsar-admin sinks stop options`, see [here](io-cli.md/#stop-1). ::: @@ -831,13 +821,12 @@ to restart a connector and perform other operations on it. $ bin/pulsar-admin sinks restart \ --tenant public \ --namespace default \ ---name pulsar-postgres-jdbc-sink +--name pulsar-postgres-jdbc-sink ``` :::tip - For more information about `pulsar-admin sinks restart options`, see [here](io-cli.md/#restart-1). ::: @@ -852,11 +841,8 @@ The sink instance has been started successfully if the following message disappe :::tip - * Optionally, you can run a standalone sink connector using `pulsar-admin sinks localrun options`. - Note that `pulsar-admin sinks localrun options` **runs a sink connector locally**, while `pulsar-admin sinks start options` **starts a sink connector in a cluster**. - * For more information about `pulsar-admin sinks localrun options`, see [here](io-cli.md#localrun-1). ::: @@ -878,7 +864,6 @@ $ bin/pulsar-admin sinks update \ :::tip - For more information about `pulsar-admin sinks update options`, see [here](io-cli.md/#update-1). ::: @@ -948,7 +933,6 @@ $ bin/pulsar-admin sinks delete \ :::tip - For more information about `pulsar-admin sinks delete options`, see [here](io-cli.md/#delete-1). ::: @@ -981,3 +965,4 @@ HTTP 404 Not Found Reason: Sink pulsar-postgres-jdbc-sink doesn't exist ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-sink.md index af298ee62952a..48f1c75678559 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-sink.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-sink title: RabbitMQ sink connector -sidebar_label: RabbitMQ sink connector +sidebar_label: "RabbitMQ sink connector" original_id: io-rabbitmq-sink --- @@ -44,46 +44,46 @@ Before using the RabbitMQ sink connector, you need to create a configuration fil * JSON - ```json - - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "exchangeName": "test-exchange", - "routingKey": "test-key" - } - - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "exchangeName": "test-exchange", + "routingKey": "test-key" + } + + ``` * YAML - ```yaml - - configs: - host: "localhost" - port: 5672 - virtualHost: "/", - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - exchangeName: "test-exchange" - routingKey: "test-key" - - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/", + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + exchangeName: "test-exchange" + routingKey: "test-key" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-source.md index 76aea7fbf2b0b..0a10cfaab1b1a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-rabbitmq-source.md @@ -1,7 +1,7 @@ --- id: io-rabbitmq-source title: RabbitMQ source connector -sidebar_label: RabbitMQ source connector +sidebar_label: "RabbitMQ source connector" original_id: io-rabbitmq-source --- @@ -42,48 +42,48 @@ Before using the RabbitMQ source connector, you need to create a configuration f * JSON - ```json - - { - "host": "localhost", - "port": "5672", - "virtualHost": "/", - "username": "guest", - "password": "guest", - "queueName": "test-queue", - "connectionName": "test-connection", - "requestedChannelMax": "0", - "requestedFrameMax": "0", - "connectionTimeout": "60000", - "handshakeTimeout": "10000", - "requestedHeartbeat": "60", - "prefetchCount": "0", - "prefetchGlobal": "false", - "passive": "false" - } - - ``` + ```json + + { + "host": "localhost", + "port": "5672", + "virtualHost": "/", + "username": "guest", + "password": "guest", + "queueName": "test-queue", + "connectionName": "test-connection", + "requestedChannelMax": "0", + "requestedFrameMax": "0", + "connectionTimeout": "60000", + "handshakeTimeout": "10000", + "requestedHeartbeat": "60", + "prefetchCount": "0", + "prefetchGlobal": "false", + "passive": "false" + } + + ``` * YAML - ```yaml - - configs: - host: "localhost" - port: 5672 - virtualHost: "/" - username: "guest" - password: "guest" - queueName: "test-queue" - connectionName: "test-connection" - requestedChannelMax: 0 - requestedFrameMax: 0 - connectionTimeout: 60000 - handshakeTimeout: 10000 - requestedHeartbeat: 60 - prefetchCount: 0 - prefetchGlobal: "false" - passive: "false" - - ``` + ```yaml + + configs: + host: "localhost" + port: 5672 + virtualHost: "/" + username: "guest" + password: "guest" + queueName: "test-queue" + connectionName: "test-connection" + requestedChannelMax: 0 + requestedFrameMax: 0 + connectionTimeout: 60000 + handshakeTimeout: 10000 + requestedHeartbeat: 60 + prefetchCount: 0 + prefetchGlobal: "false" + passive: "false" + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-redis-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-redis-sink.md index e030196e78c70..49eb119802736 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-redis-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-redis-sink.md @@ -1,7 +1,7 @@ --- id: io-redis-sink title: Redis sink connector -sidebar_label: Redis sink connector +sidebar_label: "Redis sink connector" original_id: io-redis-sink --- @@ -44,35 +44,35 @@ Before using the Redis sink connector, you need to create a configuration file t * JSON - ```json - - { - "redisHosts": "localhost:6379", - "redisPassword": "fake@123", - "redisDatabase": "1", - "clientMode": "Standalone", - "operationTimeout": "2000", - "batchSize": "100", - "batchTimeMs": "1000", - "connectTimeout": "3000" - } - - ``` + ```json + + { + "redisHosts": "localhost:6379", + "redisPassword": "fake@123", + "redisDatabase": "1", + "clientMode": "Standalone", + "operationTimeout": "2000", + "batchSize": "100", + "batchTimeMs": "1000", + "connectTimeout": "3000" + } + + ``` * YAML - ```yaml - - { - redisHosts: "localhost:6379" - redisPassword: "fake@123" - redisDatabase: 1 - clientMode: "Standalone" - operationTimeout: 2000 - batchSize: 100 - batchTimeMs: 1000 - connectTimeout: 3000 - } - - ``` + ```yaml + + { + redisHosts: "localhost:6379" + redisPassword: "fake@123" + redisDatabase: 1 + clientMode: "Standalone" + operationTimeout: 2000 + batchSize: 100 + batchTimeMs: 1000 + connectTimeout: 3000 + } + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-solr-sink.md b/site2/website-next/versioned_docs/version-2.8.0/io-solr-sink.md index 91ff0c9b6a1f7..d7b31ad97cde8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-solr-sink.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-solr-sink.md @@ -1,7 +1,7 @@ --- id: io-solr-sink title: Solr sink connector -sidebar_label: Solr sink connector +sidebar_label: "Solr sink connector" original_id: io-solr-sink --- @@ -39,31 +39,31 @@ Before using the Solr sink connector, you need to create a configuration file th * JSON - ```json - - { - "solrUrl": "localhost:2181,localhost:2182/chroot", - "solrMode": "SolrCloud", - "solrCollection": "techproducts", - "solrCommitWithinMs": 100, - "username": "fakeuser", - "password": "fake@123" - } - - ``` + ```json + + { + "solrUrl": "localhost:2181,localhost:2182/chroot", + "solrMode": "SolrCloud", + "solrCollection": "techproducts", + "solrCommitWithinMs": 100, + "username": "fakeuser", + "password": "fake@123" + } + + ``` * YAML - ```yaml - - { - solrUrl: "localhost:2181,localhost:2182/chroot" - solrMode: "SolrCloud" - solrCollection: "techproducts" - solrCommitWithinMs: 100 - username: "fakeuser" - password: "fake@123" - } - - ``` + ```yaml + + { + solrUrl: "localhost:2181,localhost:2182/chroot" + solrMode: "SolrCloud" + solrCollection: "techproducts" + solrCommitWithinMs: 100 + username: "fakeuser" + password: "fake@123" + } + + ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-twitter-source.md b/site2/website-next/versioned_docs/version-2.8.0/io-twitter-source.md index aa3affa5867df..101602e246fb8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-twitter-source.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-twitter-source.md @@ -1,7 +1,7 @@ --- id: io-twitter-source title: Twitter Firehose source connector -sidebar_label: Twitter Firehose source connector +sidebar_label: "Twitter Firehose source connector" original_id: io-twitter-source --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-twitter.md b/site2/website-next/versioned_docs/version-2.8.0/io-twitter.md index 11f583b329e3f..53f949863702d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-twitter.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-twitter.md @@ -1,7 +1,7 @@ --- id: io-twitter title: Twitter Firehose Connector -sidebar_label: Twitter Firehose Connector +sidebar_label: "Twitter Firehose Connector" original_id: io-twitter --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/io-use.md b/site2/website-next/versioned_docs/version-2.8.0/io-use.md index 1c98a98227d7a..43b7a30cadc33 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/io-use.md +++ b/site2/website-next/versioned_docs/version-2.8.0/io-use.md @@ -1,7 +1,7 @@ --- id: io-use title: How to use Pulsar connectors -sidebar_label: Use +sidebar_label: "Use" original_id: io-use --- @@ -17,7 +17,6 @@ Pulsar bundles several [builtin connectors](io-connectors) used to move data in :::note - When using a non-builtin connector, you need to specify the path of a archive file for the connector. ::: @@ -44,6 +43,7 @@ To configure a default folder for builtin connectors, set the `connectorsDirecto Set the `./connectors` folder as the default storage location for builtin connectors. ``` + ######################## # Connectors ######################## @@ -213,6 +213,7 @@ Create a source connector. Use the `create` subcommand. ``` + $ pulsar-admin sources create options ``` @@ -229,60 +230,60 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/source * Create a source connector with a **local file**. - ```java - - void createSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - - ``` + ```java + + void createSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sourceConfig` | The source configuration object + |Name|Description + |---|--- + `sourceConfig` | The source configuration object **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). + For more information, see [`createSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSource-SourceConfig-java.lang.String-). * Create a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - - void createSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - - ``` + ```java + + void createSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sourceConfig` | The source configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sourceConfig` | The source configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-). + For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#createSourceWithUrl-SourceConfig-java.lang.String-).
    @@ -314,6 +315,7 @@ Create a sink connector. Use the `create` subcommand. ``` + $ pulsar-admin sinks create options ``` @@ -329,62 +331,61 @@ Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/ * Create a sink connector with a **local file**. - - - ```java - - void createSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - ``` + ```java + + void createSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - |Name|Description - |---|--- - `sinkConfig` | The sink configuration object + |Name|Description + |---|--- + `sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). + For more information, see [`createSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSink-SinkConfig-java.lang.String-). * Create a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - - void createSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - - ``` + ```java + + void createSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `sinkConfig` | The sink configuration object - `pkgUrl` | URL from which pkg can be downloaded + Parameter| Description + |---|--- + `sinkConfig` | The sink configuration object + `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). + For more information, see [`createSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#createSinkWithUrl-SinkConfig-java.lang.String-). @@ -416,6 +417,7 @@ Start a source connector. Use the `start` subcommand. ``` + $ pulsar-admin sources start options ``` @@ -427,11 +429,11 @@ For more information, see [here](io-cli.md#start). * Start **all** source connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/start|operation/startSource?version=@pulsar:version_number@} * Start a **specified** source connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSource?version=@pulsar:version_number@}
    @@ -459,6 +461,7 @@ Start a sink connector. Use the `start` subcommand. ``` + $ pulsar-admin sinks start options ``` @@ -470,11 +473,11 @@ For more information, see [here](io-cli.md#start-1). * Start **all** sink connectors. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sources/:tenant/:namespace/:sinkName/start|operation/startSink?version=@pulsar:version_number@} * Start a **specified** sink connector. - Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@} + Send a `POST` request to this endpoint: {@inject: endpoint|POST|/admin/v3/sinks/:tenant/:namespace/:sourceName/:instanceId/start|operation/startSink?version=@pulsar:version_number@}
    @@ -502,6 +505,7 @@ Run a source connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sources localrun options ``` @@ -530,6 +534,7 @@ Run a sink connector locally. Use the `localrun` subcommand. ``` + $ pulsar-admin sinks localrun options ``` @@ -580,6 +585,7 @@ Get the information of a source connector. Use the `get` subcommand. ``` + $ pulsar-admin sources get options ``` @@ -630,6 +636,7 @@ This is a sourceConfig. This is a sourceConfig example. ``` + { "tenant": "public", "namespace": "default", @@ -702,6 +709,7 @@ Get the information of a sink connector. Use the `get` subcommand. ``` + $ pulsar-admin sinks get options ``` @@ -822,6 +830,7 @@ Get the list of all running source connectors. Use the `list` subcommand. ``` + $ pulsar-admin sources list options ``` @@ -846,7 +855,9 @@ List listSources(String tenant, **Response example** -```java ["f1", "f2", "f3"] +```java + +["f1", "f2", "f3"] ``` @@ -889,6 +900,7 @@ Get the list of all running sink connectors. Use the `list` subcommand. ``` + $ pulsar-admin sinks list options ``` @@ -913,7 +925,9 @@ List listSinks(String tenant, **Response example** -```java ["f1", "f2", "f3"] +```java + +["f1", "f2", "f3"] ``` @@ -960,6 +974,7 @@ Get the current status of a source connector. Use the `status` subcommand. ``` + $ pulsar-admin sources status options ``` @@ -982,59 +997,59 @@ For more information, see [here](io-cli.md#status). * Get the current status of **all** source connectors. - ```java - - SourceStatus getSourceStatus(String tenant, - String namespace, - String source) - throws PulsarAdminException - - ``` + ```java + + SourceStatus getSourceStatus(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSource-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - - SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, - String namespace, - String source, - int id) - throws PulsarAdminException - - ``` + ```java + + SourceStatus.SourceInstanceStatus.SourceInstanceStatusData getSourceStatus(String tenant, + String namespace, + String source, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Source instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Source instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSourceStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#getSourceStatus-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1066,6 +1081,7 @@ Get the current status of a Pulsar sink connector. Use the `status` subcommand. ``` + $ pulsar-admin sinks status options ``` @@ -1088,59 +1104,59 @@ For more information, see [here](io-cli.md#status-1). * Get the current status of **all** sink connectors. - ```java - - SinkStatus getSinkStatus(String tenant, - String namespace, - String sink) - throws PulsarAdminException - - ``` + ```java + + SinkStatus getSinkStatus(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`getSinkStatus`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-). * Gets the current status of a **specified** source connector. - ```java - - SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, - String namespace, - String sink, - int id) - throws PulsarAdminException - - ``` + ```java + + SinkStatus.SinkInstanceStatus.SinkInstanceStatusData getSinkStatus(String tenant, + String namespace, + String sink, + int id) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - Parameter| Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Source name - `id` | Sink instanceID + Parameter| Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Source name + `id` | Sink instanceID - **Exception** + **Exception** - Exception name | Description - |---|--- - `PulsarAdminException` | Unexpected error + Exception name | Description + |---|--- + `PulsarAdminException` | Unexpected error - For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`getSinkStatusWithInstanceID`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#getSinkStatus-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1178,6 +1194,7 @@ Update a running Pulsar source connector. Use the `update` subcommand. ``` + $ pulsar-admin sources update options ``` @@ -1194,62 +1211,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sources/ * Update a running source connector with a **local file**. - ```java - - void updateSource(SourceConfig sourceConfig, - String fileName) - throws PulsarAdminException - - ``` + ```java + + void updateSource(SourceConfig sourceConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sourceConfig` | The source configuration object + | Name | Description + |---|--- + |`sourceConfig` | The source configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). + For more information, see [`updateSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSource-SourceConfig-java.lang.String-). * Update a source connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - - void updateSourceWithUrl(SourceConfig sourceConfig, - String pkgUrl) - throws PulsarAdminException - - ``` + ```java + + void updateSourceWithUrl(SourceConfig sourceConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sourceConfig` | The source configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sourceConfig` | The source configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error For more information, see [`createSourceWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#updateSourceWithUrl-SourceConfig-java.lang.String-). @@ -1283,6 +1300,7 @@ Update a running Pulsar sink connector. Use the `update` subcommand. ``` + $ pulsar-admin sinks update options ``` @@ -1299,62 +1317,62 @@ Send a `PUT` request to this endpoint: {@inject: endpoint|PUT|/admin/v3/sinks/:t * Update a running sink connector with a **local file**. - ```java - - void updateSink(SinkConfig sinkConfig, - String fileName) - throws PulsarAdminException - - ``` + ```java + + void updateSink(SinkConfig sinkConfig, + String fileName) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - |`sinkConfig` | The sink configuration object + | Name | Description + |---|--- + |`sinkConfig` | The sink configuration object - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - | `PulsarAdminException.NotFoundException` | Cluster doesn't exist - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + | `PulsarAdminException.NotFoundException` | Cluster doesn't exist + | `PulsarAdminException` | Unexpected error - For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). + For more information, see [`updateSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSink-SinkConfig-java.lang.String-). * Update a sink connector using a **remote file** with a URL from which fun-pkg can be downloaded. - ```java - - void updateSinkWithUrl(SinkConfig sinkConfig, - String pkgUrl) - throws PulsarAdminException - - ``` + ```java + + void updateSinkWithUrl(SinkConfig sinkConfig, + String pkgUrl) + throws PulsarAdminException + + ``` - Supported URLs are `http` and `file`. + Supported URLs are `http` and `file`. - **Example** + **Example** - * HTTP: http://www.repo.com/fileName.jar + * HTTP: http://www.repo.com/fileName.jar - * File: file:///dir/fileName.jar + * File: file:///dir/fileName.jar - **Parameter** + **Parameter** - | Name | Description - |---|--- - | `sinkConfig` | The sink configuration object - | `pkgUrl` | URL from which pkg can be downloaded + | Name | Description + |---|--- + | `sinkConfig` | The sink configuration object + | `pkgUrl` | URL from which pkg can be downloaded - **Exception** + **Exception** - |Name|Description| - |---|--- - |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission - |`PulsarAdminException.NotFoundException` | Cluster doesn't exist - |`PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + |`PulsarAdminException.NotAuthorizedException`| You don't have the admin permission + |`PulsarAdminException.NotFoundException` | Cluster doesn't exist + |`PulsarAdminException` | Unexpected error For more information, see [`updateSinkWithUrl`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#updateSinkWithUrl-SinkConfig-java.lang.String-). @@ -1394,6 +1412,7 @@ Stop a source connector. Use the `stop` subcommand. ``` + $ pulsar-admin sources stop options ``` @@ -1416,59 +1435,59 @@ For more information, see [here](io-cli.md#stop). * Stop **all** source connectors. - ```java - - void stopSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** source connector. - ```java - - void stopSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - - ``` + ```java + + void stopSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#stopSource-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1500,6 +1519,7 @@ Stop a sink connector. Use the `stop` subcommand. ``` + $ pulsar-admin sinks stop options ``` @@ -1522,59 +1542,59 @@ For more information, see [here](io-cli.md#stop-1). * Stop **all** sink connectors. - ```java - - void stopSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-). * Stop a **specified** sink connector. - ```java - - void stopSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - - ``` + ```java + + void stopSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`stopSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#stopSink-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1612,6 +1632,7 @@ Restart a source connector. Use the `restart` subcommand. ``` + $ pulsar-admin sources restart options ``` @@ -1634,59 +1655,59 @@ For more information, see [here](io-cli.md#restart). * Restart **all** source connectors. - ```java - - void restartSource(String tenant, - String namespace, - String source) - throws PulsarAdminException - - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** source connector. - ```java - - void restartSource(String tenant, - String namespace, - String source, - int instanceId) - throws PulsarAdminException - - ``` + ```java + + void restartSource(String tenant, + String namespace, + String source, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Source instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Source instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSource`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Source.html#restartSource-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1718,6 +1739,7 @@ Restart a sink connector. Use the `restart` subcommand. ``` + $ pulsar-admin sinks restart options ``` @@ -1740,59 +1762,59 @@ For more information, see [here](io-cli.md#restart-1). * Restart all Pulsar sink connectors. - ```java - - void restartSink(String tenant, - String namespace, - String sink) - throws PulsarAdminException - - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `sink` | Sink name + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `sink` | Sink name - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-). * Restart a **specified** sink connector. - ```java - - void restartSink(String tenant, - String namespace, - String sink, - int instanceId) - throws PulsarAdminException - - ``` + ```java + + void restartSink(String tenant, + String namespace, + String sink, + int instanceId) + throws PulsarAdminException + + ``` - **Parameter** + **Parameter** - | Name | Description - |---|--- - `tenant` | Tenant name - `namespace` | Namespace name - `source` | Source name - `instanceId` | Sink instanceID + | Name | Description + |---|--- + `tenant` | Tenant name + `namespace` | Namespace name + `source` | Source name + `instanceId` | Sink instanceID - **Exception** + **Exception** - |Name|Description| - |---|--- - | `PulsarAdminException` | Unexpected error + |Name|Description| + |---|--- + | `PulsarAdminException` | Unexpected error - For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-). + For more information, see [`restartSink`](https://pulsar.apache.org/api/admin/org/apache/pulsar/client/admin/Sink.html#restartSink-java.lang.String-java.lang.String-java.lang.String-int-).
    @@ -1830,6 +1852,7 @@ Delete a source connector. Use the `delete` subcommand. ``` + $ pulsar-admin sources delete options ``` @@ -1906,6 +1929,7 @@ Delete a sink connector. Use the `delete` subcommand. ``` + $ pulsar-admin sinks delete options ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/performance-pulsar-perf.md b/site2/website-next/versioned_docs/version-2.8.0/performance-pulsar-perf.md index 9cedc027ea1c6..12e13f0069da2 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/performance-pulsar-perf.md +++ b/site2/website-next/versioned_docs/version-2.8.0/performance-pulsar-perf.md @@ -1,7 +1,7 @@ --- id: performance-pulsar-perf title: Pulsar Perf -sidebar_label: Pulsar Perf +sidebar_label: "Pulsar Perf" original_id: performance-pulsar-perf --- @@ -16,6 +16,7 @@ The Pulsar Perf is a built-in performance test tool for Apache Pulsar. You can u This example shows how the Pulsar Perf produces messages with default options. For all configuration options available for the `pulsar-perf produce` command, see [configuration options](#configuration-options-for-pulsar-perf-produce). ``` + bin/pulsar-perf produce my-topic ``` @@ -25,6 +26,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 19:53:31.459 [pulsar-perf-producer-exec-1-1] INFO org.apache.pulsar.testclient.PerformanceProducer - Created 1 producers 19:53:31.482 [pulsar-timer-5-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 19:53:40.861 [main] INFO org.apache.pulsar.testclient.PerformanceProducer - Throughput produced: 93.7 msg/s --- 0.7 Mbit/s --- failure 0.0 msg/s --- Latency: mean: 3.575 ms - med: 3.460 - 95pct: 4.790 - 99pct: 5.308 - 99.9pct: 5.834 - 99.99pct: 6.609 - Max: 6.609 @@ -97,6 +99,7 @@ The following table lists configuration options available for the `pulsar-perf p This example shows how the Pulsar Perf consumes messages with default options. ``` + bin/pulsar-perf consume my-topic ``` @@ -106,6 +109,7 @@ After the command is executed, the test data is continuously output on the Conso **Output** ``` + 20:35:37.071 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Start receiving from 1 consumers on 1 topics 20:35:41.150 [pulsar-client-io-1-9] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider 20:35:47.092 [main] INFO org.apache.pulsar.testclient.PerformanceConsumer - Throughput received: 59.572 msg/s -- 0.465 Mbit/s --- Latency: mean: 11.298 ms - med: 10 - 95pct: 15 - 99pct: 98 - 99.9pct: 137 - 99.99pct: 152 - Max: 152 @@ -168,6 +172,7 @@ By default, the Pulsar Perf uses `conf/client.conf` as the default configuration You can use the following commands to change the configuration file and the Log4j configuration file. ``` + export PULSAR_CLIENT_CONF= export PULSAR_LOG_CONF= @@ -176,6 +181,7 @@ export PULSAR_LOG_CONF= In addition, you can use the following command to configure the JVM configuration through environment variables: ``` + export PULSAR_EXTRA_OPTS='-Xms4g -Xmx4g -XX:MaxDirectMemorySize=4g' ``` @@ -188,32 +194,38 @@ To check test results through the HdrHistogram Plotter, follow these steps: 1. Clone the HdrHistogram repository from GitHub to the local. - ``` - git clone https://github.com/HdrHistogram/HdrHistogram.git - - ``` + ``` + + git clone https://github.com/HdrHistogram/HdrHistogram.git + + ``` 2. Switch to the HdrHistogram folder. - ``` - cd HdrHistogram + ``` + + cd HdrHistogram + + ``` - ``` 3. Install the HdrHistogram Plotter. - ``` - mvn clean install -DskipTests + ``` + + mvn clean install -DskipTests + + ``` - ``` 4. Transform the file generated by the Pulsar Perf. - ``` - ./HistogramLogProcessor -i -o - - ``` + ``` + + ./HistogramLogProcessor -i -o + + ``` 5. You will get two output files. Upload the output file with the filename extension of .hgrm to the [HdrHistogram Plotter](https://hdrhistogram.github.io/HdrHistogram/plotFiles.html). 6. Check the test result through the Graphical User Interface of the HdrHistogram Plotter, as shown blow. - ![](/assets/perf-produce.png) + ![](/assets/perf-produce.png) diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-cli-tools.md b/site2/website-next/versioned_docs/version-2.8.0/reference-cli-tools.md index 30a2d26a14a83..9a8d4dff70f3b 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-cli-tools.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-cli-tools.md @@ -1,7 +1,7 @@ --- id: reference-cli-tools title: Pulsar command-line tools -sidebar_label: Pulsar CLI tools +sidebar_label: "Pulsar CLI tools" original_id: reference-cli-tools --- @@ -22,9 +22,12 @@ All Pulsar command-line tools can be run from the `bin` directory of your [insta > ### Getting help > You can get help for any CLI tool, command, or subcommand using the `--help` flag, or `-h` for short. Here's an example: + > ```shell -> +> > $ bin/pulsar broker --help +> +> > ``` ## `pulsar` @@ -40,6 +43,7 @@ Usage: $ pulsar command ``` + Commands: * `bookie` * `broker` @@ -149,6 +153,7 @@ Usage $ pulsar compact-topic options ``` + Options |Flag|Description|Default| @@ -532,6 +537,7 @@ Run a consumer Usage ``` + $ pulsar-perf consume options ``` @@ -926,6 +932,7 @@ Usage $ broker-tool command ``` + Commands * `load-report` * `help` diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-configuration.md b/site2/website-next/versioned_docs/version-2.8.0/reference-configuration.md index 763a7bf2681d7..274016495fadf 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-configuration.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-configuration.md @@ -1,7 +1,7 @@ --- id: reference-configuration title: Pulsar configuration -sidebar_label: Pulsar configuration +sidebar_label: "Pulsar configuration" original_id: reference-configuration --- @@ -205,7 +205,8 @@ brokerServiceCompactionThresholdInBytes|If the estimated backlog size is greater |tlsTrustCertsFilePath| Path for the trusted TLS certificate file. This cert is used to verify that any certs presented by connecting clients are signed by a certificate authority. If this verification fails, then the certs are untrusted and the connections are dropped. || |tlsAllowInsecureConnection| Accept untrusted TLS certificate from client. If it is set to `true`, a client with a cert which cannot be verified with the 'tlsTrustCertsFilePath' cert will be allowed to connect to the server, though the cert will not be used for client authentication. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.3```, ```TLSv1.2``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + |tlsEnabledWithKeyStore| Enable TLS with KeyStore type configuration in broker |false| |tlsProvider| TLS Provider for KeyStore type || |tlsKeyStoreType| LS KeyStore type configuration in broker: JKS, PKCS12 |JKS| @@ -737,7 +738,8 @@ The [Pulsar proxy](concepts-architecture-overview.md#pulsar-proxy) can be config |tlsHostnameVerificationEnabled| Whether the hostname is validated when the proxy creates a TLS connection with brokers |false| |tlsRequireTrustedClientCertOnConnect| Whether client certificates are required for TLS. Connections are rejected if the client certificate isn’t trusted. |false| |tlsProtocols|Specify the tls protocols the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLSv1.3```, ```TLSv1.2``` || -|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ```TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256```|| +|tlsCiphers|Specify the tls cipher the broker will use to negotiate during TLS Handshake. Multiple values can be specified, separated by commas. Example:- ``` + | httpReverseProxyConfigs | HTTP directs to redirect to non-pulsar services | | | httpOutputBufferSize | HTTP output buffer size. The amount of data that will be buffered for HTTP requests before it is flushed to the channel. A larger buffer size may result in higher HTTP throughput though it may take longer for the client to see data. If using HTTP streaming via the reverse proxy, this should be set to the minimum value (1) so that clients see the data as soon as possible. | 32768 | | httpNumThreads | Number of threads to use for HTTP requests processing| 2 * Runtime.getRuntime().availableProcessors() | diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-connector-admin.md b/site2/website-next/versioned_docs/version-2.8.0/reference-connector-admin.md index c6cc98f9271d6..b995e7bb231ca 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-connector-admin.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-connector-admin.md @@ -1,7 +1,7 @@ --- id: reference-connector-admin title: Connector Admin CLI -sidebar_label: Connector Admin CLI +sidebar_label: "Connector Admin CLI" original_id: reference-connector-admin --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-metrics.md b/site2/website-next/versioned_docs/version-2.8.0/reference-metrics.md index 2e1301d2ec9b7..8d0a10fc9d818 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-metrics.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-metrics.md @@ -1,7 +1,7 @@ --- id: reference-metrics title: Pulsar Metrics -sidebar_label: Pulsar Metrics +sidebar_label: "Pulsar Metrics" original_id: reference-metrics --- @@ -25,12 +25,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -106,20 +106,21 @@ The following metrics are available for broker: - [Storage metrics](#storage-metrics) - [Broker](#broker) - [Namespace metrics](#namespace-metrics) - - [Replication metrics](#replication-metrics) + - [Replication metrics](#replication-metrics) - [Topic metrics](#topic-metrics) - - [Replication metrics](#replication-metrics-1) + - [Replication metrics](#replication-metrics-1) - [ManagedLedgerCache metrics](#managedledgercache-metrics) - [ManagedLedger metrics](#managedledger-metrics) - [LoadBalancing metrics](#loadbalancing-metrics) - - [BundleUnloading metrics](#bundleunloading-metrics) - - [BundleSplit metrics](#bundlesplit-metrics) + - [BundleUnloading metrics](#bundleunloading-metrics) + - [BundleSplit metrics](#bundlesplit-metrics) - [Subscription metrics](#subscription-metrics) - [Consumer metrics](#consumer-metrics) - [Managed ledger bookie client metrics](#managed-ledger-bookie-client-metrics) - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -275,7 +276,7 @@ All the cursor acknowledgment state metrics are labelled with the following labe Name |Type |Description |---|---|--- -brk_ml_cursor_persistLedgerSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to a ledger.| +brk_ml_cursor_persistLedgerSucceed(namespace=", ledger_name="", cursor_name:")|Gauge|The number of acknowledgment states that is persistent to a ledger.| brk_ml_cursor_persistLedgerErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to the ledger.| brk_ml_cursor_persistZookeeperSucceed(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of acknowledgment states that is persistent to ZooKeeper. brk_ml_cursor_persistZookeeperErrors(namespace="", ledger_name="", cursor_name:"")|Gauge|The number of ledger errors occurred when acknowledgment states fail to be persistent to ZooKeeper. @@ -285,20 +286,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -308,7 +309,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -419,6 +420,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -471,10 +501,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-pulsar-admin.md b/site2/website-next/versioned_docs/version-2.8.0/reference-pulsar-admin.md index 8fb8fad766712..7497ca812affa 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-pulsar-admin.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-pulsar-admin.md @@ -1,7 +1,7 @@ --- id: pulsar-admin title: Pulsar admin CLI -sidebar_label: Pulsar Admin CLI +sidebar_label: "Pulsar Admin CLI" original_id: pulsar-admin --- @@ -585,6 +585,7 @@ Create a Pulsar Function in cluster mode (i.e. deploy it on a Pulsar cluster) Usage ``` + $ pulsar-admin functions create options ``` @@ -2113,6 +2114,7 @@ Usage $ pulsar-admin namespaces get-max-subscriptions-per-topic tenant/namespace ``` + ### `remove-max-subscriptions-per-topic` Remove the maximum subscription per topic for a namespace. @@ -2333,6 +2335,7 @@ Run compaction on the specified topic (persistent topics only) Usage ``` + $ pulsar-admin topics compact persistent://tenant/namespace/topic ``` @@ -2393,13 +2396,9 @@ Create a partitioned topic. A partitioned topic must be created before producers :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2445,13 +2444,9 @@ Creates a non-partitioned topic. A non-partitioned topic must explicitly be crea :::note - By default, after 60 seconds of creation, topics are considered inactive and deleted automatically to prevent from generating trash data. - To disable this feature, set `brokerDeleteInactiveTopicsEnabled` to `false`. - To change the frequency of checking inactive topics, set `brokerDeleteInactiveTopicsFrequencySeconds` to your desired value. - For more information about these two parameters, see [here](reference-configuration.md#broker). ::: @@ -2508,6 +2503,7 @@ Get the list of topics under a namespace Usage ``` + $ pulsar-admin topics list tenant/cluster/namespace ``` @@ -2928,6 +2924,7 @@ Usage $ pulsar-admin topics remove-persistence tenant/namespace/topic ``` + ### `get-message-ttl` Get the message TTL for a topic. @@ -2962,7 +2959,7 @@ Usage ```bash -$ pulsar-admin topics remove-message-ttl tenant/namespace/topic +$ pulsar-admin topics remove-message-ttl tenant/namespace/topic ``` @@ -3005,14 +3002,12 @@ $ pulsar-admin topics remove-deduplication tenant/namespace/topic ``` -``` - ## `tenants` Operations for managing tenants Usage -``` +```bash $ pulsar-admin tenants subcommand @@ -3183,6 +3178,7 @@ Operations related to Schemas associated with Pulsar topics. Usage ``` + $ pulsar-admin schemas subcommand ``` diff --git a/site2/website-next/versioned_docs/version-2.8.0/reference-terminology.md b/site2/website-next/versioned_docs/version-2.8.0/reference-terminology.md index 3eb54b8116b3d..cce3a3f6dbf93 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/reference-terminology.md +++ b/site2/website-next/versioned_docs/version-2.8.0/reference-terminology.md @@ -1,7 +1,7 @@ --- id: reference-terminology title: Pulsar Terminology -sidebar_label: Terminology +sidebar_label: "Terminology" original_id: reference-terminology --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/schema-evolution-compatibility.md b/site2/website-next/versioned_docs/version-2.8.0/schema-evolution-compatibility.md index 60a436cb4b720..6e08e164a05ae 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/schema-evolution-compatibility.md +++ b/site2/website-next/versioned_docs/version-2.8.0/schema-evolution-compatibility.md @@ -1,7 +1,7 @@ --- id: schema-evolution-compatibility title: Schema evolution and compatibility -sidebar_label: Schema evolution and compatibility +sidebar_label: "Schema evolution and compatibility" original_id: schema-evolution-compatibility --- @@ -37,9 +37,9 @@ For more information, see [Schema compatibility check strategy](#schema-compatib 1. When a producer/consumer/reader connects to a broker, the broker deploys the schema compatibility checker configured by `schemaRegistryCompatibilityCheckers` to enforce schema compatibility check. - The schema compatibility checker is one instance per schema type. - - Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. + The schema compatibility checker is one instance per schema type. + + Currently, Avro and JSON have their own compatibility checkers, while all the other schema types share the default compatibility checker which disables schema evolution. 2. The producer/consumer/reader sends its client `SchemaInfo` to the broker. @@ -59,43 +59,43 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t | --- | --- | --- | --- | --- | | `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | All changes are allowed | All previous versions | Any order | | `ALWAYS_INCOMPATIBLE` | Disable schema evolution. | All changes are disabled | None | None | -| `BACKWARD` | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. | * Add optional fields * Delete fields | Latest version | Consumers | -| `BACKWARD_TRANSITIVE` | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. | * Add optional fields * Delete fields | All previous versions | Consumers | -| `FORWARD` | Consumers using the schema V3 or V2 can process data written by producers using the schema V3. | * Add fields * Delete optional fields | Latest version | Producers | -| `FORWARD_TRANSITIVE` | Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. | * Add fields * Delete optional fields | All previous versions | Producers | -| `FULL` | Backward and forward compatible between the schema V3 and V2. | * Modify optional fields | Latest version | Any order | -| `FULL_TRANSITIVE` | Backward and forward compatible among the schema V3, V2, and V1. | * Modify optional fields | All previous versions | Any order | +| `BACKWARD` | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. |
  • Add optional fields
  • Delete fields
  • | Latest version | Consumers | +| `BACKWARD_TRANSITIVE` | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. |
  • Add optional fields
  • Delete fields
  • | All previous versions | Consumers | +| `FORWARD` | Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | Latest version | Producers | +| `FORWARD_TRANSITIVE` | Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. |
  • Add fields
  • Delete optional fields
  • | All previous versions | Producers | +| `FULL` | Backward and forward compatible between the schema V3 and V2. |
  • Modify optional fields
  • | Latest version | Any order | +| `FULL_TRANSITIVE` | Backward and forward compatible among the schema V3, V2, and V1. |
  • Modify optional fields
  • | All previous versions | Any order | ### ALWAYS_COMPATIBLE and ALWAYS_INCOMPATIBLE | Compatibility check strategy | Definition | Note | | --- | --- | --- | | `ALWAYS_COMPATIBLE` | Disable schema compatibility check. | None | -| `ALWAYS_INCOMPATIBLE` | Disable schema evolution, that is, any schema change is rejected. | * For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. * For Avro and JSON, the default schema compatibility check strategy is `FULL`. | +| `ALWAYS_INCOMPATIBLE` | Disable schema evolution, that is, any schema change is rejected. |
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • | #### Example * Example 1 - In some situations, an application needs to store events of several different types in the same Pulsar topic. + In some situations, an application needs to store events of several different types in the same Pulsar topic. - In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. + In particular, when developing a data model in an `Event Sourcing` style, you might have several kinds of events that affect the state of an entity. - For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. + For example, for a user entity, there are `userCreated`, `userAddressChanged` and `userEnquiryReceived` events. The application requires that those events are always read in the same order. - Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. + Consequently, those events need to go in the same Pulsar partition to maintain order. This application can use `ALWAYS_COMPATIBLE` to allow different kinds of events co-exist in the same topic. * Example 2 - Sometimes we also make incompatible changes. + Sometimes we also make incompatible changes. - For example, you are modifying a field type from `string` to `int`. + For example, you are modifying a field type from `string` to `int`. - In this case, you need to: + In this case, you need to: - * Upgrade all producers and consumers to the new schema versions at the same time. + * Upgrade all producers and consumers to the new schema versions at the same time. - * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. + * Optionally, create a new topic and start migrating applications to use the new topic and the new schema, avoiding the need to handle two incompatible versions in the same topic. ### BACKWARD and BACKWARD_TRANSITIVE @@ -110,15 +110,15 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t * Example 1 - Remove a field. - - A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. + Remove a field. + + A consumer constructed to process events without one field can process events written with the old schema containing the field, and the consumer will ignore that field. * Example 2 - You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. + You want to load all Pulsar data into a Hive data warehouse and run SQL queries against the data. - Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. + Same SQL queries must continue to work even the data is changed. To support it, you can evolve the schemas using the `BACKWARD` strategy. ### FORWARD and FORWARD_TRANSITIVE @@ -151,7 +151,7 @@ Suppose that you have a topic containing three schemas (V1, V2, and V3), V1 is t | Compatibility check strategy | Definition | Description | Note | | --- | --- | --- | --- | -| `FULL` | Schemas are both backward and forward compatible, which means: Consumers using the last schema can process data written by producers using the new schema. AND Consumers using the new schema can process data written by producers using the last schema. | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. AND Consumers using the schema V3 or V2 can process data written by producers using the schema V3. | * For Avro and JSON, the default schema compatibility check strategy is `FULL`. * For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`. | +| `FULL` | Schemas are both backward and forward compatible, which means: Consumers using the last schema can process data written by producers using the new schema. AND Consumers using the new schema can process data written by producers using the last schema. | Consumers using the schema V3 can process data written by producers using the schema V3 or V2. AND Consumers using the schema V3 or V2 can process data written by producers using the schema V3. |
  • For Avro and JSON, the default schema compatibility check strategy is `FULL`.
  • For all schema types except Avro and JSON, the default schema compatibility check strategy is `ALWAYS_INCOMPATIBLE`.
  • | | `FULL_TRANSITIVE` | The new schema is backward and forward compatible with all previously registered schemas. | Consumers using the schema V3 can process data written by producers using the schema V3, V2 or V1. AND Consumers using the schema V3, V2 or V1 can process data written by producers using the schema V3. | None | #### Example @@ -168,10 +168,10 @@ When a producer tries to connect to a topic (suppose ignore the schema auto crea * Check if the schema carried by the producer exists in the schema registry or not. - * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. - - * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. - + * If the schema is already registered, then the producer is connected to a broker and produce messages with that schema. + + * If the schema is not registered, then Pulsar verifies if the schema is allowed to be registered based on the configured compatibility check strategy. + ### Consumer When a consumer tries to connect to a topic, a broker checks if a carried schema is compatible with a registered schema based on the configured schema compatibility check strategy. @@ -196,9 +196,9 @@ For example, the producers using schemas to write data to Pulsar and the consume | --- | --- | --- | | `ALWAYS_COMPATIBLE` | Any order | The compatibility check is disabled. Consequently, you can upgrade the producers and consumers in **any order**. | | `ALWAYS_INCOMPATIBLE` | None | The schema evolution is disabled. | -| * `BACKWARD` * `BACKWARD_TRANSITIVE` | Consumers | There is no guarantee that consumers using the old schema can read data produced using the new schema. Consequently, **upgrade all consumers first**, and then start producing new data. | -| * `FORWARD` * `FORWARD_TRANSITIVE` | Producers | There is no guarantee that consumers using the new schema can read data produced using the old schema. Consequently, **upgrade all producers first** to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers. | -| * `FULL` * `FULL_TRANSITIVE` | Any order | There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. Consequently, you can upgrade the producers and consumers in **any order**. | +|
  • `BACKWARD`
  • `BACKWARD_TRANSITIVE`
  • | Consumers | There is no guarantee that consumers using the old schema can read data produced using the new schema. Consequently, **upgrade all consumers first**, and then start producing new data. | +|
  • `FORWARD`
  • `FORWARD_TRANSITIVE`
  • | Producers | There is no guarantee that consumers using the new schema can read data produced using the old schema. Consequently, **upgrade all producers first**
  • to use the new schema and ensure that the data already produced using the old schemas are not available to consumers, and then upgrade the consumers.
  • | +|
  • `FULL`
  • `FULL_TRANSITIVE`
  • | Any order | There is no guarantee that consumers using the old schema can read data produced using the new schema and consumers using the new schema can read data produced using the old schema. Consequently, you can upgrade the producers and consumers in **any order**. | diff --git a/site2/website-next/versioned_docs/version-2.8.0/schema-get-started.md b/site2/website-next/versioned_docs/version-2.8.0/schema-get-started.md index 496fe9077e491..97529fbb125ca 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/schema-get-started.md +++ b/site2/website-next/versioned_docs/version-2.8.0/schema-get-started.md @@ -1,7 +1,7 @@ --- id: schema-get-started title: Get started -sidebar_label: Get started +sidebar_label: "Get started" original_id: schema-get-started --- @@ -56,6 +56,7 @@ Pulsar schema enables you to use language-specific types of data when constructi You can use the _User_ class to define the messages sent to Pulsar topics. ``` + public class User { String name; int age; @@ -72,6 +73,7 @@ If you construct a producer without specifying a schema, then the producer can o **Example** ``` + Producer producer = client.newProducer() .topic(topic) .create(); @@ -80,6 +82,7 @@ byte[] message = … // serialize the `user` by yourself; producer.send(message); ``` + ### With schema If you construct a producer with specifying a schema, then you can send a class to a topic directly without worrying about how to serialize POJOs into bytes. @@ -89,6 +92,7 @@ If you construct a producer with specifying a schema, then you can send a class This example constructs a producer with the _JSONSchema_, and you can send the _User_ class to topics directly without worrying about how to serialize it into bytes. ``` + Producer producer = client.newProducer(JSONSchema.of(User.class)) .topic(topic) .create(); diff --git a/site2/website-next/versioned_docs/version-2.8.0/schema-manage.md b/site2/website-next/versioned_docs/version-2.8.0/schema-manage.md index aa42485736939..b8377c0e6f4fb 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/schema-manage.md +++ b/site2/website-next/versioned_docs/version-2.8.0/schema-manage.md @@ -1,7 +1,7 @@ --- id: schema-manage title: Manage schema -sidebar_label: Manage schema +sidebar_label: "Manage schema" original_id: schema-manage --- @@ -35,25 +35,25 @@ For a producer, the `AutoUpdate` happens in the following cases: * If a **producer doesn’t carry a schema**: - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. - - * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **disabled** in the namespace to which the topic belongs, the producer is allowed to connect to the topic and produce data. + + * If `isSchemaValidationEnforced` or `schemaValidationEnforced` is **enabled** in the namespace to which the topic belongs, the producer is rejected and disconnected. * If a **producer carries a schema**: - A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. - - * If the schema is registered, a producer is connected to a broker. - - * If the schema is not registered: - - * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. - - * If `isAllowAutoUpdateSchema` sets to **true**: - - * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. - - * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. + A broker performs the compatibility check based on the configured compatibility check strategy of the namespace to which the topic belongs. + + * If the schema is registered, a producer is connected to a broker. + + * If the schema is not registered: + + * If `isAllowAutoUpdateSchema` sets to **false**, the producer is rejected to connect to a broker. + + * If `isAllowAutoUpdateSchema` sets to **true**: + + * If the schema passes the compatibility check, then the broker registers a new schema automatically for the topic and the producer is connected. + + * If the schema does not pass the compatibility check, then the broker does not register a schema and the producer is rejected to connect to a broker. ![AutoUpdate Producer](/assets/schema-producer.png) @@ -65,18 +65,18 @@ For a consumer, the `AutoUpdate` happens in the following cases: * If a **consumer connects to a topic with a schema**. - * If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - - * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. - + * If a topic does not have all of them (a schema/data/a local consumer and a local producer): + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + + * If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + ![AutoUpdate Consumer](/assets/schema-consumer.png) @@ -158,9 +158,9 @@ To manage schemas, you can use one of the following methods. | Method | Description | | --- | --- | -| **Admin CLI** | You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin). | -| **REST API** | Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). | -| **Java Admin API** | Pulsar provides Java admin library. | +| **Admin CLI**
  • | You can use the `pulsar-admin` tool to manage Pulsar schemas, brokers, clusters, sources, sinks, topics, tenants and so on. For more information about how to use the `pulsar-admin` tool, see [here](reference-pulsar-admin). | +| **REST API**
  • | Pulsar exposes schema related management API in Pulsar’s admin RESTful API. You can access the admin RESTful endpoint directly to manage schemas. For more information about how to use the Pulsar REST API, see [here](http://pulsar.apache.org/admin-rest-api/). | +| **Java Admin API**
  • | Pulsar provides Java admin library. | ### Upload a schema @@ -210,7 +210,7 @@ The `schema-definition-file` includes the following fields: | Field | Description | | --- | --- | | `type` | The schema type. | -| `schema` | The schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this field should be blank. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | | `properties` | The additional properties associated with the schema. | Here are examples of the `schema-definition-file` for a JSON schema. @@ -263,7 +263,7 @@ The post payload includes the following fields: | Field | Description | | --- | --- | | `type` | The schema type. | -| `schema` | The schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this field should be blank. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | | `properties` | The additional properties associated with the schema. |
    @@ -280,7 +280,7 @@ The `PostSchemaPayload` includes the following fields: | Field | Description | | --- | --- | | `type` | The schema type. | -| `schema` | The schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this field should be blank. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. | +| `schema` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | | `properties` | The additional properties associated with the schema. | Here is an example of `PostSchemaPayload`: @@ -293,9 +293,10 @@ PostSchemaPayload payload = new PostSchemaPayload(); payload.setType("INT8"); payload.setSchema(""); -admin.createSchema("my-tenant/my-ns/my-topic", payload); +admin.createSchema("my-tenant/my-ns/my-topic", payload); ``` +
    @@ -368,7 +369,7 @@ The response includes the following fields: | `version` | The schema version, which is a long number. | | `type` | The schema type. | | `timestamp` | The timestamp of creating this version of schema. | -| `data` | The schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this field should be blank. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | | `properties` | The additional properties associated with the schema. |
    @@ -386,7 +387,7 @@ The `SchemaInfo` includes the following fields: | --- | --- | | `name` | The schema name. | | `type` | The schema type. | -| `schema` | A byte array of the schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this byte array should be empty. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | | `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: @@ -395,7 +396,7 @@ Here is an example of `SchemaInfo`: PulsarAdmin admin = …; -SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); +SchemaInfo si = admin.getSchema("my-tenant/my-ns/my-topic"); ``` @@ -430,7 +431,7 @@ Use the `get` subcommand. ```bash -$ pulsar-admin schemas get --version= +$ pulsar-admin schemas get --version= ``` @@ -460,7 +461,7 @@ The response includes the following fields: | `version` | The schema version, which is a long number. | | `type` | The schema type. | | `timestamp` | The timestamp of creating this version of schema. | -| `data` | The schema definition data, which is encoded in UTF 8 charset. * If the schema is a **primitive** schema, this field should be blank. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition. | +| `data` | The schema definition data, which is encoded in UTF 8 charset.
  • If the schema is a
  • **primitive**
  • schema, this field should be blank.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition.
  • | | `properties` | The additional properties associated with the schema. |
    @@ -478,7 +479,7 @@ The `SchemaInfo` includes the following fields: | --- | --- | | `name` | The schema name. | | `type` | The schema type. | -| `schema` | A byte array of the schema definition data, which is encoded in UTF 8. * If the schema is a **primitive** schema, this byte array should be empty. * If the schema is a **struct** schema, this field should be a JSON string of the Avro schema definition converted to a byte array. | +| `schema` | A byte array of the schema definition data, which is encoded in UTF 8.
  • If the schema is a
  • **primitive**
  • schema, this byte array should be empty.
  • If the schema is a
  • **struct**
  • schema, this field should be a JSON string of the Avro schema definition converted to a byte array.
  • | | `properties` | The additional properties associated with the schema. | Here is an example of `SchemaInfo`: @@ -528,7 +529,6 @@ To delete a schema for a topic, you can use one of the following methods. :::note - In any case, the **delete** action deletes **all versions** of a schema registered for a topic. ::: @@ -596,7 +596,7 @@ Here is an example of deleting a schema. PulsarAdmin admin = …; -admin.deleteSchema("my-tenant/my-ns/my-topic"); +admin.deleteSchema("my-tenant/my-ns/my-topic"); ``` @@ -648,7 +648,6 @@ public interface SchemaStorage { :::tip - For a complete example of **schema storage** implementation, see [BookKeeperSchemaStorage](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorage.java) class. ::: @@ -668,7 +667,6 @@ public interface SchemaStorageFactory { :::tip - For a complete example of **schema storage factory** implementation, see [BookKeeperSchemaStorageFactory](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/BookkeeperSchemaStorageFactory.java) class. ::: diff --git a/site2/website-next/versioned_docs/version-2.8.0/schema-understand.md b/site2/website-next/versioned_docs/version-2.8.0/schema-understand.md index cb54d1cf44dcc..b87d21d5d5633 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/schema-understand.md +++ b/site2/website-next/versioned_docs/version-2.8.0/schema-understand.md @@ -1,7 +1,7 @@ --- id: schema-understand title: Understand schema -sidebar_label: Understand schema +sidebar_label: "Understand schema" original_id: schema-understand --- @@ -22,7 +22,7 @@ A `SchemaInfo` consists of the following fields: | Field | Description | | --- | --- | | `name` | Schema name (a string). | -| `type` | Schema type, which determines how to interpret the schema data. * Predefined schema: see [here](schema-understand.md#schema-type). * Customized schema: it is left as an empty string. | +| `type` | Schema type, which determines how to interpret the schema data.
  • Predefined schema: see [here](schema-understand.md#schema-type).
  • Customized schema: it is left as an empty string.
  • | | `schema`(`payload`) | Schema data, which is a sequence of 8-bit unsigned bytes and schema-type specific. | | `properties` | It is a user defined properties as a string/string map. Applications can use this bag for carrying any application specific logics. Possible properties might be the Git hash associated with the schema, an environment string like `dev` or `prod`. | @@ -101,21 +101,21 @@ This example demonstrates how to use a string schema. 1. Create a producer with a string schema and send messages. - ```java - - Producer producer = client.newProducer(Schema.STRING).create(); - producer.newMessage().value("Hello Pulsar!").send(); - - ``` + ```java + + Producer producer = client.newProducer(Schema.STRING).create(); + producer.newMessage().value("Hello Pulsar!").send(); + + ``` 2. Create a consumer with a string schema and receive messages. - ```java - - Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); - consumer.receive(); - - ``` + ```java + + Consumer consumer = client.newConsumer(Schema.STRING).subscribe(); + consumer.receive(); + + ``` ### Complex type @@ -168,72 +168,72 @@ This example shows how to construct a key/value schema and then use it to produc 1. Construct a key/value schema with `INLINE` encoding type. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.INLINE - ); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.INLINE + ); + + ``` 2. Optionally, construct a key/value schema with `SEPARATED` encoding type. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + ``` 3. Produce messages using a key/value schema. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Producer> producer = client.newProducer(kvSchema) - .topic(TOPIC) - .create(); - - final int key = 100; - final String value = "value-100"; - - // send the key/value message - producer.newMessage() - .value(new KeyValue(key, value)) - .send(); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Producer> producer = client.newProducer(kvSchema) + .topic(TOPIC) + .create(); + + final int key = 100; + final String value = "value-100"; + + // send the key/value message + producer.newMessage() + .value(new KeyValue(key, value)) + .send(); + + ``` 4. Consume messages using a key/value schema. - ```java - - Schema> kvSchema = Schema.KeyValue( - Schema.INT32, - Schema.STRING, - KeyValueEncodingType.SEPARATED - ); - - Consumer> consumer = client.newConsumer(kvSchema) - ... - .topic(TOPIC) - .subscriptionName(SubscriptionName).subscribe(); - - // receive key/value pair - Message> msg = consumer.receive(); - KeyValue kv = msg.getValue(); - - ``` + ```java + + Schema> kvSchema = Schema.KeyValue( + Schema.INT32, + Schema.STRING, + KeyValueEncodingType.SEPARATED + ); + + Consumer> consumer = client.newConsumer(kvSchema) + ... + .topic(TOPIC) + .subscriptionName(SubscriptionName).subscribe(); + + // receive key/value pair + Message> msg = consumer.receive(); + KeyValue kv = msg.getValue(); + + ```
    @@ -289,35 +289,35 @@ Pulsar gets the schema definition from the predefined `struct` using an Avro lib 1. Create the _User_ class to define the messages sent to Pulsar topics. - ```java - - @Builder - @AllArgsConstructor - @NoArgsConstructor - public static class User { - String name; - int age; - } - - ``` + ```java + + @Builder + @AllArgsConstructor + @NoArgsConstructor + public static class User { + String name; + int age; + } + + ``` 2. Create a producer with a `struct` schema and send messages. - ```java - - Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); - producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); - - ``` + ```java + + Producer producer = client.newProducer(Schema.AVRO(User.class)).create(); + producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); + + ``` 3. Create a consumer with a `struct` schema and receive messages - ```java - - Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); - User user = consumer.receive(); - - ``` + ```java + + Consumer consumer = client.newConsumer(Schema.AVRO(User.class)).subscribe(); + User user = consumer.receive(); + + ```
    @@ -330,25 +330,25 @@ You can define the `struct` schema using the `GenericSchemaBuilder`, generate a 1. Use `RecordSchemaBuilder` to build a schema. - ```java - - RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); - recordSchemaBuilder.field("intField").type(SchemaType.INT32); - SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - - Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + ```java + + RecordSchemaBuilder recordSchemaBuilder = SchemaBuilder.record("schemaName"); + recordSchemaBuilder.field("intField").type(SchemaType.INT32); + SchemaInfo schemaInfo = recordSchemaBuilder.build(SchemaType.AVRO); - ``` + Producer producer = client.newProducer(Schema.generic(schemaInfo)).create(); + + ``` 2. Use `RecordBuilder` to build the struct records. - ```java - - producer.newMessage().value(schema.newRecordBuilder() - .set("intField", 32) - .build()).send(); - - ``` + ```java + + producer.newMessage().value(schema.newRecordBuilder() + .set("intField", 32) + .build()).send(); + + ``` @@ -359,37 +359,37 @@ You can define the `schemaDefinition` to generate a `struct` schema. 1. Create the _User_ class to define the messages sent to Pulsar topics. - ```java - - @Builder - @AllArgsConstructor - @NoArgsConstructor - public static class User { - String name; - int age; - } - - ``` + ```java + + @Builder + @AllArgsConstructor + @NoArgsConstructor + public static class User { + String name; + int age; + } + + ``` 2. Create a producer with a `SchemaDefinition` and send messages. - ```java - - SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); - Producer producer = client.newProducer(Schema.AVRO(schemaDefinition)).create(); - producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); - - ``` + ```java + + SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); + Producer producer = client.newProducer(Schema.AVRO(schemaDefinition)).create(); + producer.newMessage().value(User.builder().name("pulsar-user").age(1).build()).send(); + + ``` 3. Create a consumer with a `SchemaDefinition` schema and receive messages - ```java - - SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); - Consumer consumer = client.newConsumer(Schema.AVRO(schemaDefinition)).subscribe(); - User user = consumer.receive().getValue(); - - ``` + ```java + + SchemaDefinition schemaDefinition = SchemaDefinition.builder().withPojo(User.class).build(); + Consumer consumer = client.newConsumer(Schema.AVRO(schemaDefinition)).subscribe(); + User user = consumer.receive().getValue(); + + ``` @@ -494,8 +494,8 @@ The table below lists the possible scenarios when this connection attempt occurs | Scenario | What happens | | --- | --- | -| * No schema exists for the topic. | (1) The producer is created using the given schema. (2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. (3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. | -| * A schema already exists. * The producer connects using the same schema that is already stored. | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible. (3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. | * A schema already exists. * The producer connects using a new schema that is compatible. | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). | +|
  • No schema exists for the topic.
  • | (1) The producer is created using the given schema. (2) Since no existing schema is compatible with the `SensorReading` schema, the schema is transmitted to the broker and stored. (3) Any consumer created using the same schema or topic can consume messages from the `sensor-data` topic. | +|
  • A schema already exists.
  • The producer connects using the same schema that is already stored.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible. (3) The broker attempts to store the schema in [BookKeeper](concepts-architecture-overview.md#persistent-storage) but then determines that it's already stored, so it is used to tag produced messages. |
  • A schema already exists.
  • The producer connects using a new schema that is compatible.
  • | (1) The schema is transmitted to the broker. (2) The broker determines that the schema is compatible and stores the new schema as the current version (with a new version number). | ## How does schema work @@ -511,9 +511,9 @@ This diagram illustrates how does schema work on the Producer side. 1. The application uses a schema instance to construct a producer instance. - The schema instance defines the schema for the data being produced using the producer instance. + The schema instance defines the schema for the data being produced using the producer instance. - Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. + Take AVRO as an example, Pulsar extract schema definition from the POJO class and construct the `SchemaInfo` that the producer needs to pass to a broker when it connects. 2. The producer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. @@ -537,7 +537,7 @@ For how to set `isAllowAutoUpdateSchema` via Pulsar admin API, see [Manage AutoU * If the schema is compatible, the broker stores it and returns the schema version to the producer. - All the messages produced by this producer are tagged with the schema version. + All the messages produced by this producer are tagged with the schema version. * If the schema is incompatible, the broker rejects it. @@ -549,24 +549,24 @@ This diagram illustrates how does Schema work on the consumer side. 1. The application uses a schema instance to construct a consumer instance. - The schema instance defines the schema that the consumer uses for decoding messages received from a broker. + The schema instance defines the schema that the consumer uses for decoding messages received from a broker. 2. The consumer connects to the broker with the `SchemaInfo` extracted from the passed-in schema instance. 3. The broker determines whether the topic has one of them (a schema/data/a local consumer and a local producer). 4. If a topic does not have all of them (a schema/data/a local consumer and a local producer): - - * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. - - * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. - + + * If `isAllowAutoUpdateSchema` sets to **true**, then the consumer registers a schema and it is connected to a broker. + + * If `isAllowAutoUpdateSchema` sets to **false**, then the consumer is rejected to connect to a broker. + 5. If a topic has one of them (a schema/data/a local consumer and a local producer), then the schema compatibility check is performed. - - * If the schema passes the compatibility check, then the consumer is connected to the broker. - - * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. + + * If the schema passes the compatibility check, then the consumer is connected to the broker. + + * If the schema does not pass the compatibility check, then the consumer is rejected to connect to the broker. 6. The consumer receives messages from the broker. - If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. + If the schema used by the consumer supports schema versioning (for example, AVRO schema), the consumer fetches the `SchemaInfo` of the version tagged in messages and uses the passed-in schema and the schema tagged in messages to decode the messages. diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-athenz.md b/site2/website-next/versioned_docs/version-2.8.0/security-athenz.md index 78945e604fe29..c4f79cb7f229c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-athenz.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-athenz.md @@ -1,7 +1,7 @@ --- id: security-athenz title: Authentication using Athenz -sidebar_label: Authentication using Athenz +sidebar_label: "Authentication using Athenz" original_id: security-athenz --- @@ -99,3 +99,4 @@ tlsAllowInsecureConnection=false tlsTrustCertsFilePath=/path/to/cacert.pem ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-authorization.md b/site2/website-next/versioned_docs/version-2.8.0/security-authorization.md index 937842e82357c..9ca30ea1dd999 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-authorization.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-authorization.md @@ -1,7 +1,7 @@ --- id: security-authorization title: Authentication and authorization in Pulsar -sidebar_label: Authorization and ACLs +sidebar_label: "Authorization and ACLs" original_id: security-authorization --- @@ -50,7 +50,7 @@ Another approach is to make the proxy role a superuser. This allows the proxy to You can specify the roles as proxy roles in [`conf/broker.conf`](reference-configuration.md#broker). -```properties +``` proxyRoles=my-proxy-role @@ -69,7 +69,7 @@ You can manage tenants using the [`pulsar-admin`](reference-pulsar-admin) tool. The following is an example tenant creation command: -```shell +``` $ bin/pulsar-admin tenants create my-tenant \ --admin-roles my-admin-role \ @@ -83,7 +83,7 @@ A client that successfully identifies itself as having the role `my-admin-role` The structure of topic names in Pulsar reflects the hierarchy between tenants, clusters, and namespaces: -```shell +``` persistent://tenant/namespace/topic @@ -95,7 +95,7 @@ You can use [Pulsar Admin Tools](admin-api-permissions) for managing permission ### Pulsar admin authentication -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("http://broker:8080") @@ -106,7 +106,7 @@ PulsarAdmin admin = PulsarAdmin.builder() To use TLS: -```java +``` PulsarAdmin admin = PulsarAdmin.builder() .serviceHttpUrl("https://broker:8080") @@ -115,3 +115,4 @@ PulsarAdmin admin = PulsarAdmin.builder() .build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-bouncy-castle.md b/site2/website-next/versioned_docs/version-2.8.0/security-bouncy-castle.md index d0469bd7366e9..b9bb8155bab83 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-bouncy-castle.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-bouncy-castle.md @@ -1,7 +1,7 @@ --- id: security-bouncy-castle title: Bouncy Castle Providers -sidebar_label: Bouncy Castle Providers +sidebar_label: "Bouncy Castle Providers" original_id: security-bouncy-castle --- @@ -35,11 +35,13 @@ Usually, You will meet error like `java.lang.SecurityException: Invalid signatur You could exclude these signatures in mvn pom file to avoid above error, by ```access transformers + META-INF/*.SF META-INF/*.DSA META-INF/*.RSA ``` + But it can also lead to new, cryptic errors, e.g. `java.security.NoSuchAlgorithmException: PBEWithSHA256And256BitAES-CBC-BC SecretKeyFactory not available` By explicitly specifying where to find the algorithm like this: `SecretKeyFactory.getInstance("PBEWithSHA256And256BitAES-CBC-BC","BC")` It will get the real error: `java.security.NoSuchProviderException: JCE cannot authenticate the provider BC` @@ -153,6 +155,7 @@ If you want to switch from BC-non-FIPS to BC-FIPS version, Here is an example fo ``` + For more example, you can reference module `bcfips-include-test`. diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-encryption.md b/site2/website-next/versioned_docs/version-2.8.0/security-encryption.md index 7e45c08991360..b0f2f4616b649 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-encryption.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-encryption.md @@ -1,7 +1,7 @@ --- id: security-encryption title: Pulsar Encryption -sidebar_label: End-to-End Encryption +sidebar_label: "End-to-End Encryption" original_id: security-encryption --- @@ -103,6 +103,7 @@ producer.close(); pulsarClient.close(); ``` + 7. Sample Consumer Application: ```java @@ -180,6 +181,7 @@ If you need to encrypt the messages using 2 keys (myapp.messagekey1 and myapp.me PulsarClient.newProducer().addEncryptionKey("myapp.messagekey1").addEncryptionKey("myapp.messagekey2"); ``` + ## Decrypt encrypted messages at the consumer application Consumers require access one of the private keys to decrypt messages that the producer produces. If you want to receive encrypted messages, create a public or private key and give your public key to the producer application to encrypt messages using your public key. diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-extending.md b/site2/website-next/versioned_docs/version-2.8.0/security-extending.md index a89ada98fb905..57128da01c25d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-extending.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-extending.md @@ -1,7 +1,7 @@ --- id: security-extending title: Extending Authentication and Authorization in Pulsar -sidebar_label: Extending +sidebar_label: "Extending" original_id: security-extending --- @@ -55,6 +55,7 @@ In `conf/broker.conf` you can choose to specify a list of valid providers: authenticationProviders= ``` + To implement `org.apache.pulsar.broker.authentication.AuthenticationProvider` on one single interface: ```java @@ -110,10 +111,10 @@ the Authorization plugin is designed only for use on the Broker however the Prox To provide a custom provider, you need to implement the `org.apache.pulsar.broker.authorization.AuthorizationProvider` interface, put this class in the Pulsar broker classpath and configure the class in `conf/broker.conf`: ```properties - + # Authorization provider fully qualified class-name authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - + ``` ```java @@ -207,3 +208,4 @@ public interface AuthorizationProvider extends Closeable { } ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-jwt.md b/site2/website-next/versioned_docs/version-2.8.0/security-jwt.md index 54a2a396e299c..84916c590993a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-jwt.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-jwt.md @@ -1,7 +1,7 @@ --- id: security-jwt title: Client authentication using tokens based on JSON Web Tokens -sidebar_label: Authentication using JWT +sidebar_label: "Authentication using JWT" original_id: security-jwt --- @@ -21,6 +21,7 @@ A user typically gets a token string from the administrator (or some automated s The compact representation of a signed JWT is a string that looks like as the following: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY ``` @@ -49,6 +50,7 @@ authParams=token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPt The token string can also be read from a file, for example: ``` + authParams=file:///path/to/token/file ``` @@ -144,6 +146,7 @@ client, err := NewClient(ClientOptions{ }) ``` + Similarly, you can also pass a `Supplier`: ```go @@ -162,6 +165,7 @@ client, err := NewClient(ClientOptions{ ```c++ + #include pulsar::ClientConfiguration config; @@ -175,6 +179,7 @@ pulsar::Client client("pulsar://broker.example.com:6650/", config); ```c# + var client = PulsarClient.Builder() .AuthenticateUsingToken("eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY") .Build(); @@ -336,3 +341,4 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 forwardAuthorizationCredentials=true ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-kerberos.md b/site2/website-next/versioned_docs/version-2.8.0/security-kerberos.md index 03f5844c68754..32e12f1b6988f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-kerberos.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-kerberos.md @@ -1,7 +1,7 @@ --- id: security-kerberos title: Authentication using Kerberos -sidebar_label: Authentication using Kerberos +sidebar_label: "Authentication using Kerberos" original_id: security-kerberos --- @@ -56,11 +56,13 @@ You need to enter the command below to specify the path to the `krb5.conf` file -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + Here is an example of the krb5.conf file: In the configuration file, `EXAMPLE.COM` is the default realm; `kdc = localhost:62037` is the kdc server url for realm `EXAMPLE.COM `: ``` + [libdefaults] default_realm = EXAMPLE.COM @@ -78,6 +80,7 @@ Usually machines configured with kerberos already have a system wide configurati You need JAAS configuration file for the client side and the broker side. JAAS configuration file provides the section of information that is used to connect KDC. Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -102,7 +105,7 @@ You need to set the `JAAS` configuration file path as JVM parameter for client a ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf ``` @@ -110,7 +113,7 @@ In the `pulsar_jaas.conf` file above 1. `PulsarBroker` is a section name in the JAAS file that each broker uses. This section tells the broker to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarBroker` allows the broker to use the keytab specified in this section. 2. `PulsarClient` is a section name in the JASS file that each broker uses. This section tells the client to use which principal inside Kerberos and the location of the keytab where the principal is stored. `PulsarClient` allows the client to use the keytab specified in this section. - The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. + The following example also reuses this `PulsarClient` section in both the Pulsar internal admin configuration and in CLI command of `bin/pulsar-client`, `bin/pulsar-perf` and `bin/pulsar-admin`. You can also add different sections for different use cases. You can have 2 separate JAAS configuration files: * the file for a broker that has sections of both `PulsarBroker` and `PulsarClient`; @@ -135,6 +138,7 @@ You can have 2 separate JAAS configuration files: Here is an example: ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -152,9 +156,10 @@ brokerClientAuthenticationParameters={"saslJaasClientSectionName":"PulsarClient" ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` + You can add this at the end of `PULSAR_EXTRA_OPTS` in the file [`pulsar_env.sh`](https://github.com/apache/pulsar/blob/master/conf/pulsar_env.sh) You must ensure that the operating system user who starts broker can reach the keytabs configured in the `pulsar_jaas.conf` file and kdc server in the `krb5.conf` file. @@ -166,6 +171,7 @@ You must ensure that the operating system user who starts broker can reach the k In client application, include `pulsar-client-auth-sasl` in your project dependency. ``` + org.apache.pulsar pulsar-client-auth-sasl @@ -183,10 +189,9 @@ You need 2 parameters: When you authenticate between client and broker with the setting in above JAAS configuration file, we need to set `saslJaasClientSectionName` to `PulsarClient` and set `serverType` to `broker`. The following is an example of creating a Java client: - ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -201,12 +206,13 @@ The following is an example of creating a Java client: .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -230,7 +236,7 @@ Step 2. Enter the command below to set JVM parameters for JAAS configuration fil ```shell - -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf + -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf ``` @@ -272,6 +278,7 @@ In comparison with the above configuration, add a new section for Pulsar Proxy i Here is an example named `pulsar_jaas.conf`: ``` + PulsarBroker { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -306,7 +313,7 @@ Here is an example named `pulsar_jaas.conf`: Pulsar client configuration is similar with client and broker configuration, except that you need to set `serverType` to `proxy` instead of `broker`, for the reason that you need to do the Kerberos authentication between client and proxy. ```java - + System.setProperty("java.security.auth.login.config", "/etc/pulsar/pulsar_jaas.conf"); System.setProperty("java.security.krb5.conf", "/etc/pulsar/krb5.conf"); @@ -321,12 +328,13 @@ Pulsar client configuration is similar with client and broker configuration, exc .serviceUrl("pulsar://my-broker.com:6650") .authentication(saslAuth) .build(); - + ``` > The first two lines in the example above are hard coded, alternatively, you can set additional JVM parameters for JAAS and krb5 configuration file when you run the application like below: ``` + java -cp -Djava.security.auth.login.config=/etc/pulsar/pulsar_jaas.conf -Djava.security.krb5.conf=/etc/pulsar/krb5.conf $APP-jar-with-dependencies.jar $CLASSNAME ``` @@ -359,6 +367,7 @@ The second part relates to authenticating between Pulsar Proxy and Pulsar Broker The broker side configuration file is the same with the above `broker.conf`, you do not need special configuration for Pulsar Proxy. ``` + authenticationEnabled=true authenticationProviders=org.apache.pulsar.broker.authentication.AuthenticationProviderSasl saslJaasClientAllowedIds=.*client.* @@ -385,6 +394,7 @@ superUserRoles=client/{clientIp}@EXAMPLE.COM Pulsar Broker acts as a Kerberos client when you authenticate with Zookeeper. According to [ZooKeeper document](https://cwiki.apache.org/confluence/display/ZOOKEEPER/Client-Server+mutual+authentication), you need these settings in `conf/zookeeper.conf`: ``` + authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider requireClientAuthScheme=sasl @@ -393,6 +403,7 @@ requireClientAuthScheme=sasl Enter the following commands to add a section of `Client` configurations in the file `pulsar_jaas.conf`, which Pulsar Broker uses: ``` + Client { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true @@ -411,6 +422,7 @@ In this setting, the principal of Pulsar Broker and keyTab file indicates the ro Pulsar Broker acts as a Kerberos client when you authenticate with Bookie. According to [BookKeeper document](http://bookkeeper.apache.org/docs/latest/security/sasl/), you need to add `bookkeeperClientAuthenticationPlugin` parameter in `broker.conf`: ``` + bookkeeperClientAuthenticationPlugin=org.apache.bookkeeper.sasl.SASLClientProviderFactory ``` @@ -420,6 +432,7 @@ In this setting, `SASLClientProviderFactory` creates a BookKeeper SASL client in Enter the following commands to add a section of `BookKeeper` configurations in the `pulsar_jaas.conf` that Pulsar Broker uses: ``` + BookKeeper { com.sun.security.auth.module.Krb5LoginModule required useKeyTab=true diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-oauth2.md b/site2/website-next/versioned_docs/version-2.8.0/security-oauth2.md index 6d5138bc24518..3596302d59834 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-oauth2.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-oauth2.md @@ -1,7 +1,7 @@ --- id: security-oauth2 title: Client authentication using OAuth 2.0 access tokens -sidebar_label: Authentication using OAuth 2.0 access tokens +sidebar_label: "Authentication using OAuth 2.0 access tokens" original_id: security-oauth2 --- @@ -114,6 +114,7 @@ PulsarClient client = PulsarClient.builder() The C++ client is similar to the Java client. You need to provide parameters of `issuerUrl`, `private_key` (the credentials file path), and the audience. ```c++ + #include pulsar::ClientConfiguration config; @@ -179,6 +180,7 @@ This section describes how to use Pulsar CLI tools to connect a cluster through This example shows how to use pulsar-admin to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-admin --admin-url https://streamnative.cloud:443 \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", @@ -196,6 +198,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-client to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-client \ --url SERVICE_URL \ --auth-plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ @@ -214,6 +217,7 @@ Set the `privateKey`, `issuerUrl`, and `audience` parameters to the values based This example shows how to use pulsar-perf to connect to a cluster through OAuth2 authentication plugin. ```shell script + bin/pulsar-perf produce --service-url pulsar+ssl://streamnative.cloud:6651 \ --auth_plugin org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2 \ --auth-params '{"privateKey":"file:///path/to/key/file.json", diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-overview.md b/site2/website-next/versioned_docs/version-2.8.0/security-overview.md index c42f643ec2b11..989c08359bb3c 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-overview.md @@ -1,7 +1,7 @@ --- id: security-overview title: Pulsar security overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: security-overview --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-tls-authentication.md b/site2/website-next/versioned_docs/version-2.8.0/security-tls-authentication.md index e77ca0e9699ea..58aabddf4df48 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-tls-authentication.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-tls-authentication.md @@ -1,7 +1,7 @@ --- id: security-tls-authentication title: Authentication using TLS -sidebar_label: Authentication using TLS +sidebar_label: "Authentication using TLS" original_id: security-tls-authentication --- @@ -48,6 +48,7 @@ $ openssl req -config openssl.cnf \ -key admin.key.pem -new -sha256 -out admin.csr.pem ``` + :::note If openssl.cnf is not specified, read [Certificate authority](http://pulsar.apache.org/docs/en/security-tls-transport/#certificate-authority) to get the openssl.cnf. @@ -130,7 +131,7 @@ When you use TLS authentication, client connects via TLS transport. You need to You need to add the following parameters to that file to use TLS authentication with the CLI tools of Pulsar: -```properties +``` webServiceUrl=https://broker.example.com:8443/ brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ @@ -144,7 +145,7 @@ authParams=tlsCertFile:/path/to/my-role.cert.pem,tlsKeyFile:/path/to/my-role.key ### Java client -```java +``` import org.apache.pulsar.client.api.PulsarClient; @@ -160,7 +161,7 @@ PulsarClient client = PulsarClient.builder() ### Python client -```python +``` from pulsar import Client, AuthenticationTLS @@ -174,7 +175,8 @@ client = Client("pulsar+ssl://broker.example.com:6651/", ### C++ client -```c++ +``` + #include pulsar::ClientConfiguration config; @@ -192,7 +194,7 @@ pulsar::Client client("pulsar+ssl://broker.example.com:6651/", config); ### Node.js client -```JavaScript +``` const Pulsar = require('pulsar-client'); @@ -213,10 +215,12 @@ const Pulsar = require('pulsar-client'); ### C# client -```c# +``` + var clientCertificate = new X509Certificate2("admin.pfx"); var client = PulsarClient.Builder() .AuthenticateUsingClientCertificate(clientCertificate) .Build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-tls-keystore.md b/site2/website-next/versioned_docs/version-2.8.0/security-tls-keystore.md index 3394c9fecfc19..1a668e64a4f96 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-tls-keystore.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-tls-keystore.md @@ -1,7 +1,7 @@ --- id: security-tls-keystore title: Using TLS with KeyStore configure -sidebar_label: Using TLS with KeyStore configure +sidebar_label: "Using TLS with KeyStore configure" original_id: security-tls-keystore --- @@ -32,7 +32,7 @@ keytool -keystore broker.keystore.jks -alias localhost -validity {validity} -gen You need to specify two parameters in the above command: 1. `keystore`: the keystore file that stores the certificate. The *keystore* file contains the private key of - the certificate; hence, it needs to be kept safely. + the certificate; hence, it needs to be kept safely. 2. `validity`: the valid time of the certificate in days. > Ensure that common name (CN) matches exactly with the fully qualified domain name (FQDN) of the server. @@ -155,13 +155,13 @@ NOTE: it is important to restrict access to the store files via filesystem permi Optional settings that may worth consider: 1. tlsClientAuthentication=false: Enable/Disable using TLS for authentication. This config when enabled will authenticate the other end - of the communication channel. It should be enabled on both brokers and clients for mutual TLS. + of the communication channel. It should be enabled on both brokers and clients for mutual TLS. 2. tlsCiphers=[TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256], A cipher suite is a named combination of authentication, encryption, MAC and key exchange - algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, - it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) - [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) + algorithm used to negotiate the security settings for a network connection using TLS network protocol. By default, + it is null. [OpenSSL Ciphers](https://www.openssl.org/docs/man1.0.2/apps/ciphers.html) + [JDK Ciphers](http://docs.oracle.com/javase/8/docs/technotes/guides/security/StandardNames.html#ciphersuites) 3. tlsProtocols=[TLSv1.3,TLSv1.2] (list out the TLS protocols that you are going to accept from clients). - By default, it is not set. + By default, it is not set. ### Configuring Clients @@ -171,33 +171,33 @@ For a a minimal configuration, user need to provide the TrustStore information. e.g. 1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - - ``` + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .build(); + + ``` 1. for java admin client @@ -264,54 +264,54 @@ Besides the TLS encryption configuring. The main work is configuring the KeyStor e.g. 1. for [Command-line tools](reference-cli-tools) like [`pulsar-admin`](reference-cli-tools#pulsar-admin), [`pulsar-perf`](reference-cli-tools#pulsar-perf), and [`pulsar-client`](reference-cli-tools#pulsar-client) use the `conf/client.conf` config file in a Pulsar installation. - ```properties - - webServiceUrl=https://broker.example.com:8443/ - brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ - useKeyStoreTls=true - tlsTrustStoreType=JKS - tlsTrustStorePath=/var/private/tls/client.truststore.jks - tlsTrustStorePassword=clientpw - authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls - authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} - - ``` + ```properties + + webServiceUrl=https://broker.example.com:8443/ + brokerServiceUrl=pulsar+ssl://broker.example.com:6651/ + useKeyStoreTls=true + tlsTrustStoreType=JKS + tlsTrustStorePath=/var/private/tls/client.truststore.jks + tlsTrustStorePassword=clientpw + authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls + authParams={"keyStoreType":"JKS","keyStorePath":"/path/to/keystorefile","keyStorePassword":"keystorepw"} + + ``` 1. for java client - ```java - - import org.apache.pulsar.client.api.PulsarClient; - - PulsarClient client = PulsarClient.builder() - .serviceUrl("pulsar+ssl://broker.example.com:6651/") - .enableTls(true) - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + import org.apache.pulsar.client.api.PulsarClient; + + PulsarClient client = PulsarClient.builder() + .serviceUrl("pulsar+ssl://broker.example.com:6651/") + .enableTls(true) + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` 1. for java admin client - ```java - - PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") - .useKeyStoreTls(true) - .tlsTrustStorePath("/var/private/tls/client.truststore.jks") - .tlsTrustStorePassword("clientpw") - .allowTlsInsecureConnection(false) - .authentication( - "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", - "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") - .build(); - - ``` + ```java + + PulsarAdmin amdin = PulsarAdmin.builder().serviceHttpUrl("https://broker.example.com:8443") + .useKeyStoreTls(true) + .tlsTrustStorePath("/var/private/tls/client.truststore.jks") + .tlsTrustStorePassword("clientpw") + .allowTlsInsecureConnection(false) + .authentication( + "org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls", + "keyStoreType:JKS,keyStorePath:/var/private/tls/client.keystore.jks,keyStorePassword:clientpw") + .build(); + + ``` ## Enabling TLS Logging diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-tls-transport.md b/site2/website-next/versioned_docs/version-2.8.0/security-tls-transport.md index 2a027cafe1606..7bf05ea7dea12 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-tls-transport.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-tls-transport.md @@ -1,7 +1,7 @@ --- id: security-tls-transport title: Transport Encryption using TLS -sidebar_label: Transport Encryption using TLS +sidebar_label: "Transport Encryption using TLS" original_id: security-tls-transport --- @@ -172,7 +172,7 @@ For JDK 11, you can obtain a list of supported values from the documentation: Proxies need to configure TLS in two directions, for clients connecting to the proxy, and for the proxy connecting to brokers. -```properties +``` # For clients connecting to the proxy tlsEnabledInProxy=true @@ -188,7 +188,7 @@ brokerClientTrustCertsFilePath=/path/to/ca.cert.pem ## Client configuration -When you enable the TLS transport encryption, you need to configure the client to use ```https://``` and port 8443 for the web service URL, and ```pulsar+ssl://``` and port 6651 for the broker service URL. +When you enable the TLS transport encryption, you need to configure the client to use ``` As the server certificate that you generated above does not belong to any of the default trust chains, you also need to either specify the path the **trust cert** (recommended), or tell the client to allow untrusted server certs. @@ -251,6 +251,7 @@ client = Client("pulsar+ssl://broker.example.com:6651/", #### C++ client ```c++ + #include ClientConfiguration config = ClientConfiguration(); @@ -280,6 +281,7 @@ const Pulsar = require('pulsar-client'); #### C# client ```c# + var certificate = new X509Certificate2("ca.cert.pem"); var client = PulsarClient.Builder() .TrustedCertificateAuthority(certificate) //If the CA is not trusted on the host, you can add it explicitly. @@ -288,3 +290,4 @@ var client = PulsarClient.Builder() .Build(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/security-token-admin.md b/site2/website-next/versioned_docs/version-2.8.0/security-token-admin.md index 68368a84e9a6d..488f5df82f599 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/security-token-admin.md +++ b/site2/website-next/versioned_docs/version-2.8.0/security-token-admin.md @@ -1,7 +1,7 @@ --- id: security-token-admin title: Token authentication admin -sidebar_label: Token authentication admin +sidebar_label: "Token authentication admin" original_id: security-token-admin --- @@ -21,9 +21,10 @@ A user will typically be given a token string by an administrator (or some autom The compact representation of a signed JWT is a string that looks like: ``` + eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJKb2UifQ.ipevRNuRP6HflG8cFKnmUPtypruRC4fb1DWtoLL62SY - ``` +``` Application will specify the token when creating the client instance. An alternative is to pass a "token supplier", that is to say a function that returns the token when the client library @@ -59,6 +60,7 @@ the brokers to allow them to validate the clients. $ bin/pulsar tokens create-secret-key --output my-secret.key ``` + To generate base64 encoded private key ```shell @@ -181,4 +183,5 @@ brokerClientAuthenticationParameters={"token":"eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ0 # Or, alternatively, read token from file # brokerClientAuthenticationParameters=file:///path/to/proxy-token.txt -``` \ No newline at end of file +``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/sql-deployment-configurations.md b/site2/website-next/versioned_docs/version-2.8.0/sql-deployment-configurations.md index 5e4e5466c99fd..a3fe137775433 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/sql-deployment-configurations.md +++ b/site2/website-next/versioned_docs/version-2.8.0/sql-deployment-configurations.md @@ -1,7 +1,7 @@ --- id: sql-deployment-configurations title: Pulsar SQL configuration and deployment -sidebar_label: Configuration and deployment +sidebar_label: "Configuration and deployment" original_id: sql-deployment-configurations --- @@ -34,9 +34,9 @@ pulsar.target-num-splits=4 ``` You can connect Presto to a Pulsar cluster with multiple hosts. To configure multiple hosts for brokers, add multiple URLs to `pulsar.web-service-url`. To configure multiple hosts for ZooKeeper, add multiple URIs to `pulsar.zookeeper-uri`. The following is an example. - ``` + pulsar.web-service-url=http://localhost:8080,localhost:8081,localhost:8082 pulsar.zookeeper-uri=localhost1,localhost2:2181 @@ -149,6 +149,7 @@ discovery.uri= 3. Start the coordinator node. ``` + $ ./bin/pulsar sql-worker run ``` @@ -156,6 +157,7 @@ $ ./bin/pulsar sql-worker run 4. Start worker nodes. ``` + $ ./bin/pulsar sql-worker run ``` @@ -177,7 +179,7 @@ presto> SELECT * FROM system.runtime.nodes; ---------+-------------------------+--------------+-------------+-------- 1 | http://192.168.2.1:8081 | testversion | true | active 3 | http://192.168.2.2:8081 | testversion | false | active - 2 | http://192.168.2.3:8081 | testversion | false | active + 2 | http://192.168.2.3:8081 | testversion | false | active ``` @@ -188,3 +190,4 @@ For more information about deployment in Presto, refer to [Presto deployment](ht The broker does not advance LAC, so when Pulsar SQL bypass broker to query data, it can only read entries up to the LAC that all the bookies learned. You can enable periodically write LAC on the broker by setting "bookkeeperExplicitLacIntervalInMills" in the broker.conf. ::: + diff --git a/site2/website-next/versioned_docs/version-2.8.0/sql-getting-started.md b/site2/website-next/versioned_docs/version-2.8.0/sql-getting-started.md index 98d3ec8d9ebad..e8c4ecb101fdd 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/sql-getting-started.md +++ b/site2/website-next/versioned_docs/version-2.8.0/sql-getting-started.md @@ -1,7 +1,7 @@ --- id: sql-getting-started title: Query data with Pulsar SQL -sidebar_label: Query data +sidebar_label: "Query data" original_id: sql-getting-started --- @@ -188,3 +188,4 @@ public class TestProducer { } ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/sql-overview.md b/site2/website-next/versioned_docs/version-2.8.0/sql-overview.md index 5f6bcf0a2fbdf..68922a33ba279 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/sql-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/sql-overview.md @@ -1,7 +1,7 @@ --- id: sql-overview title: Pulsar SQL Overview -sidebar_label: Overview +sidebar_label: "Overview" original_id: sql-overview --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/sql-rest-api.md b/site2/website-next/versioned_docs/version-2.8.0/sql-rest-api.md index edb616ffd8e49..d2b078793efd0 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/sql-rest-api.md +++ b/site2/website-next/versioned_docs/version-2.8.0/sql-rest-api.md @@ -1,7 +1,7 @@ --- id: sql-rest-api title: Pulsar SQL REST APIs -sidebar_label: REST APIs +sidebar_label: "REST APIs" original_id: sql-rest-api --- @@ -189,7 +189,6 @@ The following is an example of `show catalogs`. The query continues until the re :::note - Since the response data is not in sync with the query state from the perspective of clients, you cannot rely on the response data to determine whether the query completes. ::: diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aliyun.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aliyun.md index 2c6789c4f1ea7..27d2f16c4121d 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aliyun.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aliyun.md @@ -1,7 +1,7 @@ --- id: tiered-storage-aliyun title: Use Aliyun OSS offloader with Pulsar -sidebar_label: Aliyun OSS offloader +sidebar_label: "Aliyun OSS offloader" original_id: tiered-storage-aliyun --- @@ -27,29 +27,28 @@ This example uses Pulsar 2.8.0. 2. Download and untar the Pulsar offloaders package, then copy the Pulsar offloaders as `offloaders` in the Pulsar directory, see [here](https://pulsar.apache.org/docs/en/standalone/#install-tiered-storage-offloaders-optional). - **Output** - - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - + **Output** + + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - ``` - tiered-storage-file-system-2.8.0.nar - tiered-storage-jcloud-2.8.0.nar + ``` + + tiered-storage-file-system-2.8.0.nar + tiered-storage-jcloud-2.8.0.nar + + ``` - ``` + :::note -:::note - -* If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: ## Configuration :::note - Before offloading data from BookKeeper to Aliyun OSS, you need to configure some properties of the Aliyun OSS offload driver. ::: @@ -62,21 +61,21 @@ You can configure the Aliyun OSS offloader driver in the configuration file `bro - **Required** configurations are as below. - | Required configuration | Description | Example value | - | --- | --- |--- | - | `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | aliyun-oss | - | `offloadersDirectory` | Offloader directory | offloaders | - | `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload | - | `managedLedgerOffloadServiceEndpoint` | Endpoint | http://oss-cn-hongkong.aliyuncs.com | + | Required configuration | Description | Example value | + | --- | --- |--- | + | `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | aliyun-oss | + | `offloadersDirectory` | Offloader directory | offloaders | + | `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload | + | `managedLedgerOffloadServiceEndpoint` | Endpoint | http://oss-cn-hongkong.aliyuncs.com | - **Optional** configurations are as below. - | Optional | Description | Example value | - | --- | --- | --- | - | `managedLedgerOffloadReadBufferSizeInBytes` | Size of block read | 1 MB | - | `managedLedgerOffloadMaxBlockSizeInBytes` | Size of block write | 64 MB | - | `managedLedgerMinLedgerRolloverTimeMinutes` | Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment. | 2 | - | `managedLedgerMaxEntriesPerLedger` | Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment. | 5000 | + | Optional | Description | Example value | + | --- | --- | --- | + | `managedLedgerOffloadReadBufferSizeInBytes` | Size of block read | 1 MB | + | `managedLedgerOffloadMaxBlockSizeInBytes` | Size of block write | 64 MB | + | `managedLedgerMinLedgerRolloverTimeMinutes` | Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment. | 2 | + | `managedLedgerMaxEntriesPerLedger` | Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment. | 5000 | #### Bucket (required) @@ -98,16 +97,17 @@ The endpoint is the region where a bucket is located. :::tip - For more information about Aliyun OSS regions and endpoints, see [International website](https://www.alibabacloud.com/help/doc-detail/31837.htm) or [Chinese website](https://help.aliyun.com/document_detail/31837.html). ::: + ##### Example This example sets the endpoint as _oss-us-west-1-internal_. ``` + managedLedgerOffloadServiceEndpoint=http://oss-us-west-1-internal.aliyuncs.com ``` @@ -164,7 +164,6 @@ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namesp :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -177,84 +176,86 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Aliyun OSS until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Aliyun OSS until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the Aliyun OSS offloader to run manually using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + **Output** - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -::: + ::: - This example checks the Aliyun OSS offloader status using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash - - Offload is currently running - - ``` - - To wait for the Aliyun OSS offloader to complete the job, add the `-w` flag. - - ```bash - - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` + **Output** - **Output** - + ```bash + + Offload is currently running + + ``` - ``` - Offload was a success + To wait for the Aliyun OSS offloader to complete the job, add the `-w` flag. - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + **Output** - ```bash + ``` + + Offload was a success + + ``` - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ``` - Error in offload - null + ``` + + Error in offload + null - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` - ```` + :::tip -:::tip + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + ::: -::: diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aws.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aws.md index 2e4cedceef178..9ddfcebac7750 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aws.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-aws.md @@ -1,7 +1,7 @@ --- id: tiered-storage-aws title: Use AWS S3 offloader with Pulsar -sidebar_label: AWS S3 offloader +sidebar_label: "AWS S3 offloader" original_id: tiered-storage-aws --- @@ -31,52 +31,53 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget): - ```shell - - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ls offloaders + ls offloaders + + ``` - ``` + **Output** - **Output** + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/) and [GCS](https://cloud.google.com/storage/) for long term storage. + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar + :::note - ``` - -:::note + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. - -::: + ::: ## Configuration :::note - Before offloading data from BookKeeper to AWS S3, you need to configure some properties of the AWS S3 offload driver. ::: @@ -89,21 +90,21 @@ You can configure the AWS S3 offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 - `offloadersDirectory` | Offloader directory | offloaders - `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive.

    **Note**: there is a third driver type, S3, which is identical to AWS S3, though S3 requires that you specify an endpoint URL using `s3ManagedLedgerOffloadServiceEndpoint`. This is useful if using an S3 compatible data store other than AWS S3. | aws-s3 + `offloadersDirectory` | Offloader directory | offloaders + `s3ManagedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 - `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `s3ManagedLedgerOffloadRegion` | Bucket region

    **Note**: before specifying a value for this parameter, you need to set the following configurations. Otherwise, you might get an error.

    - Set [`s3ManagedLedgerOffloadServiceEndpoint`](https://docs.aws.amazon.com/general/latest/gr/s3.html).

    Example
    `s3ManagedLedgerOffloadServiceEndpoint=https://s3.YOUR_REGION.amazonaws.com`

    - Grant `GetBucketLocation` permission to a user.

    For how to grant `GetBucketLocation` permission to a user, see [here](https://docs.aws.amazon.com/AmazonS3/latest/dev/using-with-s3-actions.html#using-with-s3-actions-related-to-buckets).| eu-west-3 + `s3ManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `s3ManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -125,16 +126,17 @@ A bucket region is a region where a bucket is located. If a bucket region is not :::tip - For more information about AWS regions and endpoints, see [here](https://docs.aws.amazon.com/general/latest/gr/rande.html). ::: + ##### Example This example sets the bucket region as _europe-west-3_. ``` + s3ManagedLedgerOffloadRegion=eu-west-3 ``` @@ -150,49 +152,49 @@ Once you have created a set of credentials in the AWS IAM console, you can confi * Use EC2 instance metadata credentials. - If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. + If you are on AWS instance with an instance profile that provides credentials, Pulsar uses these credentials if no other mechanism is provided. * Set the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. - - ```bash + "export" is important so that the variables are made available in the environment of spawned processes. - export AWS_ACCESS_KEY_ID=ABC123456789 - export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - - ``` + ```bash + + export AWS_ACCESS_KEY_ID=ABC123456789 + export AWS_SECRET_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` * Add the Java system properties `aws.accessKeyId` and `aws.secretKey` to `PULSAR_EXTRA_OPTS` in `conf/pulsar_env.sh`. - ```bash - - PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" - - ``` + ```bash + + PULSAR_EXTRA_OPTS="${PULSAR_EXTRA_OPTS} ${PULSAR_MEM} ${PULSAR_GC} -Daws.accessKeyId=ABC123456789 -Daws.secretKey=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c -Dio.netty.leakDetectionLevel=disabled -Dio.netty.recycler.maxCapacity.default=1000 -Dio.netty.recycler.linkCapacity=1024" + + ``` * Set the access credentials in `~/.aws/credentials`. - ```conf - - [default] - aws_access_key_id=ABC123456789 - aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - - ``` + ```conf + + [default] + aws_access_key_id=ABC123456789 + aws_secret_access_key=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` * Assume an IAM role. - This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. - - The broker must be rebooted for credentials specified in `pulsar_env` to take effect. - - ```conf + This example uses the `DefaultAWSCredentialsProviderChain` for assuming this role. - s3ManagedLedgerOffloadRole= - s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload + The broker must be rebooted for credentials specified in `pulsar_env` to take effect. - ``` + ```conf + + s3ManagedLedgerOffloadRole= + s3ManagedLedgerOffloadRoleSessionName=pulsar-s3-offload + + ``` #### Size of block read/write @@ -231,7 +233,6 @@ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namesp :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -244,87 +245,88 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to AWS S3 until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the AWS S3 offloader to run manually using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + **Output** - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -::: + ::: - This example checks the AWS S3 offloader status using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash - - Offload is currently running - - ``` - - To wait for the AWS S3 offloader to complete the job, add the `-w` flag. - - ```bash - - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - + **Output** - ``` - Offload was a success + ```bash + + Offload is currently running + + ``` - ``` + To wait for the AWS S3 offloader to complete the job, add the `-w` flag. - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - ```bash + **Output** - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + ``` + + Offload was a success + + ``` - ``` + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + ``` + + Error in offload + null - ```` + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). -::: + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-azure.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-azure.md index 3b4ab88df5320..13824394e2cc3 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-azure.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-azure.md @@ -1,7 +1,7 @@ --- id: tiered-storage-azure title: Use Azure BlobStore offloader with Pulsar -sidebar_label: Azure BlobStore offloader +sidebar_label: "Azure BlobStore offloader" original_id: tiered-storage-azure --- @@ -31,52 +31,53 @@ This example uses Pulsar 2.6.2. * Use [wget](https://www.gnu.org/software/wget): - ```shell - - wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz - - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.6.2/apache-pulsar-2.6.2-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - - wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz - tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz - - ``` + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.6.2/apache-pulsar-offloaders-2.6.2-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.6.2-bin.tar.gz + + ``` 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders + ``` + + mv apache-pulsar-offloaders-2.6.2/offloaders apache-pulsar-2.6.2/offloaders - ls offloaders + ls offloaders + + ``` - ``` - - **Output** + **Output** - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. - - ``` - tiered-storage-file-system-2.6.2.nar - tiered-storage-jcloud-2.6.2.nar + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/) and [Azure](https://portal.azure.com/#home) for long term storage. - ``` + ``` + + tiered-storage-file-system-2.6.2.nar + tiered-storage-jcloud-2.6.2.nar + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: ## Configuration :::note - Before offloading data from BookKeeper to Azure BlobStore, you need to configure some properties of the Azure BlobStore offload driver. ::: @@ -89,20 +90,20 @@ You can configure the Azure BlobStore offloader driver in the configuration file - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name | azureblob - `offloadersDirectory` | Offloader directory | offloaders - `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name | azureblob + `offloadersDirectory` | Offloader directory | offloaders + `managedLedgerOffloadBucket` | Bucket | pulsar-topic-offload - **Optional** configurations are as below. - Optional | Description | Example value - |---|---|--- - `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional | Description | Example value + |---|---|--- + `managedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `managedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Bucket (required) @@ -124,14 +125,14 @@ To be able to access Azure BlobStore, you need to authenticate with Azure BlobSt * Set the environment variables `AZURE_STORAGE_ACCOUNT` and `AZURE_STORAGE_ACCESS_KEY` in `conf/pulsar_env.sh`. - "export" is important so that the variables are made available in the environment of spawned processes. + "export" is important so that the variables are made available in the environment of spawned processes. - ```bash - - export AZURE_STORAGE_ACCOUNT=ABC123456789 - export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c - - ``` + ```bash + + export AZURE_STORAGE_ACCOUNT=ABC123456789 + export AZURE_STORAGE_ACCESS_KEY=ded7db27a4558e2ea8bbf0bf37ae0e8521618f366c + + ``` #### Size of block read/write @@ -170,7 +171,6 @@ bin/pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namesp :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-set-offload-threshold-em-). ::: @@ -183,84 +183,86 @@ For individual topics, you can trigger Azure BlobStore offloader manually using - Use CLI tools (such as pulsar-admin). - To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. + To trigger it via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to Azure BlobStore until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the Azure BlobStore offloader to run manually using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 - - ``` - - **Output** + ```bash + + bin/pulsar-admin topics offload --size-threshold 10M my-tenant/my-namespace/topic1 + + ``` - ```bash + **Output** - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` - ``` + :::tip -:::tip + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-em-). - -::: + ::: - This example checks the Azure BlobStore offloader status using pulsar-admin. - ```bash - - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash - - Offload is currently running - - ``` - - To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. - - ```bash - - bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` + **Output** - **Output** - + ```bash + + Offload is currently running + + ``` - ``` - Offload was a success + To wait for the Azure BlobStore offloader to complete the job, add the `-w` flag. - ``` + ```bash + + bin/pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + **Output** - ```bash + ``` + + Offload was a success + + ``` - bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. - ``` + ```bash + + bin/pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** + **Output** - ``` - Error in offload - null + ``` + + Error in offload + null - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: + + ``` - ```` + :::tip -:::tip + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, and default values, see [here](https://pulsar.apache.org/tools/pulsar-admin/2.6.0-SNAPSHOT/#-em-offload-status-em-). + ::: -::: diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-filesystem.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-filesystem.md index 0d38463032a10..c988c98b78fe4 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-filesystem.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-filesystem.md @@ -1,7 +1,7 @@ --- id: tiered-storage-filesystem title: Use filesystem offloader with Pulsar -sidebar_label: Filesystem offloader +sidebar_label: "Filesystem offloader" original_id: tiered-storage-filesystem --- @@ -33,45 +33,47 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz - - ``` - -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - - ls offloaders + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - ``` + ls offloaders + + ``` - **Output** + **Output** - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar - - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` :::note @@ -84,7 +86,6 @@ This example uses Pulsar 2.5.1. :::note - Before offloading data from BookKeeper to filesystem, you need to configure some properties of the filesystem offloader driver. ::: @@ -97,18 +98,18 @@ You can configure filesystem offloader driver in the configuration file `broker. - **Required** configurations are as below. - Required configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem - `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 - `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml + Required configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver` | Offloader driver name, which is case-insensitive. | filesystem + `fileSystemURI` | Connection address | hdfs://127.0.0.1:9000 + `fileSystemProfilePath` | Hadoop profile path | ../conf/filesystem_offload_core_site.xml - **Optional** configurations are as below. - Optional configuration| Description | Example value - |---|---|--- - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 - `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 + Optional configuration| Description | Example value + |---|---|--- + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic

    **Note**: it is not recommended that you set this configuration in the production environment.|2 + `managedLedgerMaxEntriesPerLedger`|Maximum number of entries to append to a ledger before triggering a rollover.

    **Note**: it is not recommended that you set this configuration in the production environment.|5000 #### Offloader driver (required) @@ -153,6 +154,7 @@ fileSystemProfilePath=../conf/filesystem_offload_core_site.xml You can set the following configurations in the _filesystem_offload_core_site.xml_ file. ``` + fs.defaultFS @@ -187,7 +189,6 @@ You can set the following configurations in the _filesystem_offload_core_site.xm :::tip - For more information about the Hadoop HDFS, see [here](https://hadoop.apache.org/docs/current/). ::: @@ -218,7 +219,6 @@ pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#set-offload-threshold). ::: @@ -237,81 +237,82 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh - This example triggers the filesystem offloader to run manually using pulsar-admin. - ```bash - - pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash + ```bash + + pulsar-admin topics offload --size-threshold 10M persistent://my-tenant/my-namespace/topic1 + + ``` - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + **Output** - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). -::: + ::: - This example checks filesystem offloader status using pulsar-admin. - ```bash - - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash - - Offload is currently running - - ``` - - To wait for the filesystem to complete the job, add the `-w` flag. - - ```bash - - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 - - ``` + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - **Output** - + **Output** - ``` - Offload was a success + ```bash + + Offload is currently running + + ``` - ``` + To wait for the filesystem to complete the job, add the `-w` flag. - If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - ```bash + **Output** - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + ``` + + Offload was a success + + ``` - ``` + If there is an error in the offloading operation, the error is propagated to the `pulsar-admin topics offload-status` command. - **Output** + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + ``` + + Error in offload + null - ```` + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). -::: + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-gcs.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-gcs.md index 150638513672b..5c70cd36cd0fd 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-gcs.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-gcs.md @@ -1,7 +1,7 @@ --- id: tiered-storage-gcs title: Use GCS offloader with Pulsar -sidebar_label: GCS offloader +sidebar_label: "GCS offloader" original_id: tiered-storage-gcs --- @@ -31,53 +31,54 @@ This example uses Pulsar 2.5.1. * Use [wget](https://www.gnu.org/software/wget) - ```shell - - wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz - - ``` + ```shell + + wget https://archive.apache.org/dist/pulsar/pulsar-2.5.1/apache-pulsar-2.5.1-bin.tar.gz + + ``` 2. Download and untar the Pulsar offloaders package. - ```bash - - wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - - tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + ```bash + + wget https://downloads.apache.org/pulsar/pulsar-2.5.1/apache-pulsar-offloaders-2.5.1-bin.tar.gz - ``` + tar xvfz apache-pulsar-offloaders-2.5.1-bin.tar.gz + + ``` -:::note + :::note -* If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. -* If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + * If you are running Pulsar in a bare metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8S and DCOS), you can use the `apachepulsar/pulsar-all` image instead of the `apachepulsar/pulsar` image. `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. -::: + ::: 3. Copy the Pulsar offloaders as `offloaders` in the Pulsar directory. - ``` - mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - - ls offloaders - - ``` + ``` + + mv apache-pulsar-offloaders-2.5.1/offloaders apache-pulsar-2.5.1/offloaders - **Output** + ls offloaders + + ``` - As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. + **Output** - ``` - tiered-storage-file-system-2.5.1.nar - tiered-storage-jcloud-2.5.1.nar + As shown in the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support GCS and AWS S3 for long term storage. - ``` + ``` + + tiered-storage-file-system-2.5.1.nar + tiered-storage-jcloud-2.5.1.nar + + ``` ## Configuration :::note - Before offloading data from BookKeeper to GCS, you need to configure some properties of the GCS offloader driver. ::: @@ -90,22 +91,22 @@ You can configure GCS offloader driver in the configuration file `broker.conf` o - **Required** configurations are as below. - **Required** configuration | Description | Example value - |---|---|--- - `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage - `offloadersDirectory`|Offloader directory|offloaders - `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload - `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 - `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json + **Required** configuration | Description | Example value + |---|---|--- + `managedLedgerOffloadDriver`|Offloader driver name, which is case-insensitive.|google-cloud-storage + `offloadersDirectory`|Offloader directory|offloaders + `gcsManagedLedgerOffloadBucket`|Bucket|pulsar-topic-offload + `gcsManagedLedgerOffloadRegion`|Bucket region|europe-west3 + `gcsManagedLedgerOffloadServiceAccountKeyFile`|Authentication |/Users/user-name/Downloads/project-804d5e6a6f33.json - **Optional** configurations are as below. - Optional configuration|Description|Example value - |---|---|--- - `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB - `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB - `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 - `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 + Optional configuration|Description|Example value + |---|---|--- + `gcsManagedLedgerOffloadReadBufferSizeInBytes`|Size of block read|1 MB + `gcsManagedLedgerOffloadMaxBlockSizeInBytes`|Size of block write|64 MB + `managedLedgerMinLedgerRolloverTimeMinutes`|Minimum time between ledger rollover for a topic.|2 + `managedLedgerMaxEntriesPerLedger`|The max number of entries to append to a ledger before triggering a rollover.|5000 #### Bucket (required) @@ -127,7 +128,6 @@ Bucket region is the region where a bucket is located. If a bucket region is not :::tip - For more information about bucket location, see [here](https://cloud.google.com/storage/docs/bucket-locations). ::: @@ -137,6 +137,7 @@ For more information about bucket location, see [here](https://cloud.google.com/ This example sets the bucket region as _europe-west3_. ``` + gcsManagedLedgerOffloadRegion=europe-west3 ``` @@ -160,7 +161,7 @@ To generate service account credentials or view the public credentials that you' 4. In the **Create service account** window, type a name for the service account and select **Furnish a new private key**. - If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. + If you want to [grant G Suite domain-wide authority](https://developers.google.com/identity/protocols/OAuth2ServiceAccount#delegatingauthority) to the service account, select **Enable G Suite Domain-wide Delegation**. 5. Click **Create**. @@ -171,13 +172,12 @@ To generate service account credentials or view the public credentials that you' ::: 6. You can get the following information and set this in `broker.conf`. - - - ```conf - - gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" - ``` + ```conf + + gcsManagedLedgerOffloadServiceAccountKeyFile="/Users/user-name/Downloads/project-804d5e6a6f33.json" + + ``` :::tip @@ -223,7 +223,6 @@ pulsar-admin namespaces set-offload-threshold --size 10M my-tenant/my-namespace :::tip - For more information about the `pulsar-admin namespaces set-offload-threshold options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#set-offload-threshold). ::: @@ -236,86 +235,88 @@ For individual topics, you can trigger GCS offloader manually using one of the f - Use CLI tools (such as pulsar-admin). - To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. + To trigger the GCS via CLI tools, you need to specify the maximum amount of data (threshold) that should be retained on a Pulsar cluster for a topic. If the size of the topic data on the Pulsar cluster exceeds this threshold, segments from the topic are moved to GCS until the threshold is no longer exceeded. Older segments are moved first. #### Example - This example triggers the GCS offloader to run manually using pulsar-admin with the command `pulsar-admin topics offload (topic-name) (threshold)`. - ```bash - - pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M - - ``` - - **Output** - - ```bash + ```bash + + pulsar-admin topics offload persistent://my-tenant/my-namespace/topic1 10M + + ``` - Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + **Output** - ``` + ```bash + + Offload triggered for persistent://my-tenant/my-namespace/topic1 for messages before 2:0:-1 + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). + For more information about the `pulsar-admin topics offload options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload). -::: + ::: - This example checks the GCS offloader status using pulsar-admin with the command `pulsar-admin topics offload-status options`. - ```bash - - pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 - - ``` - - **Output** - - ```bash - - Offload is currently running - - ``` - - To wait for GCS to complete the job, add the `-w` flag. - - ```bash + ```bash + + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + **Output** - ``` + ```bash + + Offload is currently running + + ``` - **Output** + To wait for GCS to complete the job, add the `-w` flag. - ``` - Offload was a success + ```bash + + pulsar-admin topics offload-status -w persistent://my-tenant/my-namespace/topic1 + + ``` - ``` + **Output** - If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ``` + + Offload was a success + + ``` - ```bash + If there is an error in offloading, the error is propagated to the `pulsar-admin topics offload-status` command. + ```bash + pulsar-admin topics offload-status persistent://my-tenant/my-namespace/topic1 + + ``` - ``` - - **Output** - - ``` - Error in offload - null + **Output** - Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + ``` + + Error in offload + null - ```` + Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= + + ``` -:::tip + :::tip -For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). + For more information about the `pulsar-admin topics offload-status options` command, including flags, descriptions, default values, and shorthands, see [here](reference-pulsar-admin.md#offload-status). -::: + ::: ## Tutorial diff --git a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-overview.md b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-overview.md index 552cb78f1565b..8536268c48bc8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-overview.md +++ b/site2/website-next/versioned_docs/version-2.8.0/tiered-storage-overview.md @@ -1,7 +1,7 @@ --- id: tiered-storage-overview title: Overview of tiered storage -sidebar_label: Overview +sidebar_label: "Overview" original_id: tiered-storage-overview --- @@ -13,24 +13,24 @@ Pulsar's **Tiered Storage** feature allows older backlog data to be moved from B * Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -:::tip + :::tip -For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws). -For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs). + For more information about how to use the AWS S3 offloader with Pulsar, see [here](tiered-storage-aws). + For more information about how to use the GCS offloader with Pulsar, see [here](tiered-storage-gcs). -::: + ::: * Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystems for long term storage. - With Hadoop, it is easy to add support for more filesystems in the future. + With Hadoop, it is easy to add support for more filesystems in the future. -:::tip + :::tip -For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem). + For more information about how to use the filesystem offloader with Pulsar, see [here](tiered-storage-filesystem). -::: + ::: ## When to use tiered storage? diff --git a/site2/website-next/versioned_docs/version-2.8.0/transaction-api.md b/site2/website-next/versioned_docs/version-2.8.0/transaction-api.md index b8c04b9f3680b..2431f80649aaf 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/transaction-api.md +++ b/site2/website-next/versioned_docs/version-2.8.0/transaction-api.md @@ -1,7 +1,7 @@ --- id: transactions-api title: Transactions API -sidebar_label: Transactions API +sidebar_label: "Transactions API" original_id: transactions-api --- @@ -15,6 +15,7 @@ All messages in a transaction are available only to consumers after the transact 1. To enable transactions in Pulsar, you need to configure the parameter in the `broker.conf` file. ``` + transactionCoordinatorEnabled=true ``` @@ -22,6 +23,7 @@ transactionCoordinatorEnabled=true 2. Initialize transaction coordinator metadata, so the transaction coordinators can leverage advantages of the partitioned topic, such as load balance. ``` + bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone ``` @@ -33,6 +35,7 @@ After initializing transaction coordinator metadata, you can use the transaction You can enable transaction for transaction client and initialize transaction coordinator client. ``` + PulsarClient pulsarClient = PulsarClient.builder() .serviceUrl("pulsar://localhost:6650") .enableTransaction(true) @@ -44,6 +47,7 @@ PulsarClient pulsarClient = PulsarClient.builder() You can start transaction in the following way. ``` + Transaction txn = pulsarClient .newTransaction() .withTransactionTimeout(5, TimeUnit.MINUTES) @@ -57,6 +61,7 @@ Transaction txn = pulsarClient A transaction parameter is required when producing new transaction messages. The semantic of the transaction messages in Pulsar is `read-committed`, so the consumer cannot receive the ongoing transaction messages before the transaction is committed. ``` + producer.newMessage(txn).value("Hello Pulsar Transaction".getBytes()).sendAsync(); ``` @@ -66,6 +71,7 @@ producer.newMessage(txn).value("Hello Pulsar Transaction".getBytes()).sendAsync( The transaction acknowledgement requires a transaction parameter. The transaction acknowledgement marks the messages state to pending-ack state. When the transaction is committed, the pending-ack state becomes ack state. If the transaction is aborted, the pending-ack state becomes unack state. ``` + Message message = consumer.receive(); consumer.acknowledgeAsync(message.getMessageId(), txn); @@ -76,6 +82,7 @@ consumer.acknowledgeAsync(message.getMessageId(), txn); When the transaction is committed, consumers receive the transaction messages and the pending-ack state becomes ack state. ``` + txn.commit().get(); ``` @@ -85,15 +92,16 @@ txn.commit().get(); When the transaction is aborted, the transaction acknowledgement is canceled and the pending-ack messages are redelivered. ``` + txn.abort().get(); ``` ### Example The following example shows how messages are processed in transaction. - ``` + PulsarClient pulsarClient = PulsarClient.builder() .serviceUrl(getPulsarServiceList().get(0).getBrokerServiceUrl()) .statsInterval(0, TimeUnit.SECONDS) @@ -145,6 +153,7 @@ To enable batch messages in transactions, you need to enable the batch index ack To enable batch index acknowledgement, you need to set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. ``` + acknowledgmentAtBatchIndexLevelEnabled=true ``` @@ -152,6 +161,7 @@ acknowledgmentAtBatchIndexLevelEnabled=true And then you need to call the `enableBatchIndexAcknowledgment(true)` method in the consumer builder. ``` + Consumer sinkConsumer = pulsarClient .newConsumer() .topic(transferTopic) @@ -162,3 +172,4 @@ Consumer sinkConsumer = pulsarClient .subscribe(); ``` + diff --git a/site2/website-next/versioned_docs/version-2.8.0/transaction-guarantee.md b/site2/website-next/versioned_docs/version-2.8.0/transaction-guarantee.md index bd3df0c4d0e2b..b75c94a1625e8 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/transaction-guarantee.md +++ b/site2/website-next/versioned_docs/version-2.8.0/transaction-guarantee.md @@ -1,7 +1,7 @@ --- id: transactions-guarantee title: Transactions Guarantee -sidebar_label: Transactions Guarantee +sidebar_label: "Transactions Guarantee" original_id: transactions-guarantee --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/txn-how.md b/site2/website-next/versioned_docs/version-2.8.0/txn-how.md index 91db9631e0dd0..ee53dab38d9e7 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/txn-how.md +++ b/site2/website-next/versioned_docs/version-2.8.0/txn-how.md @@ -1,7 +1,7 @@ --- id: txn-how title: How transactions work? -sidebar_label: How transactions work? +sidebar_label: "How transactions work?" original_id: txn-how --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/txn-monitor.md b/site2/website-next/versioned_docs/version-2.8.0/txn-monitor.md index a9768c3fd598b..f2e470c51479a 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/txn-monitor.md +++ b/site2/website-next/versioned_docs/version-2.8.0/txn-monitor.md @@ -1,7 +1,7 @@ --- id: txn-monitor title: How to monitor transactions? -sidebar_label: How to monitor transactions? +sidebar_label: "How to monitor transactions?" original_id: txn-monitor --- diff --git a/site2/website-next/versioned_docs/version-2.8.0/txn-use.md b/site2/website-next/versioned_docs/version-2.8.0/txn-use.md index 2514ade124776..a3df8b91cf2b3 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/txn-use.md +++ b/site2/website-next/versioned_docs/version-2.8.0/txn-use.md @@ -1,7 +1,7 @@ --- id: txn-use title: How to use transactions? -sidebar_label: How to use transactions? +sidebar_label: "How to use transactions?" original_id: txn-use --- @@ -25,52 +25,57 @@ This section provides an example of how to use the transaction API to send and r 2. Enable transaction. - Change the configuration in the `broker.conf` file. + Change the configuration in the `broker.conf` file. - ``` - transactionCoordinatorEnabled=true + ``` + + transactionCoordinatorEnabled=true + + ``` - ``` + If you want to enable batch messages in transactions, follow the steps below. - If you want to enable batch messages in transactions, follow the steps below. + Set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. - Set `acknowledgmentAtBatchIndexLevelEnabled` to `true` in the `broker.conf` or `standalone.conf` file. - - ``` - acknowledgmentAtBatchIndexLevelEnabled=true - - ``` + ``` + + acknowledgmentAtBatchIndexLevelEnabled=true + + ``` 3. Initialize transaction coordinator metadata. - The transaction coordinator can leverage the advantages of partitioned topics (such as load balance). - - **Input** + The transaction coordinator can leverage the advantages of partitioned topics (such as load balance). - ``` - bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone + **Input** - ``` + ``` + + bin/pulsar initialize-transaction-coordinator-metadata -cs 127.0.0.1:2181 -c standalone + + ``` - **Output** + **Output** - ``` - Transaction coordinator metadata setup success - - ``` + ``` + + Transaction coordinator metadata setup success + + ``` 4. Initialize a Pulsar client. - ``` - PulsarClient client = PulsarClient.builder() - - .serviceUrl(“pulsar://localhost:6650”) + ``` + + PulsarClient client = PulsarClient.builder() - .enableTransaction(true) + .serviceUrl(“pulsar://localhost:6650”) - .build(); + .enableTransaction(true) - ``` + .build(); + + ``` Now you can start using the transaction API to send and receive messages. Below is an example of a `consume-process-produce` application written in Java. @@ -89,6 +94,7 @@ Let’s walk through this example step by step. [1] Example of enabling batch messages ack in transactions in the consumer builder. ``` + Consumer sinkConsumer = pulsarClient .newConsumer() .topic(transferTopic) diff --git a/site2/website-next/versioned_docs/version-2.8.0/txn-what.md b/site2/website-next/versioned_docs/version-2.8.0/txn-what.md index f9cc2483a2d0e..24d828a17d4b6 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/txn-what.md +++ b/site2/website-next/versioned_docs/version-2.8.0/txn-what.md @@ -1,7 +1,7 @@ --- id: txn-what title: What are transactions? -sidebar_label: What are transactions? +sidebar_label: "What are transactions?" original_id: txn-what --- @@ -32,9 +32,9 @@ Pulsar transactions have the following semantics: * A group of messages in a transaction can be received from, produced to, and acknowledged by multiple partitions. * Consumers are only allowed to read committed (acked) messages. In other words, the broker does not deliver transactional messages which are part of an open transaction or messages which are part of an aborted transaction. - + * Message writes across multiple partitions are atomic. - + * Message acks across multiple subscriptions are atomic. A message is acked successfully only once by a consumer under the subscription when acknowledging the message with the transaction ID. ## Transactions and stream processing @@ -57,8 +57,8 @@ Prior to Pulsar 2.8.0, there was no easy way to build stream processing applicat * [Pulsar Flink connector](https://flink.apache.org/2021/01/07/pulsar-flink-connector-270.html) - Prior to Pulsar 2.8.0, if you want to build stream applications using Pulsar and Flink, the Pulsar Flink connector only supported exactly-once source connector and at-least-once sink connector, which means the highest processing guarantee for end-to-end was at-least-once, there was possibility that the resulting messages from streaming applications produce duplicated messages to the resulting topics in Pulsar. + Prior to Pulsar 2.8.0, if you want to build stream applications using Pulsar and Flink, the Pulsar Flink connector only supported exactly-once source connector and at-least-once sink connector, which means the highest processing guarantee for end-to-end was at-least-once, there was possibility that the resulting messages from streaming applications produce duplicated messages to the resulting topics in Pulsar. - With the transaction introduced in Pulsar 2.8.0, the Pulsar Flink sink connector can support exactly-once semantics by implementing the designated `TwoPhaseCommitSinkFunction` and hooking up the Flink sink message lifecycle with Pulsar transaction API. + With the transaction introduced in Pulsar 2.8.0, the Pulsar Flink sink connector can support exactly-once semantics by implementing the designated `TwoPhaseCommitSinkFunction` and hooking up the Flink sink message lifecycle with Pulsar transaction API. * Support for Pulsar Functions and other connectors will be added in the future releases. diff --git a/site2/website-next/versioned_docs/version-2.8.0/txn-why.md b/site2/website-next/versioned_docs/version-2.8.0/txn-why.md index 37199f4edb35d..2138feecbfb0f 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/txn-why.md +++ b/site2/website-next/versioned_docs/version-2.8.0/txn-why.md @@ -1,7 +1,7 @@ --- id: txn-why title: Why transactions? -sidebar_label: Why transactions? +sidebar_label: "Why transactions?" original_id: txn-why --- @@ -40,9 +40,9 @@ In Pulsar, the highest level of message delivery guarantee is using an [idempote - Similarly, for Pulsar Function, it only guarantees exactly once semantics for an idempotent function on a single event rather than processing multiple events or producing multiple results that can happen exactly. - For example, if a function accepts multiple events and produces one result (for example, window function), the function may fail between producing the result and acknowledging the incoming messages, or even between acknowledging individual events, which causes all (or some) incoming messages to be re-delivered and reprocessed, and a new result is generated. + For example, if a function accepts multiple events and produces one result (for example, window function), the function may fail between producing the result and acknowledging the incoming messages, or even between acknowledging individual events, which causes all (or some) incoming messages to be re-delivered and reprocessed, and a new result is generated. - However, many scenarios need atomic guarantees across multiple partitions and sessions. + However, many scenarios need atomic guarantees across multiple partitions and sessions. - Consumers need to rely on more mechanisms to acknowledge (ack) messages once. diff --git a/site2/website-next/versioned_docs/version-2.8.0/window-functions-context.md b/site2/website-next/versioned_docs/version-2.8.0/window-functions-context.md index a54fcfc3ac6dd..940c06c574de5 100644 --- a/site2/website-next/versioned_docs/version-2.8.0/window-functions-context.md +++ b/site2/website-next/versioned_docs/version-2.8.0/window-functions-context.md @@ -297,11 +297,10 @@ You can access all logs produced by `LoggingFunction` via the `persistent://publ ## Metrics -Pulsar window functions can publish arbitrary metrics to the metrics interface which can be queried. +Pulsar window functions can publish arbitrary metrics to the metrics interface which can be queried. :::note - If a Pulsar window function uses the language-native interface for Java, that function is not able to publish metrics and stats to Pulsar. ::: @@ -409,7 +408,6 @@ Java SDK context object enables you to access key/value pairs provided to Pulsar :::tip - For all key/value pairs passed to Java window functions, both the `key` and the `value` are `String`. To set the value to be a different type, you need to deserialize it from the `String` type. ::: @@ -422,7 +420,7 @@ bin/pulsar-admin functions create \ --user-config '{"word-of-the-day":"verdure"}' \ # Other function configs - ``` +``` This example accesses values in a Java window function. diff --git a/site2/website/data/team.js b/site2/website/data/team.js index 2253e7ccaa893..9b5e7a0d2c9ec 100644 --- a/site2/website/data/team.js +++ b/site2/website/data/team.js @@ -45,6 +45,11 @@ module.exports = { apacheId: 'wave', roles: 'Committer, PMC' }, + { + name: 'David Kjerrumgaard', + apacheId: 'david-streamlio', + roles: 'Committer' + }, { name: 'Enrico Olivelli', apacheId: 'eolivelli', diff --git a/site2/website/pages/en/events.js b/site2/website/pages/en/events.js index 6059e7346f614..9310a377d6140 100644 --- a/site2/website/pages/en/events.js +++ b/site2/website/pages/en/events.js @@ -34,10 +34,13 @@ class Events extends React.Component {

    Events

    - - [Pulsar Summit Asia 2020](https://pulsar-summit.org/en/event/asia-2020) + - [Pulsar Summit Europe 2021](https://pulsar-summit.org/en/event/europe-2021) 6 October 2021 - - [Pulsar Summit Virtual Conference](https://www.youtube.com/playlist?list=PLqRma1oIkcWjVlPfaWlf3VO9W-XWsF_4-) + - [Pulsar Summit Asia 2021](https://pulsar-summit.org/en/event/asia-2021) 20-21 November 2021 + + + - [Pulsar Summit North America 2021](https://pulsar-summit.org/en/event/north-america-2021) 16-17 June 2021 - [ApacheCon @Home](https://www.youtube.com/watch?v=iIABx20uvmw&list=PLU2OcwpQkYCy_awEe5xwlxGTk5UieA37m) @@ -57,13 +60,32 @@ class Events extends React.Component { - [Pulsar Developer Community biweekly meetup](https://github.com/streamnative/pulsar-community-loc-cn/) +

    Groups

    + + - [NorCal Apache Pulsar Neighborhood Meetup Group](https://www.meetup.com/nor-cal-apache-pulsar-meetup-group/) + + + - [Netherlands Apache Pulsar Meetup Group](https://www.meetup.com/netherlands-apache-pulsar-meetup/) + + + - [SoCal Apache Pulsar Neighborhood Meetup Group](https://www.meetup.com/socal-apache-pulsar-meetup-group/) + + + - [New York City Apache Pulsar Meetup](https://www.meetup.com/new-york-city-apache-pulsar-meetup/) + + + - [Beijing Apache Pulsar® Meetup by StreamNative](https://www.meetup.com/beijing-apache-pulsar-meetup-by-streamnative/) + - [SF Bay Area Apache Pulsar Meetup](https://www.meetup.com/SF-Bay-Area-Apache-Pulsar-Meetup/) -

    Groups

    - [Japan Pulsar User Group](https://japan-pulsar-user-group.connpass.com/) - +
    +

    Replays

    + + - [Pulsar Summit Virtual Conference](https://www.youtube.com/playlist?list=PLqRma1oIkcWjVlPfaWlf3VO9W-XWsF_4-) 9 September 2020 + diff --git a/site2/website/versioned_docs/version-2.1.0-incubating/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.1.0-incubating/cookbooks-tiered-storage.md index d852d7b8b71bb..7c55b7d50a460 100644 --- a/site2/website/versioned_docs/version-2.1.0-incubating/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.1.0-incubating/cookbooks-tiered-storage.md @@ -132,5 +132,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.1.1-incubating/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.1.1-incubating/cookbooks-tiered-storage.md index 24b6beaf1340c..95e47fefd2a6f 100644 --- a/site2/website/versioned_docs/version-2.1.1-incubating/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.1.1-incubating/cookbooks-tiered-storage.md @@ -132,5 +132,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.2.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.2.0/cookbooks-tiered-storage.md index 406d56452ebff..6e1c78031ffe0 100644 --- a/site2/website/versioned_docs/version-2.2.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.2.0/cookbooks-tiered-storage.md @@ -8,9 +8,7 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. Tiered storage currently uses [Apache Jclouds](https://jclouds.apache.org) to supports -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. ## When should I use Tiered Storage? @@ -217,5 +215,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.4.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.4.0/cookbooks-tiered-storage.md index 4d2555123623a..d35be36d15ce4 100644 --- a/site2/website/versioned_docs/version-2.4.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.4.0/cookbooks-tiered-storage.md @@ -8,9 +8,7 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. Tiered storage currently uses [Apache Jclouds](https://jclouds.apache.org) to supports -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. ## When should I use Tiered Storage? @@ -233,5 +231,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.4.0/reference-metrics.md b/site2/website/versioned_docs/version-2.4.0/reference-metrics.md index 5b7e11af3b2de..f62de8940302c 100644 --- a/site2/website/versioned_docs/version-2.4.0/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.4.0/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -236,6 +237,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.4.1/reference-metrics.md b/site2/website/versioned_docs/version-2.4.1/reference-metrics.md index 8beaecb937c08..426e37f5806cf 100644 --- a/site2/website/versioned_docs/version-2.4.1/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.4.1/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -236,6 +237,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.4.2/reference-metrics.md b/site2/website/versioned_docs/version-2.4.2/reference-metrics.md index b1e999289fbd8..a70759121338f 100644 --- a/site2/website/versioned_docs/version-2.4.2/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.4.2/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -236,6 +237,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.5.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.5.0/cookbooks-tiered-storage.md index d1707f50ddd08..b5db1599333e4 100644 --- a/site2/website/versioned_docs/version-2.5.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.5.0/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -294,5 +290,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.5.0/io-quickstart.md b/site2/website/versioned_docs/version-2.5.0/io-quickstart.md index c4a98e461509d..1608ff5e89a54 100644 --- a/site2/website/versioned_docs/version-2.5.0/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.5.0/io-quickstart.md @@ -416,18 +416,18 @@ This example uses the MySQL 5.7 docker image to start a single-node MySQL cluste #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if MySQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.5.0/reference-metrics.md b/site2/website/versioned_docs/version-2.5.0/reference-metrics.md index e7e9ee72295a3..49a9ff02dbd6d 100644 --- a/site2/website/versioned_docs/version-2.5.0/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.5.0/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -235,6 +236,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.5.1/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.5.1/io-influxdb-sink.md index a152ef96df866..4882bb2020957 100644 --- a/site2/website/versioned_docs/version-2.5.1/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.5.1/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.5.1/io-quickstart.md b/site2/website/versioned_docs/version-2.5.1/io-quickstart.md index c54572f9dac86..c27f745173f8f 100644 --- a/site2/website/versioned_docs/version-2.5.1/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.5.1/io-quickstart.md @@ -416,18 +416,18 @@ This example uses the MySQL 5.7 docker image to start a single-node MySQL cluste #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if MySQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.5.1/reference-metrics.md b/site2/website/versioned_docs/version-2.5.1/reference-metrics.md index 222988fdd2d9f..5738b454b8505 100644 --- a/site2/website/versioned_docs/version-2.5.1/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.5.1/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -236,6 +237,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.5.2/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.5.2/io-influxdb-sink.md index 3da87cb63559d..1ba027a2b3694 100644 --- a/site2/website/versioned_docs/version-2.5.2/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.5.2/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.5.2/io-quickstart.md b/site2/website/versioned_docs/version-2.5.2/io-quickstart.md index 8b64793fae6a9..c6e7ef5855c68 100644 --- a/site2/website/versioned_docs/version-2.5.2/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.5.2/io-quickstart.md @@ -416,18 +416,18 @@ This example uses the MySQL 5.7 docker image to start a single-node MySQL cluste #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-mysql_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _3306_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the root user is _jdbc_.
    - The name for the normal user is _mysqluser_.
    - The password for the normal user is _mysqlpw_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if MySQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.5.2/reference-metrics.md b/site2/website/versioned_docs/version-2.5.2/reference-metrics.md index 3e8812397576c..9fd7dc264ee0d 100644 --- a/site2/website/versioned_docs/version-2.5.2/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.5.2/reference-metrics.md @@ -23,13 +23,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -103,6 +103,7 @@ Broker has the following kinds of metrics: * [Replication metrics](#replication-metrics-1) * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -236,6 +237,35 @@ All the consumer metrics are labelled with the following labels: | pulsar_consumer_msg_throughput_out | Gauge | The total message dispatch throughput for a consumer (bytes/second). | | pulsar_consumer_available_permits | Gauge | The available permits for for a consumer. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Monitor You can [set up a Prometheus instance](https://prometheus.io/) to collect all the metrics exposed at Pulsar components and set up diff --git a/site2/website/versioned_docs/version-2.6.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.6.0/cookbooks-tiered-storage.md index a631f18042caf..4eea2099e1164 100644 --- a/site2/website/versioned_docs/version-2.6.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.6.0/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -297,5 +293,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.6.0/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.6.0/io-influxdb-sink.md index 0056e5acb2993..9bba995efbdd5 100644 --- a/site2/website/versioned_docs/version-2.6.0/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.6.0/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.6.0/io-quickstart.md b/site2/website/versioned_docs/version-2.6.0/io-quickstart.md index b98072940191b..30b23be724fea 100644 --- a/site2/website/versioned_docs/version-2.6.0/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.6.0/io-quickstart.md @@ -416,18 +416,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.6.0/reference-metrics.md b/site2/website/versioned_docs/version-2.6.0/reference-metrics.md index d3f60bd415f89..fad48f41c354a 100644 --- a/site2/website/versioned_docs/version-2.6.0/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.6.0/reference-metrics.md @@ -26,13 +26,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -65,8 +65,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -112,6 +112,7 @@ Broker has the following kinds of metrics: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -253,20 +254,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. - broker: broker=${broker}. ${broker} is the ip address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -276,7 +277,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -345,6 +346,35 @@ All the managed ledger bookie client metrics labelled with the following labels: | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + # Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.6.1/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.6.1/cookbooks-tiered-storage.md index 67331cd09e24b..347904e879624 100644 --- a/site2/website/versioned_docs/version-2.6.1/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.6.1/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -295,5 +291,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.6.1/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.6.1/io-influxdb-sink.md index 721176893f799..bc1383518f43a 100644 --- a/site2/website/versioned_docs/version-2.6.1/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.6.1/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.6.1/io-quickstart.md b/site2/website/versioned_docs/version-2.6.1/io-quickstart.md index 030b248f7125e..52150a8fd471f 100644 --- a/site2/website/versioned_docs/version-2.6.1/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.6.1/io-quickstart.md @@ -412,18 +412,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.6.1/reference-metrics.md b/site2/website/versioned_docs/version-2.6.1/reference-metrics.md index ebd6e4616b55f..deb6d0efefa84 100644 --- a/site2/website/versioned_docs/version-2.6.1/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.6.1/reference-metrics.md @@ -26,13 +26,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -65,8 +65,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -112,6 +112,7 @@ Broker has the following kinds of metrics: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -253,20 +254,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. - broker: broker=${broker}. ${broker} is the ip address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -276,7 +277,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -345,6 +346,35 @@ All the managed ledger bookie client metrics labelled with the following labels: | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + # Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.6.2/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.6.2/cookbooks-tiered-storage.md index 5e83af30e96ea..aaadf25f68327 100644 --- a/site2/website/versioned_docs/version-2.6.2/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.6.2/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -295,5 +291,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.6.2/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.6.2/io-influxdb-sink.md index 8b0cbf8d1e6d3..4cbb30b6de444 100644 --- a/site2/website/versioned_docs/version-2.6.2/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.6.2/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.6.2/io-quickstart.md b/site2/website/versioned_docs/version-2.6.2/io-quickstart.md index ced4a18936c7c..377417a8d3d32 100644 --- a/site2/website/versioned_docs/version-2.6.2/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.6.2/io-quickstart.md @@ -412,18 +412,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.6.2/reference-metrics.md b/site2/website/versioned_docs/version-2.6.2/reference-metrics.md index e1f58788b35ed..766e3caa4313b 100644 --- a/site2/website/versioned_docs/version-2.6.2/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.6.2/reference-metrics.md @@ -26,13 +26,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -65,8 +65,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -112,6 +112,7 @@ Broker has the following kinds of metrics: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -253,20 +254,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. - broker: broker=${broker}. ${broker} is the ip address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -276,7 +277,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -345,6 +346,35 @@ All the managed ledger bookie client metrics labelled with the following labels: | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + # Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.6.3/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.6.3/cookbooks-tiered-storage.md index 43efc69f1a313..18391c897cd26 100644 --- a/site2/website/versioned_docs/version-2.6.3/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.6.3/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -295,5 +291,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.6.3/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.6.3/io-influxdb-sink.md index 6b7895f1b8576..b87d0e8355668 100644 --- a/site2/website/versioned_docs/version-2.6.3/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.6.3/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.6.3/io-quickstart.md b/site2/website/versioned_docs/version-2.6.3/io-quickstart.md index 4cf13e4da2860..e4271a6ce395a 100644 --- a/site2/website/versioned_docs/version-2.6.3/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.6.3/io-quickstart.md @@ -412,18 +412,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.6.3/reference-metrics.md b/site2/website/versioned_docs/version-2.6.3/reference-metrics.md index f0feb88e39041..2ea614e87e303 100644 --- a/site2/website/versioned_docs/version-2.6.3/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.6.3/reference-metrics.md @@ -26,13 +26,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -65,8 +65,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -112,6 +112,7 @@ Broker has the following kinds of metrics: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -253,20 +254,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. - broker: broker=${broker}. ${broker} is the ip address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -276,7 +277,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -345,6 +346,35 @@ All the managed ledger bookie client metrics labelled with the following labels: | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + # Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.6.4/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.6.4/cookbooks-tiered-storage.md index e5bb8751061b0..bac4cc6648c81 100644 --- a/site2/website/versioned_docs/version-2.6.4/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.6.4/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -295,5 +291,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.6.4/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.6.4/io-influxdb-sink.md index 49278918c5555..a345a5a8c9893 100644 --- a/site2/website/versioned_docs/version-2.6.4/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.6.4/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.6.4/io-quickstart.md b/site2/website/versioned_docs/version-2.6.4/io-quickstart.md index 07de17573a399..38853f831708a 100644 --- a/site2/website/versioned_docs/version-2.6.4/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.6.4/io-quickstart.md @@ -412,18 +412,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.6.4/reference-metrics.md b/site2/website/versioned_docs/version-2.6.4/reference-metrics.md index 9804a138ccf47..933f11f91a61b 100644 --- a/site2/website/versioned_docs/version-2.6.4/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.6.4/reference-metrics.md @@ -26,13 +26,13 @@ The metrics exposed by Pulsar are in Prometheus format. The types of metrics are - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter whose value can only increase or be reset to zero on restart. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a *gauge* is a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper The ZooKeeper metrics are exposed under "/metrics" at port 8000. You can use a different port -by configuring the `stats_server_port` system property. +by configuring the `stats_server_port` system property. ### Server metrics @@ -65,8 +65,8 @@ in `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Histogram | The histogram of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Histogram | The histogram of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Histogram | The histogram of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Histogram | The histogram of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -112,6 +112,7 @@ Broker has the following kinds of metrics: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -253,20 +254,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. - broker: broker=${broker}. ${broker} is the ip address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -276,7 +277,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you configured in broker.conf. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -345,6 +346,35 @@ All the managed ledger bookie client metrics labelled with the following labels: | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + # Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.6.4/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.6.4/tiered-storage-azure.md index e641e86693d5b..9398661299c59 100644 --- a/site2/website/versioned_docs/version-2.6.4/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.6.4/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.7.0/cookbooks-tiered-storage.md index 3d63f20c7820c..9b5e93cc7e255 100644 --- a/site2/website/versioned_docs/version-2.7.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.7.0/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -297,5 +293,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.7.0/functions-package.md b/site2/website/versioned_docs/version-2.7.0/functions-package.md index 5a9275626b755..1e7a52193dd4a 100644 --- a/site2/website/versioned_docs/version-2.7.0/functions-package.md +++ b/site2/website/versioned_docs/version-2.7.0/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.7.0/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.7.0/io-influxdb-sink.md index a23ee219f1115..d075cac341db2 100644 --- a/site2/website/versioned_docs/version-2.7.0/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.7.0/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.7.0/io-quickstart.md b/site2/website/versioned_docs/version-2.7.0/io-quickstart.md index 0ba4a3f9b7d7b..f4718dbe6e6a0 100644 --- a/site2/website/versioned_docs/version-2.7.0/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.7.0/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.7.0/reference-metrics.md b/site2/website/versioned_docs/version-2.7.0/reference-metrics.md index cd9731e7804d8..f0d12e5d9c54b 100644 --- a/site2/website/versioned_docs/version-2.7.0/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.7.0/reference-metrics.md @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -108,6 +108,7 @@ The following metrics are available for broker: * [Subscription metrics](#subscription-metrics) * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -249,20 +250,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -272,7 +273,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -340,6 +341,35 @@ All the managed ledger bookie client metrics are labelled with the following lab | pulsar_managedLedger_client_bookkeeper_ml_workers_task_execution | Summary | The worker task execution latency calculated in milliseconds. | | pulsar_managedLedger_client_bookkeeper_ml_workers_task_queued | Summary | The worker task queued latency calculated in milliseconds. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -401,4 +431,4 @@ All the proxy metrics are labelled with the following labels: | split_read_latency_per_query | Summary | Total read latency per query. | | split_record_deserialize_time | Summary | Time spent on deserializing message to record. For example, Avro, JSON, and so on. | | split_record_deserialize_time_per_query | Summary | Time spent on deserializing message to record per query. | -| split_total_execution_time | Summary | Total execution time . | \ No newline at end of file +| split_total_execution_time | Summary | Total execution time . | diff --git a/site2/website/versioned_docs/version-2.7.0/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.7.0/tiered-storage-aws.md index 20c25c6c9624d..bcda87ca96ab3 100644 --- a/site2/website/versioned_docs/version-2.7.0/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.7.0/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.0/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.7.0/tiered-storage-azure.md index 4b6d354ec7eb1..bd822b773fdc0 100644 --- a/site2/website/versioned_docs/version-2.7.0/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.7.0/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.0/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.7.0/tiered-storage-filesystem.md index e2a8b1e0184c0..45f740bb69f15 100644 --- a/site2/website/versioned_docs/version-2.7.0/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.7.0/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.0/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.7.0/tiered-storage-gcs.md index df2c44cb076fe..e273964804a8e 100644 --- a/site2/website/versioned_docs/version-2.7.0/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.7.0/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.0/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.7.0/tiered-storage-overview.md index 97313dfe646db..b36834e7fea5e 100644 --- a/site2/website/versioned_docs/version-2.7.0/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.7.0/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.1/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.7.1/cookbooks-tiered-storage.md index ebcbc6caf97ef..0d09de6b40629 100644 --- a/site2/website/versioned_docs/version-2.7.1/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.7.1/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -297,5 +293,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.7.1/functions-package.md b/site2/website/versioned_docs/version-2.7.1/functions-package.md index ef2d9dd8b518e..b60e5477d4eab 100644 --- a/site2/website/versioned_docs/version-2.7.1/functions-package.md +++ b/site2/website/versioned_docs/version-2.7.1/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.7.1/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.7.1/io-influxdb-sink.md index 0dfd0358db8d0..d94a8b61c194e 100644 --- a/site2/website/versioned_docs/version-2.7.1/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.7.1/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.7.1/io-quickstart.md b/site2/website/versioned_docs/version-2.7.1/io-quickstart.md index 145deceaf308d..379a0c8a90777 100644 --- a/site2/website/versioned_docs/version-2.7.1/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.7.1/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.7.1/reference-metrics.md b/site2/website/versioned_docs/version-2.7.1/reference-metrics.md index 0aa3ca92e08fe..b2fb2e49c6fc2 100644 --- a/site2/website/versioned_docs/version-2.7.1/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.7.1/reference-metrics.md @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -109,7 +109,8 @@ The following metrics are available for broker: * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) * [Token metrics](#token-metrics) -* [Authentication metrics](#authentication-metrics) +* [Authentication metrics](#authentication-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -251,20 +252,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -274,7 +275,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -367,6 +368,35 @@ All the authentication metrics are labelled with the following labels: | pulsar_authentication_success_count| Counter | The number of successful authentication operations. | | pulsar_authentication_failures_count | Counter | The number of failing authentication operations. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.7.1/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.7.1/tiered-storage-aws.md index 76b0fa499f85e..5b68d3a454223 100644 --- a/site2/website/versioned_docs/version-2.7.1/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.7.1/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.1/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.7.1/tiered-storage-azure.md index 8a22dad367d49..0dbc89eecd89f 100644 --- a/site2/website/versioned_docs/version-2.7.1/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.7.1/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.1/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.7.1/tiered-storage-filesystem.md index 041ac6b05a2ba..dddfd0502c9b9 100644 --- a/site2/website/versioned_docs/version-2.7.1/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.7.1/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.1/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.7.1/tiered-storage-gcs.md index 93e958e580b52..050bb65182dfa 100644 --- a/site2/website/versioned_docs/version-2.7.1/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.7.1/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.1/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.7.1/tiered-storage-overview.md index b9d0f11c5413a..d5c06cb1a9c16 100644 --- a/site2/website/versioned_docs/version-2.7.1/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.7.1/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.2/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.7.2/cookbooks-tiered-storage.md index adffcf6fe4f70..5b7cb2b398412 100644 --- a/site2/website/versioned_docs/version-2.7.2/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.7.2/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -297,5 +293,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.7.2/functions-package.md b/site2/website/versioned_docs/version-2.7.2/functions-package.md index 2f5406b0efbef..599a99d0f0e87 100644 --- a/site2/website/versioned_docs/version-2.7.2/functions-package.md +++ b/site2/website/versioned_docs/version-2.7.2/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.7.2/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.7.2/io-influxdb-sink.md index 652aa3ad27401..dde82a4acdfc4 100644 --- a/site2/website/versioned_docs/version-2.7.2/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.7.2/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.7.2/io-quickstart.md b/site2/website/versioned_docs/version-2.7.2/io-quickstart.md index 28ca6b053654f..6e88e5df0c634 100644 --- a/site2/website/versioned_docs/version-2.7.2/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.7.2/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.7.2/reference-metrics.md b/site2/website/versioned_docs/version-2.7.2/reference-metrics.md index 0b2dada20c2eb..79b05e61be30a 100644 --- a/site2/website/versioned_docs/version-2.7.2/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.7.2/reference-metrics.md @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -109,7 +109,8 @@ The following metrics are available for broker: * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) * [Token metrics](#token-metrics) -* [Authentication metrics](#authentication-metrics) +* [Authentication metrics](#authentication-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -251,20 +252,20 @@ All the managedLedger metrics are labelled with the following labels: All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -274,7 +275,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -367,6 +368,35 @@ All the authentication metrics are labelled with the following labels: | pulsar_authentication_success_count| Counter | The number of successful authentication operations. | | pulsar_authentication_failures_count | Counter | The number of failing authentication operations. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.7.2/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.7.2/tiered-storage-aws.md index 9c458e13f5522..b9e27d0cb4033 100644 --- a/site2/website/versioned_docs/version-2.7.2/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.7.2/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.2/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.7.2/tiered-storage-azure.md index 97b02f097e862..f08bfc2e0b495 100644 --- a/site2/website/versioned_docs/version-2.7.2/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.7.2/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.2/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.7.2/tiered-storage-filesystem.md index 1aa50ad0f2195..951ef08aca808 100644 --- a/site2/website/versioned_docs/version-2.7.2/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.7.2/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.2/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.7.2/tiered-storage-gcs.md index 1eef7af16bf78..65f39abc57923 100644 --- a/site2/website/versioned_docs/version-2.7.2/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.7.2/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.2/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.7.2/tiered-storage-overview.md index 26a2a276e9541..29e3ced80bd9a 100644 --- a/site2/website/versioned_docs/version-2.7.2/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.7.2/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md index 97a05128cad7c..cf216716edb63 100644 --- a/site2/website/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.7.3/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -297,5 +293,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.7.3/functions-package.md b/site2/website/versioned_docs/version-2.7.3/functions-package.md index f26fe0e33ea25..7facbb432167b 100644 --- a/site2/website/versioned_docs/version-2.7.3/functions-package.md +++ b/site2/website/versioned_docs/version-2.7.3/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.7.3/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.7.3/io-influxdb-sink.md index 96d9c7c2f19eb..b73700ebdf920 100644 --- a/site2/website/versioned_docs/version-2.7.3/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.7.3/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.7.3/io-quickstart.md b/site2/website/versioned_docs/version-2.7.3/io-quickstart.md index b2b659462fcc3..e54574adf4c7f 100644 --- a/site2/website/versioned_docs/version-2.7.3/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.7.3/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.7.3/reference-metrics.md b/site2/website/versioned_docs/version-2.7.3/reference-metrics.md index 60c26903f9847..8a48185a654be 100644 --- a/site2/website/versioned_docs/version-2.7.3/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.7.3/reference-metrics.md @@ -24,12 +24,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `stats_server_port` system property. ### Server metrics @@ -62,8 +62,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -109,7 +109,8 @@ The following metrics are available for broker: * [Consumer metrics](#consumer-metrics) * [ManagedLedger bookie client metrics](#managed-ledger-bookie-client-metrics) * [Token metrics](#token-metrics) -* [Authentication metrics](#authentication-metrics) +* [Authentication metrics](#authentication-metrics) +* [Jetty metrics](#jetty-metrics) ### Namespace metrics @@ -271,20 +272,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -294,7 +295,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -387,6 +388,35 @@ All the authentication metrics are labelled with the following labels: | pulsar_authentication_success_count| Counter | The number of successful authentication operations. | | pulsar_authentication_failures_count | Counter | The number of failing authentication operations. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: diff --git a/site2/website/versioned_docs/version-2.7.3/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.7.3/tiered-storage-aws.md index 782d4beac4b79..9229056f38c79 100644 --- a/site2/website/versioned_docs/version-2.7.3/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.7.3/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.3/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.7.3/tiered-storage-azure.md index f94d4c3372b33..3562de8c1e136 100644 --- a/site2/website/versioned_docs/version-2.7.3/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.7.3/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.3/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.7.3/tiered-storage-filesystem.md index 95682cb0ce44b..a178ed37fae33 100644 --- a/site2/website/versioned_docs/version-2.7.3/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.7.3/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.3/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.7.3/tiered-storage-gcs.md index 463d231a9e069..801cd3fcc3139 100644 --- a/site2/website/versioned_docs/version-2.7.3/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.7.3/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.7.3/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.7.3/tiered-storage-overview.md index 55f4df6569a13..7822ef0b7a564 100644 --- a/site2/website/versioned_docs/version-2.7.3/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.7.3/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.0/administration-geo.md b/site2/website/versioned_docs/version-2.8.0/administration-geo.md index 88806c4f118e6..97ddaee19e312 100644 --- a/site2/website/versioned_docs/version-2.8.0/administration-geo.md +++ b/site2/website/versioned_docs/version-2.8.0/administration-geo.md @@ -60,12 +60,12 @@ Suppose that you have 3 replication clusters: `us-west`, `us-cent`, and `us-east Run the following command on `us-west`. -```shell -$ bin/pulsar-admin clusters create \ - --broker-url pulsar://: \ - --url http://: \ - us-east -``` + ```shell + $ bin/pulsar-admin clusters create \ + --broker-url pulsar://: \ + --url http://: \ + us-east + ``` > #### Tip > @@ -76,12 +76,12 @@ $ bin/pulsar-admin clusters create \ Run the following command on `us-west`. -```shell -$ bin/pulsar-admin clusters create \ - --broker-url pulsar://: \ - --url http://: \ - us-cent -``` + ```shell + $ bin/pulsar-admin clusters create \ + --broker-url pulsar://: \ + --url http://: \ + us-cent + ``` 3. Run similar commands on `us-east` and `us-cent` to create connections among clusters. diff --git a/site2/website/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md index 96e8117724d39..82ec830266c92 100644 --- a/site2/website/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.8.0/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -307,5 +303,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.8.0/functions-package.md b/site2/website/versioned_docs/version-2.8.0/functions-package.md index 151288e5847a4..da8d801cf4786 100644 --- a/site2/website/versioned_docs/version-2.8.0/functions-package.md +++ b/site2/website/versioned_docs/version-2.8.0/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.8.0/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.8.0/io-influxdb-sink.md index ed9abf410dace..d1e151b4f1f79 100644 --- a/site2/website/versioned_docs/version-2.8.0/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.8.0/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.8.0/io-quickstart.md b/site2/website/versioned_docs/version-2.8.0/io-quickstart.md index 1810f71152c13..979b32f190baf 100644 --- a/site2/website/versioned_docs/version-2.8.0/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.8.0/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.8.0/reference-metrics.md b/site2/website/versioned_docs/version-2.8.0/reference-metrics.md index 6d871347694b9..324b453097b0e 100644 --- a/site2/website/versioned_docs/version-2.8.0/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.8.0/reference-metrics.md @@ -25,12 +25,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -120,6 +120,7 @@ The following metrics are available for broker: - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -285,20 +286,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -308,7 +309,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -419,6 +420,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -471,10 +501,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/website/versioned_docs/version-2.8.0/reference-pulsar-admin.md b/site2/website/versioned_docs/version-2.8.0/reference-pulsar-admin.md index b904d1cc79899..fb260ed9a62ac 100644 --- a/site2/website/versioned_docs/version-2.8.0/reference-pulsar-admin.md +++ b/site2/website/versioned_docs/version-2.8.0/reference-pulsar-admin.md @@ -2456,7 +2456,6 @@ $ pulsar-admin topics remove-deduplication tenant/namespace/topic ``` -``` ## `tenants` Operations for managing tenants diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-aliyun.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-aliyun.md index 488486098387d..39c194865a1d1 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-aliyun.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-aliyun.md @@ -23,20 +23,20 @@ This example uses Pulsar 2.8.0. 2. Download and untar the Pulsar offloaders package, then copy the Pulsar offloaders as `offloaders` in the Pulsar directory, see [here](https://pulsar.apache.org/docs/en/standalone/#install-tiered-storage-offloaders-optional). - **Output** - - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - - ``` - tiered-storage-file-system-2.8.0.nar - tiered-storage-jcloud-2.8.0.nar - ``` - - > **Note** - > - > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - > - > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + **Output** + + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. + + ``` + tiered-storage-file-system-2.8.0.nar + tiered-storage-jcloud-2.8.0.nar + ``` + + > **Note** + > + > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + > + > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. ## Configuration @@ -211,7 +211,7 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > **Tip** > diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-aws.md index e319985ebc687..c580f9a2edf97 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-azure.md index a1a19f538804e..72a5969831c0f 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-filesystem.md index 3a4ef8d495c14..a8c6e53b035f6 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-gcs.md index 2c8ee9632fa91..5cb5105f225ca 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.0/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.8.0/tiered-storage-overview.md index 55bc6864ef93b..fb87b49adea90 100644 --- a/site2/website/versioned_docs/version-2.8.0/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.8.0/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.1/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.8.1/cookbooks-tiered-storage.md index 6a3d538706e97..70dffe108a1e2 100644 --- a/site2/website/versioned_docs/version-2.8.1/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.8.1/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -307,5 +303,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.8.1/functions-package.md b/site2/website/versioned_docs/version-2.8.1/functions-package.md index 236bc3d305639..1ed17aa9c596a 100644 --- a/site2/website/versioned_docs/version-2.8.1/functions-package.md +++ b/site2/website/versioned_docs/version-2.8.1/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.8.1/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.8.1/io-influxdb-sink.md index 8a0ed899afb77..3270382e67d04 100644 --- a/site2/website/versioned_docs/version-2.8.1/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.8.1/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.8.1/io-quickstart.md b/site2/website/versioned_docs/version-2.8.1/io-quickstart.md index 4d7773b16577b..a7c6be9cabca7 100644 --- a/site2/website/versioned_docs/version-2.8.1/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.8.1/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. 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 f81378d23d37d..bbcb80fcfacbc 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 @@ -25,12 +25,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -120,6 +120,7 @@ The following metrics are available for broker: - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -288,20 +289,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -311,7 +312,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -422,6 +423,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -474,10 +504,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/website/versioned_docs/version-2.8.1/reference-pulsar-admin.md b/site2/website/versioned_docs/version-2.8.1/reference-pulsar-admin.md index 41e5427b806d2..0ca39092a71c7 100644 --- a/site2/website/versioned_docs/version-2.8.1/reference-pulsar-admin.md +++ b/site2/website/versioned_docs/version-2.8.1/reference-pulsar-admin.md @@ -2459,7 +2459,6 @@ $ pulsar-admin topics remove-deduplication tenant/namespace/topic ``` -``` ## `tenants` Operations for managing tenants diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-aliyun.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-aliyun.md index 0c525e927cd35..d7cf3e828c507 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-aliyun.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-aliyun.md @@ -23,20 +23,20 @@ This example uses Pulsar 2.8.0. 2. Download and untar the Pulsar offloaders package, then copy the Pulsar offloaders as `offloaders` in the Pulsar directory, see [here](https://pulsar.apache.org/docs/en/standalone/#install-tiered-storage-offloaders-optional). - **Output** - - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - - ``` - tiered-storage-file-system-2.8.0.nar - tiered-storage-jcloud-2.8.0.nar - ``` - - > **Note** - > - > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - > - > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + **Output** + + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. + + ``` + tiered-storage-file-system-2.8.0.nar + tiered-storage-jcloud-2.8.0.nar + ``` + + > **Note** + > + > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + > + > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. ## Configuration @@ -211,7 +211,7 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > **Tip** > diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-aws.md index 399d76dc3b834..8574d99c409b8 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-azure.md index 04a64b28d311f..8efe825be43a0 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-filesystem.md index 02318288acefa..5b6cfa06f65ae 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-gcs.md index 210c5555c74c0..8bcc6cc255114 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.1/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.8.1/tiered-storage-overview.md index f6a5f298254d8..4195afd0125e1 100644 --- a/site2/website/versioned_docs/version-2.8.1/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.8.1/tiered-storage-overview.md @@ -7,11 +7,9 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. - With jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. + With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.2/cookbooks-tiered-storage.md b/site2/website/versioned_docs/version-2.8.2/cookbooks-tiered-storage.md index 52f3eec7e9460..c328673ca93f4 100644 --- a/site2/website/versioned_docs/version-2.8.2/cookbooks-tiered-storage.md +++ b/site2/website/versioned_docs/version-2.8.2/cookbooks-tiered-storage.md @@ -7,13 +7,9 @@ original_id: cookbooks-tiered-storage Pulsar's **Tiered Storage** feature allows older backlog data to be offloaded to long term storage, thereby freeing up space in BookKeeper and reducing storage costs. This cookbook walks you through using tiered storage in your Pulsar cluster. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) -for long term storage. With Jclouds, it is easy to add support for more -[cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [Google Cloud Storage](https://cloud.google.com/storage/)(GCS for short) for long term storage. With Jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. -* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. -With Hadoop, it is easy to add support for more filesystem in the future. +* Tiered storage uses [Apache Hadoop](http://hadoop.apache.org/) to support filesystem for long term storage. With Hadoop, it is easy to add support for more filesystem in the future. ## When should I use Tiered Storage? @@ -307,5 +303,5 @@ Error in offload null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= -```` +``` diff --git a/site2/website/versioned_docs/version-2.8.2/functions-package.md b/site2/website/versioned_docs/version-2.8.2/functions-package.md index 03b5cb654434a..d2083ad196fdd 100644 --- a/site2/website/versioned_docs/version-2.8.2/functions-package.md +++ b/site2/website/versioned_docs/version-2.8.2/functions-package.md @@ -201,7 +201,7 @@ To package a function with **one python file** in Python, complete the following The implementation of a Python function depends on the Python client, so before deploying a Python function, you need to install the corresponding version of the Python client. ```bash - pip install python-client==2.6.0 + pip install pulsar-client==2.6.0 ``` 3. Run the Python Function. @@ -217,7 +217,7 @@ To package a function with **one python file** in Python, complete the following ```bash ./bin/pulsar-admin functions localrun \ - --classname org.example.test.ExclamationFunction \ + --classname . \ --py \ --inputs persistent://public/default/my-topic-1 \ --output persistent://public/default/test-1 \ diff --git a/site2/website/versioned_docs/version-2.8.2/io-influxdb-sink.md b/site2/website/versioned_docs/version-2.8.2/io-influxdb-sink.md index 30a1fe3b3f705..4141574c9b486 100644 --- a/site2/website/versioned_docs/version-2.8.2/io-influxdb-sink.md +++ b/site2/website/versioned_docs/version-2.8.2/io-influxdb-sink.md @@ -62,7 +62,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:9999" organization: "example-org" bucket: "example-bucket" @@ -72,7 +72,6 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` #### InfluxDBv1 @@ -95,7 +94,7 @@ Before using the InfluxDB sink connector, you need to create a configuration fil * YAML ```yaml - { + configs: influxdbUrl: "http://localhost:8086" database: "test_db" consistencyLevel: "ONE" @@ -104,5 +103,4 @@ Before using the InfluxDB sink connector, you need to create a configuration fil gzipEnable: false batchTimeMs: 1000 batchSize: 100 - } ``` diff --git a/site2/website/versioned_docs/version-2.8.2/io-quickstart.md b/site2/website/versioned_docs/version-2.8.2/io-quickstart.md index f0fa76fc04408..70f59655ff793 100644 --- a/site2/website/versioned_docs/version-2.8.2/io-quickstart.md +++ b/site2/website/versioned_docs/version-2.8.2/io-quickstart.md @@ -415,18 +415,18 @@ This example uses the PostgreSQL 12 docker image to start a single-node PostgreS #### Tip - Flag | Description | This example - ---|---|---| - `-d` | To start a container in detached mode. | / - `-it` | Keep STDIN open even if not attached and allocate a terminal. | / - `--rm` | Remove the container automatically when it exits. | / - `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. - `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. - `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. - - > #### Tip - > - > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). + Flag | Description | This example + ---|---|---| + `-d` | To start a container in detached mode. | / + `-it` | Keep STDIN open even if not attached and allocate a terminal. | / + `--rm` | Remove the container automatically when it exits. | / + `-name` | Assign a name to the container. | This example specifies _pulsar-postgres_ for the container. + `-p` | Publish the port of the container to the host. | This example publishes the port _5432_ of the container to the host. + `-e` | Set environment variables. | This example sets the following variables:
    - The password for the user is _password_.
    - The name for the user is _postgres_. + + > #### Tip + > + > For more information about Docker commands, see [Docker CLI](https://docs.docker.com/engine/reference/commandline/run/). 3. Check if PostgreSQL has been started successfully. diff --git a/site2/website/versioned_docs/version-2.8.2/reference-metrics.md b/site2/website/versioned_docs/version-2.8.2/reference-metrics.md index 278c20f37b846..c4314a4c392aa 100644 --- a/site2/website/versioned_docs/version-2.8.2/reference-metrics.md +++ b/site2/website/versioned_docs/version-2.8.2/reference-metrics.md @@ -25,12 +25,12 @@ The following types of metrics are available: - [Counter](https://prometheus.io/docs/concepts/metric_types/#counter): a cumulative metric that represents a single monotonically increasing counter. The value increases by default. You can reset the value to zero or restart your cluster. - [Gauge](https://prometheus.io/docs/concepts/metric_types/#gauge): a metric that represents a single numerical value that can arbitrarily go up and down. -- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. +- [Histogram](https://prometheus.io/docs/concepts/metric_types/#histogram): a histogram samples observations (usually things like request durations or response sizes) and counts them in configurable buckets. The `_bucket` suffix is the number of observations within a histogram bucket, configured with parameter `{le=""}`. The `_count` suffix is the number of observations, shown as a time series and behaves like a counter. The `_sum` suffix is the sum of observed values, also shown as a time series and behaves like a counter. These suffixes are together denoted by `_*` in this doc. - [Summary](https://prometheus.io/docs/concepts/metric_types/#summary): similar to a histogram, a summary samples observations (usually things like request durations and response sizes). While it also provides a total count of observations and a sum of all observed values, it calculates configurable quantiles over a sliding time window. ## ZooKeeper -The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. +The ZooKeeper metrics are exposed under "/metrics" at port `8000`. You can use a different port by configuring the `metricsProvider.httpPort` in conf/zookeeper.conf. ### Server metrics @@ -63,8 +63,8 @@ in the `bookkeeper.conf` configuration file. | bookkeeper_server_READ_ENTRY_count | Counter | The total number of READ_ENTRY requests received at the bookie. The `success` label is used to distinguish successes and failures. | | bookie_WRITE_BYTES | Counter | The total number of bytes written to the bookie. | | bookie_READ_BYTES | Counter | The total number of bytes read from the bookie. | -| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | -| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_ADD_ENTRY_REQUEST | Summary | The summary of request latency of ADD_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | +| bookkeeper_server_READ_ENTRY_REQUEST | Summary | The summary of request latency of READ_ENTRY requests at the bookie. The `success` label is used to distinguish successes and failures. | ### Journal metrics @@ -120,6 +120,7 @@ The following metrics are available for broker: - [Token metrics](#token-metrics) - [Authentication metrics](#authentication-metrics) - [Connection metrics](#connection-metrics) + - [Jetty metrics](#jetty-metrics) - [Pulsar Functions](#pulsar-functions) - [Proxy](#proxy) - [Pulsar SQL Worker](#pulsar-sql-worker) @@ -297,20 +298,20 @@ brk_ml_cursor_nonContiguousDeletedMessagesRange(namespace="", ledger_name="", cu All the loadbalancing metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. - broker: broker=${broker}. ${broker} is the IP address of the broker -- metric: metric="loadBalancing". +- metric: metric="loadBalancing". | Name | Type | Description | | --- | --- | --- | -| pulsar_lb_bandwidth_in_usage | Gauge | The broker bandwith in usage | -| pulsar_lb_bandwidth_out_usage | Gauge | The broker bandwith out usage | -| pulsar_lb_cpu_usage | Gauge | The broker cpu usage | -| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage | -| pulsar_lb_memory_usage | Gauge | The broker process memory usage | +| pulsar_lb_bandwidth_in_usage | Gauge | The broker inbound bandwith usage (in percent). | +| pulsar_lb_bandwidth_out_usage | Gauge | The broker outbound bandwith usage (in percent). | +| pulsar_lb_cpu_usage | Gauge | The broker cpu usage (in percent). | +| pulsar_lb_directMemory_usage | Gauge | The broker process direct memory usage (in percent). | +| pulsar_lb_memory_usage | Gauge | The broker process memory usage (in percent). | #### BundleUnloading metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundleUnloading". +- metric: metric="bundleUnloading". | Name | Type | Description | | --- | --- | --- | @@ -320,7 +321,7 @@ All the bundleUnloading metrics are labelled with the following labels: #### BundleSplit metrics All the bundleUnloading metrics are labelled with the following labels: - cluster: cluster=${pulsar_cluster}. ${pulsar_cluster} is the cluster name that you have configured in the `broker.conf` file. -- metric: metric="bundlesSplit". +- metric: metric="bundlesSplit". | Name | Type | Description | | --- | --- | --- | @@ -431,6 +432,35 @@ All the connection metrics are labelled with the following labels: | pulsar_broker_throttled_connections | Gauge | The number of throttled connections. | | pulsar_broker_throttled_connections_global_limit | Gauge | The number of throttled connections because of per-connection limit. | +### Jetty metrics + +> For a functions-worker running separately from brokers, its Jetty metrics are only exposed when `includeStandardPrometheusMetrics` is set to `true`. + +All the jetty metrics are labelled with the following labels: + +- *cluster*: `cluster=${pulsar_cluster}`. `${pulsar_cluster}` is the cluster name that you have configured in the `broker.conf` file. + +| Name | Type | Description | +|---|---|---| +| jetty_requests_total | Counter | Number of requests. | +| jetty_requests_active | Gauge | Number of requests currently active. | +| jetty_requests_active_max | Gauge | Maximum number of requests that have been active at once. | +| jetty_request_time_max_seconds | Gauge | Maximum time spent handling requests. | +| jetty_request_time_seconds_total | Counter | Total time spent in all request handling. | +| jetty_dispatched_total | Counter | Number of dispatches. | +| jetty_dispatched_active | Gauge | Number of dispatches currently active. | +| jetty_dispatched_active_max | Gauge | Maximum number of active dispatches being handled. | +| jetty_dispatched_time_max | Gauge | Maximum time spent in dispatch handling. | +| jetty_dispatched_time_seconds_total | Counter | Total time spent in dispatch handling. | +| jetty_async_requests_total | Counter | Total number of async requests. | +| jetty_async_requests_waiting | Gauge | Currently waiting async requests. | +| jetty_async_requests_waiting_max | Gauge | Maximum number of waiting async requests. | +| jetty_async_dispatches_total | Counter | Number of requested that have been asynchronously dispatched. | +| jetty_expires_total | Counter | Number of async requests requests that have expired. | +| jetty_responses_total | Counter | Number of responses, labeled by status code. The `code` label can be "1xx", "2xx", "3xx", "4xx", or "5xx". | +| jetty_stats_seconds | Gauge | Time in seconds stats have been collected for. | +| jetty_responses_bytes_total | Counter | Total number of bytes across all responses. | + ## Pulsar Functions All the Pulsar Functions metrics are labelled with the following labels: @@ -483,10 +513,10 @@ Connector metrics contain **source** metrics and **sink** metrics. | Name | Type | Description | |---|---|---| - pulsar_sink_written_total|Counter| The total number of records processed by a sink. + pulsar_sink_written_total|Counter| The total number of records processed by a sink. pulsar_sink_written_total_1min|Counter| The total number of records processed by a sink in the last 1 minute. - pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. - pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. + pulsar_sink_received_total_1min|Counter| The total number of messages that a sink has received from Pulsar topics in the last 1 minute. + pulsar_sink_received_total|Counter| The total number of records that a sink has received from Pulsar topics. pulsar_sink_last_invocation|Gauge|The timestamp of the last invocation of the sink. pulsar_sink_sink_exception|Gauge|The exception from a sink. pulsar_sink_sink_exceptions_total|Counter|The total number of sink exceptions. diff --git a/site2/website/versioned_docs/version-2.8.2/reference-pulsar-admin.md b/site2/website/versioned_docs/version-2.8.2/reference-pulsar-admin.md index f4f52781a13b3..f030d3c55b1c8 100644 --- a/site2/website/versioned_docs/version-2.8.2/reference-pulsar-admin.md +++ b/site2/website/versioned_docs/version-2.8.2/reference-pulsar-admin.md @@ -2459,8 +2459,6 @@ $ pulsar-admin topics remove-deduplication tenant/namespace/topic ``` -``` - ## `tenants` Operations for managing tenants diff --git a/site2/website/versioned_docs/version-2.8.2/security-encryption.md b/site2/website/versioned_docs/version-2.8.2/security-encryption.md index 6364381ca41b8..6d1f9d3538f89 100644 --- a/site2/website/versioned_docs/version-2.8.2/security-encryption.md +++ b/site2/website/versioned_docs/version-2.8.2/security-encryption.md @@ -40,63 +40,151 @@ openssl ec -in test_ecdsa_privkey.pem -pubout -outform pem -out test_ecdsa_pubke 4. Add encryption key name to producer builder: PulsarClient.newProducer().addEncryptionKey("myapp.key"). -5. Add CryptoKeyReader implementation to producer or consumer builder: PulsarClient.newProducer().cryptoKeyReader(keyReader) / PulsarClient.newConsumer().cryptoKeyReader(keyReader). +5. Configure a `CryptoKeyReader` to a producer, consumer or reader. -6. Sample producer application: + + ```java -class RawFileKeyReader implements CryptoKeyReader { +PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); +String topic = "persistent://my-tenant/my-ns/my-topic"; +// RawFileKeyReader is just an example implementation that's not provided by Pulsar +CryptoKeyReader keyReader = new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem"); + +Producer producer = pulsarClient.newProducer() + .topic(topic) + .cryptoKeyReader(keyReader) + .addEncryptionKey(“myappkey”) + .create(); + +Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName("my-subscriber-name") + .cryptoKeyReader(keyReader) + .subscribe(); + +Reader reader = pulsarClient.newReader() + .topic(topic) + .startMessageId(MessageId.earliest) + .cryptoKeyReader(keyReader) + .create(); +``` - String publicKeyFile = ""; - String privateKeyFile = ""; + + +```c++ +Client client("pulsar://localhost:6650"); +std::string topic = "persistent://my-tenant/my-ns/my-topic"; +// DefaultCryptoKeyReader is a built-in implementation that reads public key and private key from files +auto keyReader = std::make_shared("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem"); + +Producer producer; +ProducerConfiguration producerConf; +producerConf.setCryptoKeyReader(keyReader); +producerConf.addEncryptionKey("myappkey"); +client.createProducer(topic, producerConf, producer); + +Consumer consumer; +ConsumerConfiguration consumerConf; +consumerConf.setCryptoKeyReader(keyReader); +client.subscribe(topic, "my-subscriber-name", consumerConf, consumer); + +Reader reader; +ReaderConfiguration readerConf; +readerConf.setCryptoKeyReader(keyReader); +client.createReader(topic, MessageId::earliest(), readerConf, reader); +``` - RawFileKeyReader(String pubKeyFile, String privKeyFile) { - publicKeyFile = pubKeyFile; - privateKeyFile = privKeyFile; - } + - @Override - public EncryptionKeyInfo getPublicKey(String keyName, Map keyMeta) { - EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); - try { - keyInfo.setKey(Files.readAllBytes(Paths.get(publicKeyFile))); - } catch (IOException e) { - System.out.println("ERROR: Failed to read public key from file " + publicKeyFile); - e.printStackTrace(); - } - return keyInfo; - } +```python +from pulsar import Client, CryptoKeyReader - @Override - public EncryptionKeyInfo getPrivateKey(String keyName, Map keyMeta) { - EncryptionKeyInfo keyInfo = new EncryptionKeyInfo(); - try { - keyInfo.setKey(Files.readAllBytes(Paths.get(privateKeyFile))); - } catch (IOException e) { - System.out.println("ERROR: Failed to read private key from file " + privateKeyFile); - e.printStackTrace(); - } - return keyInfo; - } -} +client = Client('pulsar://localhost:6650') +topic = 'persistent://my-tenant/my-ns/my-topic' +# CryptoKeyReader is a built-in implementation that reads public key and private key from files +key_reader = CryptoKeyReader('test_ecdsa_pubkey.pem', 'test_ecdsa_privkey.pem') -PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); +producer = client.create_producer( + topic=topic, + encryption_key='myappkey', + crypto_key_reader=key_reader +) + +consumer = client.subscribe( + topic=topic, + subscription_name='my-subscriber-name', + crypto_key_reader=key_reader +) + +reader = client.create_reader( + topic=topic, + start_message_id=MessageId.earliest, + crypto_key_reader=key_reader +) -Producer producer = pulsarClient.newProducer() - .topic("persistent://my-tenant/my-ns/my-topic") - .addEncryptionKey("myappkey") - .cryptoKeyReader(new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem")) - .create(); +client.close() +``` + + + +```nodejs +const Pulsar = require('pulsar-client'); + +(async () => { +// Create a client +const client = new Pulsar.Client({ + serviceUrl: 'pulsar://localhost:6650', + operationTimeoutSeconds: 30, +}); + +// Create a producer +const producer = await client.createProducer({ + topic: 'persistent://public/default/my-topic', + sendTimeoutMs: 30000, + batchingEnabled: true, + publicKeyPath: "public-key.client-rsa.pem", + encryptionKey: "encryption-key" +}); + +// Create a consumer +const consumer = await client.subscribe({ + topic: 'persistent://public/default/my-topic', + subscription: 'sub1', + subscriptionType: 'Shared', + ackTimeoutMs: 10000, + privateKeyPath: "private-key.client-rsa.pem" +}); + +// Send messages +for (let i = 0; i < 10; i += 1) { + const msg = `my-message-${i}`; + producer.send({ + data: Buffer.from(msg), + }); + console.log(`Sent message: ${msg}`); +} +await producer.flush(); -for (int i = 0; i < 10; i++) { - producer.send("my-message".getBytes()); +// Receive messages +for (let i = 0; i < 10; i += 1) { + const msg = await consumer.receive(); + console.log(msg.getData().toString()); + consumer.acknowledge(msg); } -producer.close(); -pulsarClient.close(); +await consumer.close(); +await producer.close(); +await client.close(); +})(); ``` -7. Sample Consumer Application: + + +6. Below is an example of a **customized** `CryptoKeyReader` implementation. + + + ```java class RawFileKeyReader implements CryptoKeyReader { @@ -132,27 +220,39 @@ class RawFileKeyReader implements CryptoKeyReader { return keyInfo; } } +``` -PulsarClient pulsarClient = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); -Consumer consumer = pulsarClient.newConsumer() - .topic("persistent://my-tenant/my-ns/my-topic") - .subscriptionName("my-subscriber-name") - .cryptoKeyReader(new RawFileKeyReader("test_ecdsa_pubkey.pem", "test_ecdsa_privkey.pem")) - .subscribe(); -Message msg = null; - -for (int i = 0; i < 10; i++) { - msg = consumer.receive(); - // do something - System.out.println("Received: " + new String(msg.getData())); -} + +```c++ +class CustomCryptoKeyReader : public CryptoKeyReader { + public: + Result getPublicKey(const std::string& keyName, std::map& metadata, + EncryptionKeyInfo& encKeyInfo) const override { + // TODO: + return ResultOk; + } -// Acknowledge the consumption of all messages at once -consumer.acknowledgeCumulative(msg); -consumer.close(); -pulsarClient.close(); + Result getPrivateKey(const std::string& keyName, std::map& metadata, + EncryptionKeyInfo& encKeyInfo) const override { + // TODO: + return ResultOk; + } +}; + +auto keyReader = std::make_shared(/* ... */); +// TODO: create producer, consumer or reader based on keyReader here ``` +Besides, you can use the **default** implementation of `CryptoKeyReader` by specifying the paths of `private key` and `public key`. + + +Currently, **customized** `CryptoKeyReader` implementation is not supported in Python. However, you can use the **default** implementation by specifying the path of `private key` and `public key`. + + +Currently, **customized** `CryptoKeyReader` implementation is not supported in Node.JS. However, you can use the **default** implementation by specifying the path of `private key` and `public key`. + + + ## Key rotation Pulsar generates a new AES data key every 4 hours or after publishing a certain number of messages. A producer fetches the asymmetric public key every 4 hours by calling CryptoKeyReader.getPublicKey() to retrieve the latest version. diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-aliyun.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-aliyun.md index 93f8409046412..002c8f02a9e75 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-aliyun.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-aliyun.md @@ -23,20 +23,20 @@ This example uses Pulsar 2.8.0. 2. Download and untar the Pulsar offloaders package, then copy the Pulsar offloaders as `offloaders` in the Pulsar directory, see [here](https://pulsar.apache.org/docs/en/standalone/#install-tiered-storage-offloaders-optional). - **Output** - - As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. - - ``` - tiered-storage-file-system-2.8.0.nar - tiered-storage-jcloud-2.8.0.nar - ``` - - > **Note** - > - > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. - > - > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. + **Output** + + As shown from the output, Pulsar uses [Apache jclouds](https://jclouds.apache.org) to support [AWS S3](https://aws.amazon.com/s3/), [GCS](https://cloud.google.com/storage/), [Azure](https://portal.azure.com/#home), and [Aliyun OSS](https://www.aliyun.com/product/oss) for long-term storage. + + ``` + tiered-storage-file-system-2.8.0.nar + tiered-storage-jcloud-2.8.0.nar + ``` + + > **Note** + > + > * If you are running Pulsar in a bare-metal cluster, make sure that `offloaders` tarball is unzipped in every broker's Pulsar directory. + > + > * If you are running Pulsar in Docker or deploying Pulsar using a Docker image (such as K8s and DCOS), you can use the `apachepulsar/pulsar-all` image. The `apachepulsar/pulsar-all` image has already bundled tiered storage offloaders. ## Configuration @@ -211,7 +211,7 @@ For individual topics, you can trigger the Aliyun OSS offloader manually using o null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > **Tip** > diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-aws.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-aws.md index 14002be60d818..69da9c8d75ae6 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-aws.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-aws.md @@ -271,7 +271,7 @@ For individual topics, you can trigger AWS S3 offloader manually using one of th null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-azure.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-azure.md index b86cffb7a11cd..efaea2118c2af 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-azure.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-azure.md @@ -218,7 +218,7 @@ For individual topics, you can trigger Azure BlobStore offloader manually using null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-filesystem.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-filesystem.md index fed7f757133e7..ec5707a7b88f3 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-filesystem.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-filesystem.md @@ -258,7 +258,7 @@ To trigger via CLI tools, you need to specify the maximum amount of data (thresh null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-gcs.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-gcs.md index 2614fa7ee05fb..d9db76b3d956a 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-gcs.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-gcs.md @@ -263,7 +263,7 @@ For individual topics, you can trigger GCS offloader manually using one of the f null Reason: Error offloading: org.apache.bookkeeper.mledger.ManagedLedgerException: java.util.concurrent.CompletionException: com.amazonaws.services.s3.model.AmazonS3Exception: Anonymous users cannot initiate multipart uploads. Please authenticate. (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: 798758DE3F1776DF; S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g=), S3 Extended Request ID: dhBFz/lZm1oiG/oBEepeNlhrtsDlzoOhocuYMpKihQGXe6EG8puRGOkK6UwqzVrMXTWBxxHcS+g= - ```` + ``` > #### Tip > diff --git a/site2/website/versioned_docs/version-2.8.2/tiered-storage-overview.md b/site2/website/versioned_docs/version-2.8.2/tiered-storage-overview.md index 6986ac38ec1f2..6b624a4147b01 100644 --- a/site2/website/versioned_docs/version-2.8.2/tiered-storage-overview.md +++ b/site2/website/versioned_docs/version-2.8.2/tiered-storage-overview.md @@ -7,8 +7,7 @@ original_id: tiered-storage-overview Pulsar's **Tiered Storage** feature allows older backlog data to be moved from BookKeeper to long term and cheaper storage, while still allowing clients to access the backlog as if nothing has changed. -* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support -[Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. +* Tiered storage uses [Apache jclouds](https://jclouds.apache.org) to support [Amazon S3](https://aws.amazon.com/s3/) and [GCS (Google Cloud Storage)](https://cloud.google.com/storage/) for long term storage. With jclouds, it is easy to add support for more [cloud storage providers](https://jclouds.apache.org/reference/providers/#blobstore-providers) in the future. diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/FunctionsCLITest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/FunctionsCLITest.java index a89d36d579acc..75987f1473d31 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/FunctionsCLITest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/FunctionsCLITest.java @@ -54,7 +54,7 @@ public String uploadFunction() throws Exception { }; ContainerExecResult output = pulsarCluster.getAnyWorker().execCmd(commands); assertEquals(0, output.getExitCode()); - assertTrue(output.getStdout().contains("\"Uploaded successfully\"")); + assertTrue(output.getStdout().contains("Uploaded successfully")); return bkPkgPath; } @@ -77,7 +77,7 @@ public void testUploadDownload() throws Exception { WorkerContainer container = pulsarCluster.getAnyWorker(); ContainerExecResult output = container.execCmd(commands); assertEquals(0, output.getExitCode()); - assertTrue(output.getStdout().contains("\"Downloaded successfully\"")); + assertTrue(output.getStdout().contains("Downloaded successfully")); String[] diffCommand = { "diff", PulsarCluster.ADMIN_SCRIPT, diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java index 87357a04cf6fb..97d2c7bb31c7f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarFunctionsTest.java @@ -272,7 +272,7 @@ public void testWindowFunction(String type, String[] expectedResults) throws Exc }; ContainerExecResult containerExecResult = pulsarCluster.getAnyWorker().execCmd(commands); - assertTrue(containerExecResult.getStdout().contains("\"Created successfully\"")); + assertTrue(containerExecResult.getStdout().contains("Created successfully")); // get function info getFunctionInfoSuccess(functionName); @@ -848,7 +848,7 @@ private void submitFunction(Runtime runtime, }; ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd( commands); - assertTrue(result.getStdout().contains("\"Created successfully\"")); + assertTrue(result.getStdout().contains("Created successfully")); if (StringUtils.isNotEmpty(inputTopicName)) { ensureSubscriptionCreated( @@ -869,7 +869,7 @@ private void updateFunctionParallelism(String functionName, int parallelism) thr }; ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd( commands); - assertTrue(result.getStdout().contains("\"Updated successfully\"")); + assertTrue(result.getStdout().contains("Updated successfully")); } protected void submitFunction(Runtime runtime, @@ -912,7 +912,7 @@ protected void submitFunction(Runtime runtime, }; ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd( commands); - assertTrue(result.getStdout().contains("\"Created successfully\"")); + assertTrue(result.getStdout().contains("Created successfully")); } private void ensureSubscriptionCreated(String inputTopicName, @@ -1514,7 +1514,7 @@ private void submitJavaLoggingFunction(String inputTopicName, }; ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd( commands); - assertTrue(result.getStdout().contains("\"Created successfully\"")); + assertTrue(result.getStdout().contains("Created successfully")); ensureSubscriptionCreated(inputTopicName, String.format("public/default/%s", functionName), schema); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java index c5fb6ecffebff..21b1df1115b96 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/functions/PulsarStateTest.java @@ -199,7 +199,7 @@ private void submitSourceConnector(String sourceName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -218,7 +218,7 @@ private void submitSinkConnector(String sinkName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -267,7 +267,7 @@ private void submitFunction(Runtime runtime, }; ContainerExecResult result = container.execCmd( commands); - assertTrue(result.getStdout().contains("\"Created successfully\"")); + assertTrue(result.getStdout().contains("Created successfully")); ensureSubscriptionCreated(inputTopicName, String.format("public/default/%s", functionName), inputTopicSchema); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarGenericObjectSinkTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarGenericObjectSinkTest.java index fc266a54dde51..e5b16984ec76d 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarGenericObjectSinkTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarGenericObjectSinkTest.java @@ -270,7 +270,7 @@ private void submitSinkConnector(String sinkName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java index 92b61967b172d..4b055d1371bb9 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java @@ -173,7 +173,7 @@ protected void submitSinkConnector(@SuppressWarnings("rawtypes") SinkTester test log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -212,7 +212,7 @@ protected void updateSinkConnector(@SuppressWarnings("rawtypes") SinkTester test log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd(commands); assertTrue( - result.getStdout().contains("\"Updated successfully\""), + result.getStdout().contains("Updated successfully"), result.getStdout()); } @@ -450,4 +450,4 @@ protected Map produceSchemaDeleteMessagesToInputTopic(String inp } return kvs; } -} \ No newline at end of file +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/AvroKafkaSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/AvroKafkaSourceTest.java index 256c49e467af4..94b69695480f9 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/AvroKafkaSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/AvroKafkaSourceTest.java @@ -328,7 +328,7 @@ protected void submitSourceConnector(String tenant, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java index 8ed9e5445420c..a3d9b23e7d3ce 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java @@ -118,7 +118,7 @@ private void submitSourceConnector(String sourceName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -224,4 +224,4 @@ private static void getSourceInfoNotFound(StandaloneContainer container, String } } -} \ No newline at end of file +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/GenericRecordSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/GenericRecordSourceTest.java index 3c961fae6e931..d9484ac0f34b5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/GenericRecordSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/GenericRecordSourceTest.java @@ -116,7 +116,7 @@ private void submitSourceConnector(String sourceName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -255,4 +255,4 @@ private static void getSourceInfoNotFound(StandaloneContainer container, String } } -} \ No newline at end of file +} diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java index 68e80fd644727..ddad6ed18212b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java @@ -137,7 +137,7 @@ protected void submitSourceConnector(SourceTester tester, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } @@ -162,7 +162,7 @@ protected void updateSourceConnector(SourceTester tester, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = pulsarCluster.getAnyWorker().execCmd(commands); assertTrue( - result.getStdout().contains("\"Updated successfully\""), + result.getStdout().contains("Updated successfully"), result.getStdout()); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarSourcePropertyTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarSourcePropertyTest.java index 19da91ae86fe5..71b74454d053a 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarSourcePropertyTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarSourcePropertyTest.java @@ -104,7 +104,7 @@ private void submitSourceConnector(String sourceName, log.info("Run command : {}", StringUtils.join(commands, ' ')); ContainerExecResult result = container.execCmd(commands); assertTrue( - result.getStdout().contains("\"Created successfully\""), + result.getStdout().contains("Created successfully"), result.getStdout()); } diff --git a/tests/integration/src/test/resources/pulsar-io-sources.xml b/tests/integration/src/test/resources/pulsar-io-sources.xml index a5afc5d8d0322..636b3e479195f 100644 --- a/tests/integration/src/test/resources/pulsar-io-sources.xml +++ b/tests/integration/src/test/resources/pulsar-io-sources.xml @@ -23,6 +23,8 @@ + + \ No newline at end of file diff --git a/tests/integration/src/test/resources/pulsar-messaging.xml b/tests/integration/src/test/resources/pulsar-messaging.xml index aa31852fd6f62..631e961cbfb20 100644 --- a/tests/integration/src/test/resources/pulsar-messaging.xml +++ b/tests/integration/src/test/resources/pulsar-messaging.xml @@ -25,8 +25,9 @@ + + - \ No newline at end of file diff --git a/tests/integration/src/test/resources/pulsar-python.xml b/tests/integration/src/test/resources/pulsar-python.xml new file mode 100644 index 0000000000000..a5faa6389e0f1 --- /dev/null +++ b/tests/integration/src/test/resources/pulsar-python.xml @@ -0,0 +1,28 @@ + + + + + + + + + diff --git a/tests/integration/src/test/resources/pulsar-schema.xml b/tests/integration/src/test/resources/pulsar-schema.xml index c24b4fae0ccb1..e07fdf2b2d86f 100644 --- a/tests/integration/src/test/resources/pulsar-schema.xml +++ b/tests/integration/src/test/resources/pulsar-schema.xml @@ -24,6 +24,7 @@ + diff --git a/tests/integration/src/test/resources/pulsar-semantics.xml b/tests/integration/src/test/resources/pulsar-semantics.xml new file mode 100644 index 0000000000000..5b5402af4623b --- /dev/null +++ b/tests/integration/src/test/resources/pulsar-semantics.xml @@ -0,0 +1,28 @@ + + + + + + + + + diff --git a/tests/integration/src/test/resources/pulsar-upgrade.xml b/tests/integration/src/test/resources/pulsar-upgrade.xml new file mode 100644 index 0000000000000..a52db54753372 --- /dev/null +++ b/tests/integration/src/test/resources/pulsar-upgrade.xml @@ -0,0 +1,28 @@ + + + + + + + + + diff --git a/tests/integration/src/test/resources/pulsar.xml b/tests/integration/src/test/resources/pulsar.xml index 7993d0c6cd238..5382d9b5f5bef 100644 --- a/tests/integration/src/test/resources/pulsar.xml +++ b/tests/integration/src/test/resources/pulsar.xml @@ -36,5 +36,8 @@ + + + diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 6a204d56de951..e3fc68ab7e218 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -141,4 +141,9 @@ public void seekForward(long position) throws IOException { public void close() { buffer.release(); } + + @Override + public int available() throws IOException { + return (int)(objectLen - cursor) + buffer.readableBytes(); + } } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 2bf380d8c4126..f4dc1b8b4e421 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -105,6 +105,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr CompletableFuture promise = new CompletableFuture<>(); executor.submit(() -> { List entries = new ArrayList(); + boolean seeked = false; try { if (firstEntry > lastEntry || firstEntry < 0 @@ -115,14 +116,13 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr long entriesToRead = (lastEntry - firstEntry) + 1; long nextExpectedId = firstEntry; - // seek the position to the first entry position, otherwise we will get the unexpected entry ID when doing - // the first read, that would cause read an unexpected entry id which is out of range between firstEntry - // and lastEntry - // for example, when we get 1-10 entries at first, then the next request is get 2-9, the following code - // will read the entry id from the stream and that is not the correct entry id, so it will seek to the - // correct position then read the stream as normal. But the entry id may exceed the last entry id, that - // will cause we are hardly to know the edge of the request range. - inputStream.seek(index.getIndexEntryForEntry(firstEntry).getDataOffset()); + // checking the data stream has enough data to read to avoid throw EOF exception when reading data. + // 12 bytes represent the stream have the length and entryID to read. + if (dataStream.available() < 12) { + log.warn("There hasn't enough data to read, current available data has {} bytes," + + " seek to the first entry {} to avoid EOF exception", inputStream.available(), firstEntry); + inputStream.seek(index.getIndexEntryForEntry(firstEntry).getDataOffset()); + } while (entriesToRead > 0) { if (state == State.Closed) { @@ -149,14 +149,20 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr log.warn("The read entry {} is not the expected entry {} but in the range of {} - {}," + " seeking to the right position", entryId, nextExpectedId, nextExpectedId, lastEntry); inputStream.seek(index.getIndexEntryForEntry(nextExpectedId).getDataOffset()); - continue; } else if (entryId < nextExpectedId && !index.getIndexEntryForEntry(nextExpectedId).equals(index.getIndexEntryForEntry(entryId))) { log.warn("Read an unexpected entry id {} which is smaller than the next expected entry id {}" + ", seeking to the right position", entries, nextExpectedId); inputStream.seek(index.getIndexEntryForEntry(nextExpectedId).getDataOffset()); - continue; } else if (entryId > lastEntry) { + // in the normal case, the entry id should increment in order. But if there has random access in + // the read method, we should allow to seek to the right position and the entry id should + // never over to the last entry again. + if (!seeked) { + inputStream.seek(index.getIndexEntryForEntry(nextExpectedId).getDataOffset()); + seeked = true; + continue; + } log.info("Expected to read {}, but read {}, which is greater than last entry {}", nextExpectedId, entryId, lastEntry); throw new BKException.BKUnexpectedConditionException(); @@ -218,12 +224,32 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize) throws IOException { - Blob blob = blobStore.getBlob(bucket, indexKey); - versionCheck.check(indexKey, blob); - OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create(); - OffloadIndexBlock index; - try (InputStream payLoadStream = blob.getPayload().openStream()) { - index = (OffloadIndexBlock) indexBuilder.fromStream(payLoadStream); + int retryCount = 3; + OffloadIndexBlock index = null; + IOException lastException = null; + // The following retry is used to avoid to some network issue cause read index file failure. + // If it can not recovery in the retry, we will throw the exception and the dispatcher will schedule to + // next read. + // If we use a backoff to control the retry, it will introduce a concurrent operation. + // We don't want to make it complicated, because in the most of case it shouldn't in the retry loop. + while (retryCount-- > 0) { + Blob blob = blobStore.getBlob(bucket, indexKey); + versionCheck.check(indexKey, blob); + OffloadIndexBlockBuilder indexBuilder = OffloadIndexBlockBuilder.create(); + try (InputStream payLoadStream = blob.getPayload().openStream()) { + index = (OffloadIndexBlock) indexBuilder.fromStream(payLoadStream); + } catch (IOException e) { + // retry to avoid the network issue caused read failure + log.warn("Failed to get index block from the offoaded index file {}, still have {} times to retry", + indexKey, retryCount, e); + lastException = e; + continue; + } + lastException = null; + break; + } + if (lastException != null) { + throw lastException; } BackedInputStream inputStream = new BlobStoreBackedInputStreamImpl(blobStore, bucket, key, diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java index 2f64089c81cd9..a3fa14e763a06 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockImpl.java @@ -338,11 +338,7 @@ private OffloadIndexBlock fromStream(DataInputStream dis) throws IOException { int segmentMetadataLength = dis.readInt(); byte[] metadataBytes = new byte[segmentMetadataLength]; - - if (segmentMetadataLength != dis.read(metadataBytes)) { - log.error("Read ledgerMetadata from bytes failed"); - throw new IOException("Read ledgerMetadata from bytes failed"); - } + dis.readFully(metadataBytes); this.segmentMetadata = parseLedgerMetadata(metadataBytes); for (int i = 0; i < indexEntryCount; i++) { diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java index ffe8fb20e3f21..36541b42c7527 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java @@ -260,4 +260,27 @@ public void testSeekForward() throws Exception { toTest.seekForward(after); assertStreamsMatch(toTest, toCompare); } + + @Test + public void testAvailable() throws IOException { + String objectKey = "testAvailable"; + int objectSize = 2048; + RandomInputStream toWrite = new RandomInputStream(0, objectSize); + Payload payload = Payloads.newInputStreamPayload(toWrite); + payload.getContentMetadata().setContentLength((long)objectSize); + Blob blob = blobStore.blobBuilder(objectKey) + .payload(payload) + .contentLength(objectSize) + .build(); + String ret = blobStore.putBlob(BUCKET, blob); + BackedInputStream bis = new BlobStoreBackedInputStreamImpl( + blobStore, BUCKET, objectKey, (k, md) -> {}, objectSize, 512); + Assert.assertEquals(bis.available(), objectSize); + bis.seek(500); + Assert.assertEquals(bis.available(), objectSize - 500); + bis.seek(1024); + Assert.assertEquals(bis.available(), 1024); + bis.seek(2048); + Assert.assertEquals(bis.available(), 0); + } } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index 90d8b1198f489..77dfc55b7770e 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -24,6 +24,7 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.Collections; @@ -477,4 +478,22 @@ public void testReadUnknownIndexVersion() throws Exception { Assert.assertTrue(e.getCause().getMessage().contains("Invalid object version")); } } + + @Test + public void testReadEOFException() throws Throwable { + ReadHandle toWrite = buildReadHandle(DEFAULT_BLOCK_SIZE, 1); + LedgerOffloader offloader = getOffloader(); + UUID uuid = UUID.randomUUID(); + offloader.offload(toWrite, uuid, new HashMap<>()).get(); + + ReadHandle toTest = offloader.readOffloaded(toWrite.getId(), uuid, Collections.emptyMap()).get(); + Assert.assertEquals(toTest.getLastAddConfirmed(), toWrite.getLastAddConfirmed()); + toTest.readAsync(0, toTest.getLastAddConfirmed()).get(); + + try { + toTest.readAsync(0, 0).get(); + } catch (Exception e) { + fail("Get unexpected exception when reading entries", e); + } + } }