This is an automated email from the ASF dual-hosted git repository. sijie pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/incubator-pulsar.git
The following commit(s) were added to refs/heads/master by this push: new 069efdb Expose offload deletion lag in broker configuration (#1805) 069efdb is described below commit 069efdbc7520fbcff6e9392c05a8ebbbe8492b0d Author: Ivan Kelly <iv...@apache.org> AuthorDate: Mon May 21 09:57:14 2018 +0200 Expose offload deletion lag in broker configuration (#1805) With this configuration, the administrator can configure how long the bookkeeper ledger will be kept around after a ledger has been successfully offloaded. Master Issue: #1511 --- conf/broker.conf | 4 ++++ .../org/apache/bookkeeper/mledger/ManagedLedgerConfig.java | 2 +- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 12 ++++++++++++ .../java/org/apache/pulsar/broker/service/BrokerService.java | 2 ++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index 39523cd..2a3d457 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -323,6 +323,10 @@ managedLedgerMinLedgerRolloverTimeMinutes=10 # Maximum time before forcing a ledger rollover for a topic managedLedgerMaxLedgerRolloverTimeMinutes=240 +# Delay between a ledger being successfully offloaded to long term storage +# and the ledger being deleted from bookkeeper (default is 4 hours) +managedLedgerOffloadDeletionLagMs=14400000 + # Max number of entries to append to a cursor ledger managedLedgerCursorMaxEntriesPerLedger=50000 diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 566701c..fc5499b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -395,7 +395,7 @@ public class ManagedLedgerConfig { * @param lagTime period to wait before deleting offloaded ledgers from bookkeeper * @param unit timeunit for lagTime */ - public ManagedLedgerConfig setOffloadLedgerDeletionLag(int lagTime, TimeUnit unit) { + public ManagedLedgerConfig setOffloadLedgerDeletionLag(long lagTime, TimeUnit unit) { this.offloadLedgerDeletionLagMs = unit.toMillis(lagTime); return this; } 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 d28ff4c..0cdaec8 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 @@ -23,6 +23,7 @@ import java.util.List; import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.client.api.DigestType; import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; @@ -334,6 +335,9 @@ public class ServiceConfiguration implements PulsarConfiguration { private int managedLedgerMinLedgerRolloverTimeMinutes = 10; // Maximum time before forcing a ledger rollover for a topic private int managedLedgerMaxLedgerRolloverTimeMinutes = 240; + // Delay between a ledger being successfully offloaded to long term storage + // and the ledger being deleted from bookkeeper + private long managedLedgerOffloadDeletionLagMs = TimeUnit.HOURS.toMillis(4); // Max number of entries to append to a cursor ledger private int managedLedgerCursorMaxEntriesPerLedger = 50000; // Max time before triggering a rollover on a cursor ledger @@ -1224,6 +1228,14 @@ public class ServiceConfiguration implements PulsarConfiguration { this.managedLedgerMaxLedgerRolloverTimeMinutes = managedLedgerMaxLedgerRolloverTimeMinutes; } + public long getManagedLedgerOffloadDeletionLagMs() { + return managedLedgerOffloadDeletionLagMs; + } + + public void setManagedLedgerOffloadDeletionLag(long amount, TimeUnit unit) { + this.managedLedgerOffloadDeletionLagMs = unit.toMillis(amount); + } + public int getManagedLedgerCursorMaxEntriesPerLedger() { return managedLedgerCursorMaxEntriesPerLedger; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 6a823f4..3b31344 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -729,6 +729,8 @@ public class BrokerService implements Closeable, ZooKeeperCacheListener<Policies managedLedgerConfig.setRetentionSizeInMB(retentionPolicies.getRetentionSizeInMB()); managedLedgerConfig.setLedgerOffloader(pulsar.getManagedLedgerOffloader()); + managedLedgerConfig.setOffloadLedgerDeletionLag(serviceConfig.getManagedLedgerOffloadDeletionLagMs(), + TimeUnit.MILLISECONDS); future.complete(managedLedgerConfig); }, (exception) -> future.completeExceptionally(exception))); -- To stop receiving notification emails like this one, please contact si...@apache.org.