Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 9 additions & 0 deletions changelog/unreleased/solr-18064.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,9 @@
# See https://github.com/apache/solr/blob/main/dev-docs/changelog.adoc
title: CrossDC Producer - add more detailed metrics.
type: changed # added, changed, fixed, deprecated, removed, dependency_update, security, other
authors:
- name: Andrzej Bialecki
nick: ab
links:
- name: SOLR-18064
url: https://issues.apache.org/jira/browse/SOLR-18064
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@ public void processAdd(final AddUpdateCommand cmd) throws IOException {
producerMetrics.getDocumentSize().record(estimatedDocSizeInBytes);
final boolean tooLargeForKafka = estimatedDocSizeInBytes > maxMirroringDocSizeBytes;
if (tooLargeForKafka && !indexUnmirrorableDocs) {
producerMetrics.getDocumentTooLarge().inc();
throw new SolrException(
SolrException.ErrorCode.BAD_REQUEST,
"Update exceeds the doc-size limit and is unmirrorable. id="
Expand Down Expand Up @@ -181,9 +182,11 @@ public void processAdd(final AddUpdateCommand cmd) throws IOException {
try {
requestMirroringHandler.mirror(mirrorRequest);
producerMetrics.getSubmitted().inc();
producerMetrics.getSubmittedAdd().inc();
} catch (Exception e) {
log.error("mirror submit failed", e);
producerMetrics.getSubmitError().inc();
producerMetrics.getSubmittedAddError().inc();
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
}
}
Expand Down Expand Up @@ -250,7 +253,7 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {
return;
}
super.processDelete(cmd); // let this throw to prevent mirroring invalid requests

producerMetrics.getLocal().inc();
if (doMirroring) {
boolean isLeader = false;
UpdateRequest mirrorRequest = createMirrorRequest();
Expand All @@ -271,8 +274,11 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {

try {
requestMirroringHandler.mirror(mirrorRequest);
producerMetrics.getSubmitted().inc();
producerMetrics.getSubmittedDeleteById().inc();
} catch (Exception e) {
log.error("mirror submit failed", e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should there be a producerMetrics.getSubmitError().inc(); here? Looks like above that is the pattern. Also the best practice of the repo looks to be log or throw but not both. So I guess remove these log.errrors?

producerMetrics.getSubmittedDeleteByIdError().inc();
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
}
}
Expand All @@ -289,8 +295,12 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {

try {
requestMirroringHandler.mirror(mirrorRequest);
producerMetrics.getSubmitted().inc();
producerMetrics.getSubmittedDeleteByQuery().inc();
} catch (Exception e) {
log.error("mirror submit failed", e);
producerMetrics.getSubmitError().inc();
producerMetrics.getSubmittedDeleteByQueryError().inc();
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
}
}
Expand Down Expand Up @@ -390,7 +400,10 @@ private boolean shouldMirrorCommit(SolrQueryRequest req) {
@Override
public void processCommit(CommitUpdateCommand cmd) throws IOException {
log.debug("process commit cmd={}", cmd);
if (next != null) next.processCommit(cmd);
if (next != null) {
next.processCommit(cmd);
producerMetrics.getLocal().inc();
}
if (!mirrorCommits) {
return;
}
Expand Down Expand Up @@ -424,8 +437,12 @@ public void processCommit(CommitUpdateCommand cmd) throws IOException {
log.debug(" --doMirroring commit req={}", req);
try {
requestMirroringHandler.mirror(req);
producerMetrics.getSubmitted().inc();
producerMetrics.getSubmittedCommit().inc();
} catch (Exception e) {
log.error("mirror submit failed", e);
producerMetrics.getSubmitError().inc();
producerMetrics.getSubmittedCommitError().inc();
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,14 @@ public class ProducerMetrics {
private final AttributedLongCounter localError;
private final AttributedLongCounter submitted;
private final AttributedLongCounter submitError;
private final AttributedLongCounter submittedAdd;
private final AttributedLongCounter submittedAddError;
private final AttributedLongCounter submittedDeleteById;
private final AttributedLongCounter submittedDeleteByIdError;
private final AttributedLongCounter submittedDeleteByQuery;
private final AttributedLongCounter submittedDeleteByQueryError;
private final AttributedLongCounter submittedCommit;
private final AttributedLongCounter submittedCommitError;
private final AttributedLongHistogram documentSize;
private final AttributedLongCounter documentTooLarge;

Expand All @@ -45,6 +53,22 @@ public ProducerMetrics(SolrMetricsContext solrMetricsContext, SolrCore solrCore)
solrMetricsContext.longCounter(
"solr_core_crossdc_producer_submitted",
"The number of documents submitted to the Kafka topic (success or error)");
var localSubmittedAdd =
solrMetricsContext.longCounter(
"solr_core_crossdc_producer_submitted_add",
"The number of add requests submitted to the Kafka topic (success or error)");
var localSubmittedDbi =
solrMetricsContext.longCounter(
"solr_core_crossdc_producer_submitted_delete_by_id",
"The number of Delete-By-Id requests submitted to the Kafka topic (success or error)");
var localSubmittedDbq =
solrMetricsContext.longCounter(
"solr_core_crossdc_producer_submitted_delete_by_query",
"The number of Delete-By-Query requests submitted to the Kafka topic (success or error)");
var localSubmittedCommit =
solrMetricsContext.longCounter(
"solr_core_crossdc_producer_submitted_commit",
"The number of standalone Commit requests submitted to the Kafka topic (success or error)");
var histogramDocSizes =
solrMetricsContext.longHistogram(
"solr_core_crossdc_producer_document_size",
Expand All @@ -67,6 +91,30 @@ public ProducerMetrics(SolrMetricsContext solrMetricsContext, SolrCore solrCore)
this.submitError =
new AttributedLongCounter(
localSubmitted, attributes.toBuilder().put(TYPE_ATTR, "error").build());
this.submittedAdd =
new AttributedLongCounter(
localSubmittedAdd, attributes.toBuilder().put(TYPE_ATTR, "success").build());
this.submittedAddError =
new AttributedLongCounter(
localSubmittedAdd, attributes.toBuilder().put(TYPE_ATTR, "error").build());
this.submittedDeleteById =
new AttributedLongCounter(
localSubmittedDbi, attributes.toBuilder().put(TYPE_ATTR, "success").build());
this.submittedDeleteByIdError =
new AttributedLongCounter(
localSubmittedDbi, attributes.toBuilder().put(TYPE_ATTR, "error").build());
this.submittedDeleteByQuery =
new AttributedLongCounter(
localSubmittedDbq, attributes.toBuilder().put(TYPE_ATTR, "success").build());
this.submittedDeleteByQueryError =
new AttributedLongCounter(
localSubmittedDbq, attributes.toBuilder().put(TYPE_ATTR, "error").build());
this.submittedCommit =
new AttributedLongCounter(
localSubmittedCommit, attributes.toBuilder().put(TYPE_ATTR, "success").build());
this.submittedCommitError =
new AttributedLongCounter(
localSubmittedCommit, attributes.toBuilder().put(TYPE_ATTR, "error").build());
this.documentSize = new AttributedLongHistogram(histogramDocSizes, attributes);
this.documentTooLarge = new AttributedLongCounter(tooLargeErrors, attributes);
}
Expand Down Expand Up @@ -94,6 +142,58 @@ public AttributedLongCounter getSubmitError() {
return this.submitError;
}

/** Counter representing the number of add requests submitted to the Kafka topic. */
public AttributedLongCounter getSubmittedAdd() {
return this.submittedAdd;
}

/**
* Counter representing the number of add requests that were not submitted to the Kafka topic
* because of exception during execution.
*/
public AttributedLongCounter getSubmittedAddError() {
return this.submittedAddError;
}

/** Counter representing the number of delete-by-id requests submitted to the Kafka topic. */
public AttributedLongCounter getSubmittedDeleteById() {
return this.submittedDeleteById;
}

/**
* Counter representing the number of delete-by-id requests that were not submitted to the Kafka
* topic because of exception during execution.
*/
public AttributedLongCounter getSubmittedDeleteByIdError() {
return this.submittedDeleteByIdError;
}

/** Counter representing the number of delete-by-query requests submitted to the Kafka topic. */
public AttributedLongCounter getSubmittedDeleteByQuery() {
return this.submittedDeleteByQuery;
}

/**
* Counter representing the number of delete-by-query requests that were not submitted to the
* Kafka topic because of exception during execution.
*/
public AttributedLongCounter getSubmittedDeleteByQueryError() {
return this.submittedDeleteByQueryError;
}

/** Counter representing the number of standalone Commit requests submitted to the Kafka topic. */
public AttributedLongCounter getSubmittedCommit() {
return this.submittedCommit;
}

/**
* Counter representing the number of standalone Commit requests that were not submitted to the
* Kafka topic because of exception during execution.
*/
public AttributedLongCounter getSubmittedCommitError() {
return this.submittedCommitError;
}

/** Histogram of the processed document size. */
public AttributedLongHistogram getDocumentSize() {
return this.documentSize;
Expand Down
Loading