mirror of
https://github.com/google/nomulus.git
synced 2025-05-13 07:57:13 +02:00
Add stackdriver metrics to publishDnsUpdates
Adding the following metrics: - how long does an update take, per TLD - number of domains published, per TLD - number of hosts published, per TLD All are distributions. ------------- Created by MOE: https://github.com/google/moe MOE_MIGRATED_REVID=172933834
This commit is contained in:
parent
c702b4486c
commit
d577a281b8
3 changed files with 235 additions and 24 deletions
|
@ -14,31 +14,61 @@
|
|||
|
||||
package google.registry.dns;
|
||||
|
||||
import static google.registry.request.RequestParameters.PARAM_TLD;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.config.RegistryEnvironment;
|
||||
import google.registry.monitoring.metrics.DistributionFitter;
|
||||
import google.registry.monitoring.metrics.EventMetric;
|
||||
import google.registry.monitoring.metrics.ExponentialFitter;
|
||||
import google.registry.monitoring.metrics.FibonacciFitter;
|
||||
import google.registry.monitoring.metrics.IncrementableMetric;
|
||||
import google.registry.monitoring.metrics.LabelDescriptor;
|
||||
import google.registry.monitoring.metrics.MetricRegistryImpl;
|
||||
import google.registry.request.Parameter;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/** DNS instrumentation. */
|
||||
// TODO(b/67947699):Once load testing is done, revisit these to rename them and delete the ones that
|
||||
// we don't expect to need long-term.
|
||||
public class DnsMetrics {
|
||||
|
||||
/** Disposition of a publish request. */
|
||||
public enum Status { ACCEPTED, REJECTED }
|
||||
public enum PublishStatus { ACCEPTED, REJECTED }
|
||||
|
||||
private static final ImmutableSet<LabelDescriptor> LABEL_DESCRIPTORS =
|
||||
/** Disposition of writer.commit(). */
|
||||
public enum CommitStatus { SUCCESS, FAILURE }
|
||||
|
||||
private static final ImmutableSet<LabelDescriptor> LABEL_DESCRIPTORS_FOR_PUBLISH_REQUESTS =
|
||||
ImmutableSet.of(
|
||||
LabelDescriptor.create("tld", "TLD"),
|
||||
LabelDescriptor.create(
|
||||
"status", "Whether the publish request was accepted or rejected."));
|
||||
|
||||
private static final ImmutableSet<LabelDescriptor> LABEL_DESCRIPTORS_FOR_COMMIT =
|
||||
ImmutableSet.of(
|
||||
LabelDescriptor.create("tld", "TLD"),
|
||||
LabelDescriptor.create(
|
||||
"status", "Whether writer.commit() succeeded or failed."));
|
||||
|
||||
// Finer-grained fitter than the DEFAULT_FITTER, allows values between 1. and 2^20, which gives
|
||||
// over 15 minutes.
|
||||
private static final DistributionFitter EXPONENTIAL_FITTER =
|
||||
ExponentialFitter.create(20, 2.0, 1.0);
|
||||
|
||||
// Fibonacci fitter more suitible for integer-type values. Allows values between 0 and 10946,
|
||||
// which is the 21th Fibonacci number.
|
||||
private static final DistributionFitter FIBONACCI_FITTER =
|
||||
FibonacciFitter.create(10946);
|
||||
|
||||
private static final IncrementableMetric publishDomainRequests =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newIncrementableMetric(
|
||||
"/dns/publish_domain_requests",
|
||||
"count of publishDomain requests",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS);
|
||||
LABEL_DESCRIPTORS_FOR_PUBLISH_REQUESTS);
|
||||
|
||||
private static final IncrementableMetric publishHostRequests =
|
||||
MetricRegistryImpl.getDefault()
|
||||
|
@ -46,7 +76,88 @@ public class DnsMetrics {
|
|||
"/dns/publish_host_requests",
|
||||
"count of publishHost requests",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS);
|
||||
LABEL_DESCRIPTORS_FOR_PUBLISH_REQUESTS);
|
||||
|
||||
private static final IncrementableMetric commitCount =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newIncrementableMetric(
|
||||
"/dns/commit_requests",
|
||||
"Count of writer.commit() calls",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT);
|
||||
|
||||
private static final IncrementableMetric domainsCommittedCount =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newIncrementableMetric(
|
||||
"/dns/domains_committed",
|
||||
"Count of domains committed",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT);
|
||||
|
||||
private static final IncrementableMetric hostsCommittedCount =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newIncrementableMetric(
|
||||
"/dns/hosts_committed",
|
||||
"Count of hosts committed",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT);
|
||||
|
||||
private static final EventMetric processingTimePerCommitDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_batch/processing_time",
|
||||
"publishDnsUpdates Processing Time",
|
||||
"milliseconds",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
EXPONENTIAL_FITTER);
|
||||
|
||||
private static final EventMetric normalizedProcessingTimePerCommitDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_batch/processing_time_per_dns_update",
|
||||
"publishDnsUpdates Processing Time, divided by the batch size",
|
||||
"milliseconds",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
EXPONENTIAL_FITTER);
|
||||
|
||||
private static final EventMetric totalBatchSizePerCommitDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_batch/batch_size",
|
||||
"Number of hosts and domains committed in each publishDnsUpdates",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
FIBONACCI_FITTER);
|
||||
|
||||
private static final EventMetric processingTimePerItemDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_item/processing_time",
|
||||
"publishDnsUpdates Processing Time",
|
||||
"milliseconds",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
EXPONENTIAL_FITTER);
|
||||
|
||||
private static final EventMetric normalizedProcessingTimePerItemDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_item/processing_time_per_dns_update",
|
||||
"publishDnsUpdates Processing Time, divided by the batch size",
|
||||
"milliseconds",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
EXPONENTIAL_FITTER);
|
||||
|
||||
private static final EventMetric totalBatchSizePerItemDist =
|
||||
MetricRegistryImpl.getDefault()
|
||||
.newEventMetric(
|
||||
"/dns/per_item/batch_size",
|
||||
"Batch sizes for hosts and domains",
|
||||
"count",
|
||||
LABEL_DESCRIPTORS_FOR_COMMIT,
|
||||
FIBONACCI_FITTER);
|
||||
|
||||
@Inject RegistryEnvironment registryEnvironment;
|
||||
@Inject @Parameter(PARAM_TLD) String tld;
|
||||
|
||||
@Inject
|
||||
DnsMetrics() {}
|
||||
|
@ -55,15 +166,61 @@ public class DnsMetrics {
|
|||
* Increment a monotonic counter that tracks calls to {@link
|
||||
* google.registry.dns.writer.DnsWriter#publishDomain(String)}, per TLD.
|
||||
*/
|
||||
public void incrementPublishDomainRequests(String tld, Status status) {
|
||||
publishDomainRequests.increment(tld, status.name());
|
||||
public void incrementPublishDomainRequests(long numRequests, PublishStatus status) {
|
||||
if (numRequests > 0) {
|
||||
publishDomainRequests.incrementBy(numRequests, tld, status.name());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Increment a monotonic counter that tracks calls to {@link
|
||||
* google.registry.dns.writer.DnsWriter#publishHost(String)}, per TLD.
|
||||
*/
|
||||
public void incrementPublishHostRequests(String tld, Status status) {
|
||||
publishHostRequests.increment(tld, status.name());
|
||||
public void incrementPublishHostRequests(long numRequests, PublishStatus status) {
|
||||
if (numRequests > 0) {
|
||||
publishHostRequests.incrementBy(numRequests, tld, status.name());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Measures information about the entire batched commit, per TLD.
|
||||
*
|
||||
* The information includes running times (per item and per commit), and batch sizes (per item and
|
||||
* per commit)
|
||||
*
|
||||
* This is to be used for load testing the system, and will not measure anything in prod.
|
||||
*/
|
||||
void recordCommit(
|
||||
CommitStatus status,
|
||||
Duration processingDuration,
|
||||
int numberOfDomains,
|
||||
int numberOfHosts) {
|
||||
// We don't want to record all these metrics in production, as they are quite expensive
|
||||
if (registryEnvironment == RegistryEnvironment.PRODUCTION) {
|
||||
return;
|
||||
}
|
||||
int batchSize = numberOfDomains + numberOfHosts;
|
||||
|
||||
processingTimePerCommitDist.record(processingDuration.getMillis(), tld, status.name());
|
||||
processingTimePerItemDist.record(
|
||||
processingDuration.getMillis(), batchSize, tld, status.name());
|
||||
|
||||
if (batchSize > 0) {
|
||||
normalizedProcessingTimePerCommitDist.record(
|
||||
(double) processingDuration.getMillis() / batchSize,
|
||||
tld, status.name());
|
||||
normalizedProcessingTimePerItemDist.record(
|
||||
(double) processingDuration.getMillis() / batchSize,
|
||||
batchSize,
|
||||
tld, status.name());
|
||||
}
|
||||
|
||||
totalBatchSizePerCommitDist.record(batchSize, tld, status.name());
|
||||
|
||||
totalBatchSizePerItemDist.record(batchSize, batchSize, tld, status.name());
|
||||
|
||||
commitCount.increment(tld, status.name());
|
||||
domainsCommittedCount.incrementBy(numberOfDomains, tld, status.name());
|
||||
hostsCommittedCount.incrementBy(numberOfHosts, tld, status.name());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,8 @@ import static google.registry.util.CollectionUtils.nullToEmpty;
|
|||
|
||||
import com.google.common.net.InternetDomainName;
|
||||
import google.registry.config.RegistryConfig.Config;
|
||||
import google.registry.dns.DnsMetrics.Status;
|
||||
import google.registry.dns.DnsMetrics.CommitStatus;
|
||||
import google.registry.dns.DnsMetrics.PublishStatus;
|
||||
import google.registry.dns.writer.DnsWriter;
|
||||
import google.registry.model.registry.Registry;
|
||||
import google.registry.request.Action;
|
||||
|
@ -28,11 +29,13 @@ import google.registry.request.HttpException.ServiceUnavailableException;
|
|||
import google.registry.request.Parameter;
|
||||
import google.registry.request.auth.Auth;
|
||||
import google.registry.request.lock.LockHandler;
|
||||
import google.registry.util.Clock;
|
||||
import google.registry.util.DomainNameUtils;
|
||||
import google.registry.util.FormattingLogger;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import javax.inject.Inject;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
/** Task that sends domain and host updates to the DNS server. */
|
||||
|
@ -48,6 +51,7 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
|||
public static final String PARAM_DNS_WRITER = "dnsWriter";
|
||||
public static final String PARAM_DOMAINS = "domains";
|
||||
public static final String PARAM_HOSTS = "hosts";
|
||||
public static final String LOCK_NAME = "DNS updates";
|
||||
|
||||
private static final FormattingLogger logger = FormattingLogger.getLoggerForCallerClass();
|
||||
|
||||
|
@ -70,17 +74,17 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
|||
@Inject @Parameter(PARAM_HOSTS) Set<String> hosts;
|
||||
@Inject @Parameter(PARAM_TLD) String tld;
|
||||
@Inject LockHandler lockHandler;
|
||||
@Inject Clock clock;
|
||||
@Inject PublishDnsUpdatesAction() {}
|
||||
|
||||
/** Runs the task. */
|
||||
@Override
|
||||
public void run() {
|
||||
String lockName = String.format("DNS zone %s", tld);
|
||||
// If executeWithLocks fails to get the lock, it does not throw an exception, simply returns
|
||||
// false. We need to make sure to take note of this error; otherwise, a failed lock might result
|
||||
// in the update task being dequeued and dropped. A message will already have been logged
|
||||
// to indicate the problem.
|
||||
if (!lockHandler.executeWithLocks(this, tld, timeout, lockName)) {
|
||||
if (!lockHandler.executeWithLocks(this, tld, timeout, LOCK_NAME)) {
|
||||
throw new ServiceUnavailableException("Lock failure");
|
||||
}
|
||||
}
|
||||
|
@ -94,30 +98,65 @@ public final class PublishDnsUpdatesAction implements Runnable, Callable<Void> {
|
|||
|
||||
/** Steps through the domain and host refreshes contained in the parameters and processes them. */
|
||||
private void processBatch() {
|
||||
DateTime timeAtStart = clock.nowUtc();
|
||||
|
||||
DnsWriter writer = dnsWriterProxy.getByClassNameForTld(dnsWriter, tld);
|
||||
|
||||
int domainsPublished = 0;
|
||||
int domainsRejected = 0;
|
||||
for (String domain : nullToEmpty(domains)) {
|
||||
if (!DomainNameUtils.isUnder(
|
||||
InternetDomainName.from(domain), InternetDomainName.from(tld))) {
|
||||
dnsMetrics.incrementPublishDomainRequests(tld, Status.REJECTED);
|
||||
logger.severefmt("%s: skipping domain %s not under tld", tld, domain);
|
||||
domainsRejected += 1;
|
||||
} else {
|
||||
dnsMetrics.incrementPublishDomainRequests(tld, Status.ACCEPTED);
|
||||
writer.publishDomain(domain);
|
||||
logger.infofmt("%s: published domain %s", tld, domain);
|
||||
domainsPublished += 1;
|
||||
}
|
||||
}
|
||||
dnsMetrics.incrementPublishDomainRequests(domainsPublished, PublishStatus.ACCEPTED);
|
||||
dnsMetrics.incrementPublishDomainRequests(domainsRejected, PublishStatus.REJECTED);
|
||||
|
||||
int hostsPublished = 0;
|
||||
int hostsRejected = 0;
|
||||
for (String host : nullToEmpty(hosts)) {
|
||||
if (!DomainNameUtils.isUnder(
|
||||
InternetDomainName.from(host), InternetDomainName.from(tld))) {
|
||||
dnsMetrics.incrementPublishHostRequests(tld, Status.REJECTED);
|
||||
logger.severefmt("%s: skipping host %s not under tld", tld, host);
|
||||
hostsRejected += 1;
|
||||
} else {
|
||||
dnsMetrics.incrementPublishHostRequests(tld, Status.ACCEPTED);
|
||||
writer.publishHost(host);
|
||||
logger.infofmt("%s: published host %s", tld, host);
|
||||
hostsPublished += 1;
|
||||
}
|
||||
}
|
||||
dnsMetrics.incrementPublishHostRequests(hostsPublished, PublishStatus.ACCEPTED);
|
||||
dnsMetrics.incrementPublishHostRequests(hostsRejected, PublishStatus.REJECTED);
|
||||
|
||||
// If we got here it means we managed to stage the entire batch without any errors.
|
||||
writer.commit();
|
||||
// Next we will commit the batch.
|
||||
CommitStatus commitStatus = CommitStatus.FAILURE;
|
||||
try {
|
||||
writer.commit();
|
||||
// No error was thrown
|
||||
commitStatus = CommitStatus.SUCCESS;
|
||||
} finally {
|
||||
Duration duration = new Duration(timeAtStart, clock.nowUtc());
|
||||
dnsMetrics.recordCommit(
|
||||
commitStatus,
|
||||
duration,
|
||||
domainsPublished,
|
||||
hostsPublished);
|
||||
logger.info(
|
||||
"writer.commit() statistics"
|
||||
+ "\nTLD: " + tld
|
||||
+ "\ncommitStatus: " + commitStatus
|
||||
+ "\nduration: " + duration
|
||||
+ "\ndomainsPublished: " + domainsPublished
|
||||
+ "\ndomainsRejected: " + domainsRejected
|
||||
+ "\nhostsPublished: " + hostsPublished
|
||||
+ "\nhostsRejected: " + hostsRejected);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,13 +19,13 @@ import static google.registry.testing.DatastoreHelper.persistActiveDomain;
|
|||
import static google.registry.testing.DatastoreHelper.persistActiveSubordinateHost;
|
||||
import static google.registry.testing.DatastoreHelper.persistResource;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyNoMoreInteractions;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import google.registry.dns.DnsMetrics.Status;
|
||||
import google.registry.dns.DnsMetrics.CommitStatus;
|
||||
import google.registry.dns.DnsMetrics.PublishStatus;
|
||||
import google.registry.dns.writer.DnsWriter;
|
||||
import google.registry.model.domain.DomainResource;
|
||||
import google.registry.model.ofy.Ofy;
|
||||
|
@ -90,6 +90,7 @@ public class PublishDnsUpdatesActionTest {
|
|||
action.dnsWriterProxy = new DnsWriterProxy(ImmutableMap.of("mock", dnsWriter));
|
||||
action.dnsMetrics = dnsMetrics;
|
||||
action.lockHandler = lockHandler;
|
||||
action.clock = clock;
|
||||
return action;
|
||||
}
|
||||
|
||||
|
@ -103,7 +104,11 @@ public class PublishDnsUpdatesActionTest {
|
|||
verify(dnsWriter).commit();
|
||||
verifyNoMoreInteractions(dnsWriter);
|
||||
|
||||
verify(dnsMetrics).incrementPublishHostRequests("xn--q9jyb4c", Status.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(0, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(1, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).recordCommit(CommitStatus.SUCCESS, Duration.ZERO, 0, 1);
|
||||
verifyNoMoreInteractions(dnsMetrics);
|
||||
}
|
||||
|
||||
|
@ -117,7 +122,11 @@ public class PublishDnsUpdatesActionTest {
|
|||
verify(dnsWriter).commit();
|
||||
verifyNoMoreInteractions(dnsWriter);
|
||||
|
||||
verify(dnsMetrics).incrementPublishDomainRequests("xn--q9jyb4c", Status.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(1, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(0, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).recordCommit(CommitStatus.SUCCESS, Duration.ZERO, 1, 0);
|
||||
verifyNoMoreInteractions(dnsMetrics);
|
||||
}
|
||||
|
||||
|
@ -137,8 +146,11 @@ public class PublishDnsUpdatesActionTest {
|
|||
verify(dnsWriter).commit();
|
||||
verifyNoMoreInteractions(dnsWriter);
|
||||
|
||||
verify(dnsMetrics, times(2)).incrementPublishDomainRequests("xn--q9jyb4c", Status.ACCEPTED);
|
||||
verify(dnsMetrics, times(3)).incrementPublishHostRequests("xn--q9jyb4c", Status.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(2, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(3, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(0, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).recordCommit(CommitStatus.SUCCESS, Duration.ZERO, 2, 3);
|
||||
verifyNoMoreInteractions(dnsMetrics);
|
||||
}
|
||||
|
||||
|
@ -152,8 +164,11 @@ public class PublishDnsUpdatesActionTest {
|
|||
verify(dnsWriter).commit();
|
||||
verifyNoMoreInteractions(dnsWriter);
|
||||
|
||||
verify(dnsMetrics, times(2)).incrementPublishDomainRequests("xn--q9jyb4c", Status.REJECTED);
|
||||
verify(dnsMetrics, times(3)).incrementPublishHostRequests("xn--q9jyb4c", Status.REJECTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(0, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishDomainRequests(2, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(0, PublishStatus.ACCEPTED);
|
||||
verify(dnsMetrics).incrementPublishHostRequests(3, PublishStatus.REJECTED);
|
||||
verify(dnsMetrics).recordCommit(CommitStatus.SUCCESS, Duration.ZERO, 0, 0);
|
||||
verifyNoMoreInteractions(dnsMetrics);
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue