Skip to content

Commit 9e073e9

Browse files
authored
SOLR-18064: CrossDC Producer - add more detailed metrics. (cherry-pick from apache#4106) (apache#4150)
1 parent 5ebffb9 commit 9e073e9

File tree

4 files changed

+254
-10
lines changed

4 files changed

+254
-10
lines changed
Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,9 @@
1+
# See https://github.com/apache/solr/blob/main/dev-docs/changelog.adoc
2+
title: CrossDC Producer - add more detailed metrics.
3+
type: changed # added, changed, fixed, deprecated, removed, dependency_update, security, other
4+
authors:
5+
- name: Andrzej Bialecki
6+
nick: ab
7+
links:
8+
- name: SOLR-18064
9+
url: https://issues.apache.org/jira/browse/SOLR-18064

solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/MirroringUpdateProcessor.java

Lines changed: 19 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -146,6 +146,7 @@ public void processAdd(final AddUpdateCommand cmd) throws IOException {
146146
producerMetrics.getDocumentSize().record(estimatedDocSizeInBytes);
147147
final boolean tooLargeForKafka = estimatedDocSizeInBytes > maxMirroringDocSizeBytes;
148148
if (tooLargeForKafka && !indexUnmirrorableDocs) {
149+
producerMetrics.getDocumentTooLarge().inc();
149150
throw new SolrException(
150151
SolrException.ErrorCode.BAD_REQUEST,
151152
"Update exceeds the doc-size limit and is unmirrorable. id="
@@ -181,9 +182,11 @@ public void processAdd(final AddUpdateCommand cmd) throws IOException {
181182
try {
182183
requestMirroringHandler.mirror(mirrorRequest);
183184
producerMetrics.getSubmitted().inc();
185+
producerMetrics.getSubmittedAdd().inc();
184186
} catch (Exception e) {
185187
log.error("mirror submit failed", e);
186188
producerMetrics.getSubmitError().inc();
189+
producerMetrics.getSubmittedAddError().inc();
187190
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
188191
}
189192
}
@@ -250,7 +253,7 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {
250253
return;
251254
}
252255
super.processDelete(cmd); // let this throw to prevent mirroring invalid requests
253-
256+
producerMetrics.getLocal().inc();
254257
if (doMirroring) {
255258
boolean isLeader = false;
256259
UpdateRequest mirrorRequest = createMirrorRequest();
@@ -271,8 +274,11 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {
271274

272275
try {
273276
requestMirroringHandler.mirror(mirrorRequest);
277+
producerMetrics.getSubmitted().inc();
278+
producerMetrics.getSubmittedDeleteById().inc();
274279
} catch (Exception e) {
275280
log.error("mirror submit failed", e);
281+
producerMetrics.getSubmittedDeleteByIdError().inc();
276282
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
277283
}
278284
}
@@ -289,8 +295,12 @@ public void processDelete(final DeleteUpdateCommand cmd) throws IOException {
289295

290296
try {
291297
requestMirroringHandler.mirror(mirrorRequest);
298+
producerMetrics.getSubmitted().inc();
299+
producerMetrics.getSubmittedDeleteByQuery().inc();
292300
} catch (Exception e) {
293301
log.error("mirror submit failed", e);
302+
producerMetrics.getSubmitError().inc();
303+
producerMetrics.getSubmittedDeleteByQueryError().inc();
294304
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
295305
}
296306
}
@@ -390,7 +400,10 @@ private boolean shouldMirrorCommit(SolrQueryRequest req) {
390400
@Override
391401
public void processCommit(CommitUpdateCommand cmd) throws IOException {
392402
log.debug("process commit cmd={}", cmd);
393-
if (next != null) next.processCommit(cmd);
403+
if (next != null) {
404+
next.processCommit(cmd);
405+
producerMetrics.getLocal().inc();
406+
}
394407
if (!mirrorCommits) {
395408
return;
396409
}
@@ -424,8 +437,12 @@ public void processCommit(CommitUpdateCommand cmd) throws IOException {
424437
log.debug(" --doMirroring commit req={}", req);
425438
try {
426439
requestMirroringHandler.mirror(req);
440+
producerMetrics.getSubmitted().inc();
441+
producerMetrics.getSubmittedCommit().inc();
427442
} catch (Exception e) {
428443
log.error("mirror submit failed", e);
444+
producerMetrics.getSubmitError().inc();
445+
producerMetrics.getSubmittedCommitError().inc();
429446
throw new SolrException(SERVER_ERROR, "mirror submit failed", e);
430447
}
431448

solr/modules/cross-dc/src/java/org/apache/solr/crossdc/update/processor/ProducerMetrics.java

Lines changed: 100 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,14 @@ public class ProducerMetrics {
3131
private final AttributedLongCounter localError;
3232
private final AttributedLongCounter submitted;
3333
private final AttributedLongCounter submitError;
34+
private final AttributedLongCounter submittedAdd;
35+
private final AttributedLongCounter submittedAddError;
36+
private final AttributedLongCounter submittedDeleteById;
37+
private final AttributedLongCounter submittedDeleteByIdError;
38+
private final AttributedLongCounter submittedDeleteByQuery;
39+
private final AttributedLongCounter submittedDeleteByQueryError;
40+
private final AttributedLongCounter submittedCommit;
41+
private final AttributedLongCounter submittedCommitError;
3442
private final AttributedLongHistogram documentSize;
3543
private final AttributedLongCounter documentTooLarge;
3644

@@ -45,6 +53,22 @@ public ProducerMetrics(SolrMetricsContext solrMetricsContext, SolrCore solrCore)
4553
solrMetricsContext.longCounter(
4654
"solr_core_crossdc_producer_submitted",
4755
"The number of documents submitted to the Kafka topic (success or error)");
56+
var localSubmittedAdd =
57+
solrMetricsContext.longCounter(
58+
"solr_core_crossdc_producer_submitted_add",
59+
"The number of add requests submitted to the Kafka topic (success or error)");
60+
var localSubmittedDbi =
61+
solrMetricsContext.longCounter(
62+
"solr_core_crossdc_producer_submitted_delete_by_id",
63+
"The number of Delete-By-Id requests submitted to the Kafka topic (success or error)");
64+
var localSubmittedDbq =
65+
solrMetricsContext.longCounter(
66+
"solr_core_crossdc_producer_submitted_delete_by_query",
67+
"The number of Delete-By-Query requests submitted to the Kafka topic (success or error)");
68+
var localSubmittedCommit =
69+
solrMetricsContext.longCounter(
70+
"solr_core_crossdc_producer_submitted_commit",
71+
"The number of standalone Commit requests submitted to the Kafka topic (success or error)");
4872
var histogramDocSizes =
4973
solrMetricsContext.longHistogram(
5074
"solr_core_crossdc_producer_document_size",
@@ -67,6 +91,30 @@ public ProducerMetrics(SolrMetricsContext solrMetricsContext, SolrCore solrCore)
6791
this.submitError =
6892
new AttributedLongCounter(
6993
localSubmitted, attributes.toBuilder().put(TYPE_ATTR, "error").build());
94+
this.submittedAdd =
95+
new AttributedLongCounter(
96+
localSubmittedAdd, attributes.toBuilder().put(TYPE_ATTR, "success").build());
97+
this.submittedAddError =
98+
new AttributedLongCounter(
99+
localSubmittedAdd, attributes.toBuilder().put(TYPE_ATTR, "error").build());
100+
this.submittedDeleteById =
101+
new AttributedLongCounter(
102+
localSubmittedDbi, attributes.toBuilder().put(TYPE_ATTR, "success").build());
103+
this.submittedDeleteByIdError =
104+
new AttributedLongCounter(
105+
localSubmittedDbi, attributes.toBuilder().put(TYPE_ATTR, "error").build());
106+
this.submittedDeleteByQuery =
107+
new AttributedLongCounter(
108+
localSubmittedDbq, attributes.toBuilder().put(TYPE_ATTR, "success").build());
109+
this.submittedDeleteByQueryError =
110+
new AttributedLongCounter(
111+
localSubmittedDbq, attributes.toBuilder().put(TYPE_ATTR, "error").build());
112+
this.submittedCommit =
113+
new AttributedLongCounter(
114+
localSubmittedCommit, attributes.toBuilder().put(TYPE_ATTR, "success").build());
115+
this.submittedCommitError =
116+
new AttributedLongCounter(
117+
localSubmittedCommit, attributes.toBuilder().put(TYPE_ATTR, "error").build());
70118
this.documentSize = new AttributedLongHistogram(histogramDocSizes, attributes);
71119
this.documentTooLarge = new AttributedLongCounter(tooLargeErrors, attributes);
72120
}
@@ -94,6 +142,58 @@ public AttributedLongCounter getSubmitError() {
94142
return this.submitError;
95143
}
96144

145+
/** Counter representing the number of add requests submitted to the Kafka topic. */
146+
public AttributedLongCounter getSubmittedAdd() {
147+
return this.submittedAdd;
148+
}
149+
150+
/**
151+
* Counter representing the number of add requests that were not submitted to the Kafka topic
152+
* because of exception during execution.
153+
*/
154+
public AttributedLongCounter getSubmittedAddError() {
155+
return this.submittedAddError;
156+
}
157+
158+
/** Counter representing the number of delete-by-id requests submitted to the Kafka topic. */
159+
public AttributedLongCounter getSubmittedDeleteById() {
160+
return this.submittedDeleteById;
161+
}
162+
163+
/**
164+
* Counter representing the number of delete-by-id requests that were not submitted to the Kafka
165+
* topic because of exception during execution.
166+
*/
167+
public AttributedLongCounter getSubmittedDeleteByIdError() {
168+
return this.submittedDeleteByIdError;
169+
}
170+
171+
/** Counter representing the number of delete-by-query requests submitted to the Kafka topic. */
172+
public AttributedLongCounter getSubmittedDeleteByQuery() {
173+
return this.submittedDeleteByQuery;
174+
}
175+
176+
/**
177+
* Counter representing the number of delete-by-query requests that were not submitted to the
178+
* Kafka topic because of exception during execution.
179+
*/
180+
public AttributedLongCounter getSubmittedDeleteByQueryError() {
181+
return this.submittedDeleteByQueryError;
182+
}
183+
184+
/** Counter representing the number of standalone Commit requests submitted to the Kafka topic. */
185+
public AttributedLongCounter getSubmittedCommit() {
186+
return this.submittedCommit;
187+
}
188+
189+
/**
190+
* Counter representing the number of standalone Commit requests that were not submitted to the
191+
* Kafka topic because of exception during execution.
192+
*/
193+
public AttributedLongCounter getSubmittedCommitError() {
194+
return this.submittedCommitError;
195+
}
196+
97197
/** Histogram of the processed document size. */
98198
public AttributedLongHistogram getDocumentSize() {
99199
return this.documentSize;

0 commit comments

Comments
 (0)