From 77078fedb107157fbf13781521721ccc146658d8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jesse=20Tu=C4=9Flu?= Date: Thu, 18 Dec 2025 11:43:54 -0800 Subject: [PATCH 1/5] Add reason dimension to ingest/events/thrownAway metric --- docs/operations/metrics.md | 2 +- .../resources/defaultMetricDimensions.json | 3 +- .../src/main/resources/defaultMetrics.json | 3 +- .../src/main/resources/defaultMetrics.json | 2 +- .../resources/defaultMetricDimensions.json | 2 +- .../stats/DropwizardRowIngestionMeters.java | 21 ++- .../stats/TaskRealtimeMetricsMonitor.java | 29 +++- .../common/task/AbstractBatchIndexTask.java | 2 +- .../FilteringCloseableInputRowIterator.java | 17 +- .../druid/indexing/common/task/RowFilter.java | 76 +++++++++ .../SeekableStreamIndexTaskRunner.java | 30 ++-- .../seekablestream/StreamChunkParser.java | 11 +- .../TaskRealtimeMetricsMonitorTest.java | 149 ++++++++++++++++++ .../DropwizardRowIngestionMetersTest.java | 107 +++++++++++++ ...ilteringCloseableInputRowIteratorTest.java | 116 +++++++++++++- .../indexing/common/task/RowFilterTest.java | 122 ++++++++++++++ .../SeekableStreamIndexTaskRunnerTest.java | 58 ++++++- .../seekablestream/StreamChunkParserTest.java | 21 +-- .../incremental/NoopRowIngestionMeters.java | 8 +- .../incremental/RowIngestionMeters.java | 11 +- .../incremental/SimpleRowIngestionMeters.java | 17 +- .../segment/incremental/ThrownAwayReason.java | 64 ++++++++ .../SimpleRowIngestionMetersTest.java | 39 ++++- .../incremental/ThrownAwayReasonTest.java | 51 ++++++ 24 files changed, 901 insertions(+), 60 deletions(-) create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java create mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md index aa1c25a693ad..e2c957b9cee4 100644 --- a/docs/operations/metrics.md +++ b/docs/operations/metrics.md @@ -270,7 +270,7 @@ batch ingestion emit the following metrics. These metrics are deltas for each em |`ingest/events/processed`|Number of events processed per emission period.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Equal to the number of events per emission period.| |`ingest/events/processedWithError`|Number of events processed with some partial errors per emission period. Events processed with partial errors are counted towards both this metric and `ingest/events/processed`.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| |`ingest/events/unparseable`|Number of events rejected because the events are unparseable.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| -|`ingest/events/thrownAway`|Number of events rejected because they are null, or filtered by `transformSpec`, or outside one of `lateMessageRejectionPeriod`, `earlyMessageRejectionPeriod`.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| +|`ingest/events/thrownAway`|Number of events rejected because they are null, or filtered by `transformSpec`, or outside one of `lateMessageRejectionPeriod`, `earlyMessageRejectionPeriod`. The `reason` dimension indicates why the event was thrown away.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`, `reason`|0| |`ingest/events/duplicate`|Number of events rejected because the events are duplicated.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0| |`ingest/input/bytes`|Number of bytes read from input sources, after decompression but prior to parsing. This covers all data read, including data that does not end up being fully processed and ingested. For example, this includes data that ends up being rejected for being unparseable or filtered out.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on the amount of data read.| |`ingest/rows/output`|Number of Druid rows persisted.|`dataSource`, `taskId`, `taskType`, `groupId`|Your number of events with rollup.| diff --git a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json index 6b8d9a9887d7..bc0c451870f7 100644 --- a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json @@ -126,7 +126,8 @@ }, "ingest/events/thrownAway": { "dimensions": [ - "dataSource" + "dataSource", + "reason" ], "type": "counter" }, diff --git a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json index 311a06e6e4ca..69100697a62a 100644 --- a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json @@ -54,7 +54,8 @@ "query/cache/total/timeouts": [], "query/cache/total/errors": [], "ingest/events/thrownAway": [ - "dataSource" + "dataSource", + "reason" ], "ingest/events/unparseable": [ "dataSource" diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 82ced1e9abb8..9fd546762ace 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -104,7 +104,7 @@ "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, "ingest/events/processedWithError" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events processed with some partial errors per emission period." }, "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." }, - "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because they are outside the windowPeriod."}, + "ingest/events/thrownAway" : { "dimensions" : ["dataSource", "reason"], "type" : "count", "help": "Number of events rejected because they are null, filtered by transformSpec, or outside the message rejection periods. The reason dimension indicates why: null, beforeMinMessageTime, afterMaxMessageTime, or filtered."}, "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."}, "ingest/input/bytes" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of bytes read from input sources, after decompression but prior to parsing." }, "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."}, diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json index 0e3862f3da56..e6b7f9f594a9 100644 --- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -38,7 +38,7 @@ "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge" }, "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge" }, - "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count" }, + "ingest/events/thrownAway" : { "dimensions" : ["dataSource", "reason"], "type" : "count" }, "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count" }, "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count" }, "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count" }, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java index e9fe0683f225..98b284550efe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java @@ -23,7 +23,9 @@ import com.codahale.metrics.MetricRegistry; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.ThrownAwayReason; +import java.util.EnumMap; import java.util.HashMap; import java.util.Map; @@ -33,11 +35,14 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters public static final String FIVE_MINUTE_NAME = "5m"; public static final String FIFTEEN_MINUTE_NAME = "15m"; + private static final int NUM_THROWN_AWAY_REASONS = ThrownAwayReason.values().length; + private final Meter processed; private final Meter processedBytes; private final Meter processedWithError; private final Meter unparseable; private final Meter thrownAway; + private final Meter[] thrownAwayByReason = new Meter[NUM_THROWN_AWAY_REASONS]; public DropwizardRowIngestionMeters() { @@ -47,6 +52,9 @@ public DropwizardRowIngestionMeters() this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR); this.unparseable = metricRegistry.meter(UNPARSEABLE); this.thrownAway = metricRegistry.meter(THROWN_AWAY); + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + this.thrownAwayByReason[reason.ordinal()] = metricRegistry.meter(THROWN_AWAY + "_" + reason.name()); + } } @Override @@ -104,9 +112,20 @@ public long getThrownAway() } @Override - public void incrementThrownAway() + public void incrementThrownAway(ThrownAwayReason reason) { thrownAway.mark(); + thrownAwayByReason[reason.ordinal()].mark(); + } + + @Override + public Map getThrownAwayByReason() + { + EnumMap result = new EnumMap<>(ThrownAwayReason.class); + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + result.put(reason, thrownAwayByReason[reason.ordinal()].getCount()); + } + return result; } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 4895776a796c..3639294b8c88 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -25,14 +25,19 @@ import org.apache.druid.java.util.metrics.AbstractMonitor; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import java.util.EnumMap; +import java.util.Map; + /** * Emits metrics from {@link SegmentGenerationMetrics} and {@link RowIngestionMeters}. */ public class TaskRealtimeMetricsMonitor extends AbstractMonitor { private static final EmittingLogger log = new EmittingLogger(TaskRealtimeMetricsMonitor.class); + private static final String REASON_DIMENSION = "reason"; private final SegmentGenerationMetrics segmentGenerationMetrics; private final RowIngestionMeters rowIngestionMeters; @@ -40,6 +45,7 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor private SegmentGenerationMetrics previousSegmentGenerationMetrics; private RowIngestionMetersTotals previousRowIngestionMetersTotals; + private Map previousThrownAwayByReason; public TaskRealtimeMetricsMonitor( SegmentGenerationMetrics segmentGenerationMetrics, @@ -52,6 +58,7 @@ public TaskRealtimeMetricsMonitor( this.builder = metricEventBuilder; previousSegmentGenerationMetrics = new SegmentGenerationMetrics(); previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0); + previousThrownAwayByReason = new EnumMap<>(ThrownAwayReason.class); } @Override @@ -60,14 +67,28 @@ public boolean doMonitor(ServiceEmitter emitter) SegmentGenerationMetrics metrics = segmentGenerationMetrics.snapshot(); RowIngestionMetersTotals rowIngestionMetersTotals = rowIngestionMeters.getTotals(); - final long thrownAway = rowIngestionMetersTotals.getThrownAway() - previousRowIngestionMetersTotals.getThrownAway(); - if (thrownAway > 0) { + // Emit per-reason metrics with the reason dimension + final Map currentThrownAwayByReason = rowIngestionMeters.getThrownAwayByReason(); + long totalThrownAway = 0; + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + final long currentCount = currentThrownAwayByReason.getOrDefault(reason, 0L); + final long previousCount = previousThrownAwayByReason.getOrDefault(reason, 0L); + final long delta = currentCount - previousCount; + if (delta > 0) { + totalThrownAway += delta; + emitter.emit( + builder.setDimension(REASON_DIMENSION, reason.getMetricValue()) + .setMetric("ingest/events/thrownAway", delta) + ); + } + } + previousThrownAwayByReason = currentThrownAwayByReason; + if (totalThrownAway > 0) { log.warn( "[%,d] events thrown away. Possible causes: null events, events filtered out by transformSpec, or events outside earlyMessageRejectionPeriod / lateMessageRejectionPeriod.", - thrownAway + totalThrownAway ); } - emitter.emit(builder.setMetric("ingest/events/thrownAway", thrownAway)); final long unparseable = rowIngestionMetersTotals.getUnparseable() - previousRowIngestionMetersTotals.getUnparseable(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 97dc70f33578..61351448349f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -230,7 +230,7 @@ public static FilteringCloseableInputRowIterator inputSourceReader( ); return new FilteringCloseableInputRowIterator( inputSourceReader.read(ingestionMeters), - rowFilter, + RowFilter.fromPredicate(rowFilter), ingestionMeters, parseExceptionHandler ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java index 30af8febaaee..a846a4a1f4ee 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java @@ -24,20 +24,20 @@ import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.ThrownAwayReason; import java.io.IOException; import java.util.NoSuchElementException; -import java.util.function.Predicate; /** * An {@link InputRow} iterator used by ingestion {@link Task}s. It can filter out rows which do not satisfy the given - * {@link #filter} or throw {@link ParseException} while parsing them. The relevant metric should be counted whenever + * {@link RowFilter} or throw {@link ParseException} while parsing them. The relevant metric should be counted whenever * it filters out rows based on the filter. ParseException handling is delegatged to {@link ParseExceptionHandler}. */ public class FilteringCloseableInputRowIterator implements CloseableIterator { private final CloseableIterator delegate; - private final Predicate filter; + private final RowFilter rowFilter; private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; @@ -45,13 +45,13 @@ public class FilteringCloseableInputRowIterator implements CloseableIterator delegate, - Predicate filter, + RowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) { this.delegate = delegate; - this.filter = filter; + this.rowFilter = rowFilter; this.rowIngestionMeters = rowIngestionMeters; this.parseExceptionHandler = parseExceptionHandler; } @@ -66,11 +66,12 @@ public boolean hasNext() while (next == null && delegate.hasNext()) { // delegate.next() can throw ParseException final InputRow row = delegate.next(); - // filter.test() can throw ParseException - if (filter.test(row)) { + // rowFilter.test() can throw ParseException, returns null if accepted, or reason if rejected + final ThrownAwayReason rejectionReason = rowFilter.test(row); + if (rejectionReason == null) { next = row; } else { - rowIngestionMeters.incrementThrownAway(); + rowIngestionMeters.incrementThrownAway(rejectionReason); } } break; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java new file mode 100644 index 000000000000..1e599105fe75 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java @@ -0,0 +1,76 @@ +/* + * 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.druid.indexing.common.task; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.segment.incremental.ThrownAwayReason; + +import javax.annotation.Nullable; +import java.util.function.Predicate; + +/** + * A filter for input rows during ingestion that can report the reason for rejection or null for acceptance. + * This is similar to {@link Predicate} but returns the rejection reason instead of just a boolean. + */ +@FunctionalInterface +public interface RowFilter +{ + /** + * Tests whether the given row should be accepted. + * + * @param row the input row to test + * @return null if the row should be accepted, or the {@link ThrownAwayReason} if the row should be rejected + */ + @Nullable + ThrownAwayReason test(InputRow row); + + /** + * Creates a {@link RowFilter} from a Predicate. When the predicate returns false, + * the rejection reason will be {@link ThrownAwayReason#FILTERED}. + */ + static RowFilter fromPredicate(Predicate predicate) + { + return row -> predicate.test(row) ? null : ThrownAwayReason.FILTERED; + } + + /** + * Fully-permissive {@link RowFilter} used mainly for tests. + */ + static RowFilter allow() + { + return row -> null; + } + + /** + * Combines this filter with another filter. A row is rejected if either filter rejects it. + * The rejection reason from the first rejecting filter (this filter first) is returned. + */ + default RowFilter and(RowFilter other) + { + return row -> { + ThrownAwayReason reason = this.test(row); + if (reason != null) { + return reason; + } + return other.test(row); + }; + } +} + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 16505f761bbb..a77ffb58d682 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -66,6 +66,7 @@ import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; +import org.apache.druid.indexing.common.task.RowFilter; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -81,6 +82,7 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -419,7 +421,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception inputRowSchema, task.getDataSchema().getTransformSpec(), toolbox.getIndexingTmpDir(), - row -> row != null && withinMinMaxRecordTime(row), + this::getRowRejectionReason, rowIngestionMeters, parseExceptionHandler ); @@ -2144,26 +2146,36 @@ private void refreshMinMaxMessageTime() ); } - public boolean withinMinMaxRecordTime(final InputRow row) + /** + * Returns the rejection reason for a row, or null if the row should be accepted. + * This method is used as a {@link RowFilter} for the {@link StreamChunkParser}. + */ + @Nullable + ThrownAwayReason getRowRejectionReason(final InputRow row) { - final boolean beforeMinimumMessageTime = minMessageTime.isAfter(row.getTimestamp()); - final boolean afterMaximumMessageTime = maxMessageTime.isBefore(row.getTimestamp()); - - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { + if (row == null) { + return ThrownAwayReason.NULL; + } + if (minMessageTime.isAfter(row.getTimestamp())) { + if (log.isDebugEnabled()) { log.debug( "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", row.getTimestamp(), minMessageTime ); - } else if (afterMaximumMessageTime) { + } + return ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + } + if (maxMessageTime.isBefore(row.getTimestamp())) { + if (log.isDebugEnabled()) { log.debug( "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", row.getTimestamp(), maxMessageTime ); } + return ThrownAwayReason.AFTER_MAX_MESSAGE_TIME; } - return !beforeMinimumMessageTime && !afterMaximumMessageTime; + return null; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 3ac952c16c2a..6b1f27c8d43c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -28,11 +28,13 @@ import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.common.task.FilteringCloseableInputRowIterator; +import org.apache.druid.indexing.common.task.RowFilter; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; @@ -42,7 +44,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.function.Predicate; /** * Abstraction for parsing stream data which internally uses {@link org.apache.druid.data.input.InputEntityReader} @@ -54,7 +55,7 @@ class StreamChunkParser private final InputRowParser parser; @Nullable private final SettableByteEntityReader byteEntityReader; - private final Predicate rowFilter; + private final RowFilter rowFilter; private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; @@ -67,7 +68,7 @@ class StreamChunkParser InputRowSchema inputRowSchema, TransformSpec transformSpec, File indexingTmpDir, - Predicate rowFilter, + RowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -96,7 +97,7 @@ class StreamChunkParser StreamChunkParser( @Nullable InputRowParser parser, @Nullable SettableByteEntityReader byteEntityReader, - Predicate rowFilter, + RowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -117,7 +118,7 @@ List parse(@Nullable List streamChunk, boolean isEndOfShar if (!isEndOfShard) { // We do not count end of shard record as thrown away event since this is a record created by Druid // Note that this only applies to Kinesis - rowIngestionMeters.incrementThrownAway(); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); } return Collections.emptyList(); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java index 252d265d67d8..5ab65a014ce1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java @@ -28,6 +28,8 @@ import org.apache.druid.java.util.metrics.MonitorUtils; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.RowIngestionMeters; +import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.junit.Assert; import org.junit.Before; @@ -39,7 +41,9 @@ import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; @RunWith(MockitoJUnitRunner.class) @@ -127,6 +131,151 @@ public void testMessageGapAggStats() Assert.assertTrue(emittedEvents.containsKey("ingest/events/avgMessageGap")); } + @Test + public void testThrownAwayEmitsReasonDimension() + { + SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + + List allEmittedEvents = new ArrayList<>(); + ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); + Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + Mockito + .doAnswer(invocation -> { + ServiceMetricEvent e = invocation.getArgument(0); + allEmittedEvents.add(e); + return null; + }) + .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); + + TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( + segmentGenerationMetrics, + realMeters, + createMetricEventBuilder() + ); + + monitor.doMonitor(captureEmitter); + + Map thrownAwayByReason = new HashMap<>(); + for (ServiceMetricEvent event : allEmittedEvents) { + if ("ingest/events/thrownAway".equals(event.getMetric())) { + Object reason = event.getUserDims().get("reason"); + thrownAwayByReason.put(reason.toString(), event.getValue().longValue()); + } + } + + Assert.assertEquals(Long.valueOf(2), thrownAwayByReason.get("null")); + Assert.assertEquals(Long.valueOf(3), thrownAwayByReason.get("beforeMinMessageTime")); + Assert.assertEquals(Long.valueOf(1), thrownAwayByReason.get("afterMaxMessageTime")); + Assert.assertEquals(Long.valueOf(4), thrownAwayByReason.get("filtered")); + } + + @Test + public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() + { + SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + + List allEmittedEvents = new ArrayList<>(); + ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); + Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + Mockito + .doAnswer(invocation -> { + ServiceMetricEvent e = invocation.getArgument(0); + allEmittedEvents.add(e); + return null; + }) + .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); + + TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( + segmentGenerationMetrics, + realMeters, + createMetricEventBuilder() + ); + + monitor.doMonitor(captureEmitter); + + List emittedReasons = new ArrayList<>(); + for (ServiceMetricEvent event : allEmittedEvents) { + if ("ingest/events/thrownAway".equals(event.getMetric())) { + Object reason = event.getUserDims().get("reason"); + emittedReasons.add(reason.toString()); + } + } + + // Only reasons with non-zero counts should be emitted + Assert.assertEquals(2, emittedReasons.size()); + Assert.assertTrue(emittedReasons.contains("null")); + Assert.assertTrue(emittedReasons.contains("filtered")); + Assert.assertFalse(emittedReasons.contains("beforeMinMessageTime")); + Assert.assertFalse(emittedReasons.contains("afterMaxMessageTime")); + } + + @Test + public void testThrownAwayReasonDeltaAcrossMonitorCalls() + { + SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); + + List allEmittedEvents = new ArrayList<>(); + ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); + Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); + Mockito + .doAnswer(invocation -> { + ServiceMetricEvent e = invocation.getArgument(0); + allEmittedEvents.add(e); + return null; + }) + .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); + + TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( + segmentGenerationMetrics, + realMeters, + createMetricEventBuilder() + ); + + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + monitor.doMonitor(captureEmitter); + + long firstCallNullCount = 0; + for (ServiceMetricEvent event : allEmittedEvents) { + if ("ingest/events/thrownAway".equals(event.getMetric()) + && "null".equals(event.getUserDims().get("reason"))) { + firstCallNullCount = event.getValue().longValue(); + } + } + Assert.assertEquals(2, firstCallNullCount); + + allEmittedEvents.clear(); + realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + monitor.doMonitor(captureEmitter); + + // Find counts from second call - should be deltas only + Map secondCallCounts = new HashMap<>(); + for (ServiceMetricEvent event : allEmittedEvents) { + if ("ingest/events/thrownAway".equals(event.getMetric())) { + Object reason = event.getUserDims().get("reason"); + secondCallCounts.put(reason.toString(), event.getValue().longValue()); + } + } + + // Should emit only the delta (1 more NULL, 2 new FILTERED) + Assert.assertEquals(Long.valueOf(1), secondCallCounts.get("null")); + Assert.assertEquals(Long.valueOf(2), secondCallCounts.get("filtered")); + } + private ServiceMetricEvent.Builder createMetricEventBuilder() { final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java new file mode 100644 index 000000000000..0f2069d7ea2d --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java @@ -0,0 +1,107 @@ +/* + * 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.druid.indexing.common.stats; + +import org.apache.druid.segment.incremental.RowIngestionMetersTotals; +import org.apache.druid.segment.incremental.ThrownAwayReason; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class DropwizardRowIngestionMetersTest +{ + @Test + public void testBasicIncrements() + { + DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); + meters.incrementProcessed(); + meters.incrementProcessedBytes(100); + meters.incrementProcessedWithError(); + meters.incrementUnparseable(); + meters.incrementThrownAway(ThrownAwayReason.NULL); + + Assert.assertEquals(1, meters.getProcessed()); + Assert.assertEquals(100, meters.getProcessedBytes()); + Assert.assertEquals(1, meters.getProcessedWithError()); + Assert.assertEquals(1, meters.getUnparseable()); + Assert.assertEquals(1, meters.getThrownAway()); + + RowIngestionMetersTotals totals = meters.getTotals(); + Assert.assertEquals(1, totals.getProcessed()); + Assert.assertEquals(100, totals.getProcessedBytes()); + Assert.assertEquals(1, totals.getProcessedWithError()); + Assert.assertEquals(1, totals.getUnparseable()); + Assert.assertEquals(1, totals.getThrownAway()); + } + + @Test + public void testIncrementThrownAwayWithReason() + { + DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); + + meters.incrementThrownAway(ThrownAwayReason.NULL); + meters.incrementThrownAway(ThrownAwayReason.NULL); + meters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + meters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + meters.incrementThrownAway(ThrownAwayReason.FILTERED); + meters.incrementThrownAway(ThrownAwayReason.FILTERED); + meters.incrementThrownAway(ThrownAwayReason.FILTERED); + + // Total thrownAway should be sum of all reasons + Assert.assertEquals(7, meters.getThrownAway()); + + // Check per-reason counts + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.NULL)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(3), byReason.get(ThrownAwayReason.FILTERED)); + } + + @Test + public void testGetThrownAwayByReasonReturnsAllReasons() + { + DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); + + // Even with no increments, all reasons should be present with 0 counts + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(ThrownAwayReason.values().length, byReason.size()); + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); + } + } + + @Test + public void testMovingAverages() + { + DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); + + meters.incrementProcessed(); + meters.incrementThrownAway(ThrownAwayReason.FILTERED); + + Map movingAverages = meters.getMovingAverages(); + Assert.assertNotNull(movingAverages); + Assert.assertTrue(movingAverages.containsKey(DropwizardRowIngestionMeters.ONE_MINUTE_NAME)); + Assert.assertTrue(movingAverages.containsKey(DropwizardRowIngestionMeters.FIVE_MINUTE_NAME)); + Assert.assertTrue(movingAverages.containsKey(DropwizardRowIngestionMeters.FIFTEEN_MINUTE_NAME)); + } +} + diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java index 2b4ffe26f31b..4d54576eb65a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java @@ -31,6 +31,7 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Before; @@ -42,6 +43,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -78,7 +80,7 @@ public void testFilterOutRows() final Predicate filter = row -> (Integer) row.getRaw("dim1") == 10; final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( CloseableIterators.withEmptyBaggage(ROWS.iterator()), - filter, + RowFilter.fromPredicate(filter), rowIngestionMeters, parseExceptionHandler ); @@ -125,7 +127,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -163,7 +165,7 @@ public boolean test(InputRow inputRow) final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - filter, + RowFilter.fromPredicate(filter), rowIngestionMeters, parseExceptionHandler ); @@ -214,7 +216,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -260,7 +262,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -281,7 +283,7 @@ public void testCloseDelegateIsClosed() throws IOException ); final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( delegate, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -330,7 +332,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -350,6 +352,106 @@ public void close() } + @Test + public void testRowFilterWithReasons() + { + // RowFilter that returns different reasons based on dim1 value + final RowFilter rowFilter = row -> { + int dim1 = (Integer) row.getRaw("dim1"); + if (dim1 == 10) { + return null; // accept + } else if (dim1 == 20) { + return ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + } else { + return ThrownAwayReason.FILTERED; + } + }; + + final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( + CloseableIterators.withEmptyBaggage(ROWS.iterator()), + rowFilter, + rowIngestionMeters, + parseExceptionHandler + ); + + final List filteredRows = new ArrayList<>(); + rowIterator.forEachRemaining(filteredRows::add); + + // Only rows with dim1=10 should pass + Assert.assertEquals(4, filteredRows.size()); + for (InputRow row : filteredRows) { + Assert.assertEquals(10, row.getRaw("dim1")); + } + + // Check total thrown away + Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); + + // Check per-reason counts + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.NULL)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); // dim1=20 + Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.FILTERED)); // dim1=30 + } + + @Test + public void testRowFilterFromPredicate() + { + // Use the static helper to convert a Predicate to RowFilter + final Predicate predicate = row -> (Integer) row.getRaw("dim1") == 10; + final RowFilter rowFilter = RowFilter.fromPredicate(predicate); + + final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( + CloseableIterators.withEmptyBaggage(ROWS.iterator()), + rowFilter, + rowIngestionMeters, + parseExceptionHandler + ); + + final List filteredRows = new ArrayList<>(); + rowIterator.forEachRemaining(filteredRows::add); + + Assert.assertEquals(4, filteredRows.size()); + Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); + + // All thrown away should have FILTERED reason when using fromPredicate + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.FILTERED)); + Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.NULL)); + Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + } + + @Test + public void testRowFilterAnd() + { + // First filter: reject nulls (simulated by checking dim1) + final RowFilter nullFilter = row -> row == null ? ThrownAwayReason.NULL : null; + + // Second filter: reject if dim1 != 10 + final RowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? null : ThrownAwayReason.FILTERED; + + // Combine filters + final RowFilter combinedFilter = nullFilter.and(valueFilter); + + final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( + CloseableIterators.withEmptyBaggage(ROWS.iterator()), + combinedFilter, + rowIngestionMeters, + parseExceptionHandler + ); + + final List filteredRows = new ArrayList<>(); + rowIterator.forEachRemaining(filteredRows::add); + + Assert.assertEquals(4, filteredRows.size()); + Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); + + // All rejected rows should have FILTERED reason (from second filter) + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.FILTERED)); + } + private static InputRow newRow(DateTime timestamp, Object dim1Val, Object dim2Val) { return new MapBasedInputRow(timestamp, DIMENSIONS, ImmutableMap.of("dim1", dim1Val, "dim2", dim2Val)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java new file mode 100644 index 000000000000..45ea80bdb1fd --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java @@ -0,0 +1,122 @@ +/* + * 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.druid.indexing.common.task; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.segment.incremental.ThrownAwayReason; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class RowFilterTest +{ + private static final List DIMENSIONS = ImmutableList.of("dim1"); + + @Test + public void testFromPredicateAccept() + { + RowFilter filter = RowFilter.fromPredicate(row -> true); + InputRow row = newRow(100); + + Assert.assertNull(filter.test(row)); + } + + @Test + public void testFromPredicateReject() + { + RowFilter filter = RowFilter.fromPredicate(row -> false); + InputRow row = newRow(100); + + Assert.assertEquals(ThrownAwayReason.FILTERED, filter.test(row)); + } + + @Test + public void testAndBothAccept() + { + RowFilter filter1 = row -> null; + RowFilter filter2 = row -> null; + RowFilter combined = filter1.and(filter2); + + InputRow row = newRow(100); + Assert.assertNull(combined.test(row)); + } + + @Test + public void testAndFirstRejects() + { + RowFilter filter1 = row -> ThrownAwayReason.NULL; + RowFilter filter2 = row -> null; + RowFilter combined = filter1.and(filter2); + + InputRow row = newRow(100); + Assert.assertEquals(ThrownAwayReason.NULL, combined.test(row)); + } + + @Test + public void testAndSecondRejects() + { + RowFilter filter1 = row -> null; + RowFilter filter2 = row -> ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + RowFilter combined = filter1.and(filter2); + + InputRow row = newRow(100); + Assert.assertEquals(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, combined.test(row)); + } + + @Test + public void testAndBothRejectReturnsFirst() + { + RowFilter filter1 = row -> ThrownAwayReason.NULL; + RowFilter filter2 = row -> ThrownAwayReason.FILTERED; + RowFilter combined = filter1.and(filter2); + + InputRow row = newRow(100); + // Should return reason from first filter + Assert.assertEquals(ThrownAwayReason.NULL, combined.test(row)); + } + + @Test + public void testChainedAnd() + { + RowFilter filter1 = row -> null; + RowFilter filter2 = row -> null; + RowFilter filter3 = row -> ThrownAwayReason.AFTER_MAX_MESSAGE_TIME; + + RowFilter combined = filter1.and(filter2).and(filter3); + + InputRow row = newRow(100); + Assert.assertEquals(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME, combined.test(row)); + } + + private static InputRow newRow(Object dim1Val) + { + return new MapBasedInputRow( + DateTimes.of("2020-01-01"), + DIMENSIONS, + ImmutableMap.of("dim1", dim1Val) + ); + } +} + diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index 7518bb7f172a..0ec57c3e5def 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.indexing.DataSchema; import org.joda.time.DateTime; import org.junit.Assert; @@ -105,13 +106,13 @@ public void testWithinMinMaxTime() LockGranularity.TIME_CHUNK); Mockito.when(row.getTimestamp()).thenReturn(now); - Assert.assertTrue(runner.withinMinMaxRecordTime(row)); + Assert.assertNull(runner.getRowRejectionReason(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertFalse(runner.withinMinMaxRecordTime(row)); + Assert.assertEquals(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, runner.getRowRejectionReason(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertFalse(runner.withinMinMaxRecordTime(row)); + Assert.assertEquals(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME, runner.getRowRejectionReason(row)); } @Test @@ -157,13 +158,58 @@ public void testWithinMinMaxTimeNotPopulated() TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, LockGranularity.TIME_CHUNK); - Assert.assertTrue(runner.withinMinMaxRecordTime(row)); + Mockito.when(row.getTimestamp()).thenReturn(now); + Assert.assertNull(runner.getRowRejectionReason(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertTrue(runner.withinMinMaxRecordTime(row)); + Assert.assertNull(runner.getRowRejectionReason(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertTrue(runner.withinMinMaxRecordTime(row)); + Assert.assertNull(runner.getRowRejectionReason(row)); + } + + @Test + public void testGetRowRejectionReasonForNullRow() + { + DimensionsSpec dimensionsSpec = new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("d1"), + new StringDimensionSchema("d2") + ) + ); + DataSchema schema = + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(dimensionsSpec) + .withGranularity( + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null) + ) + .build(); + + SeekableStreamIndexTaskTuningConfig tuningConfig = Mockito.mock(SeekableStreamIndexTaskTuningConfig.class); + SeekableStreamIndexTaskIOConfig ioConfig = Mockito.mock(SeekableStreamIndexTaskIOConfig.class); + SeekableStreamStartSequenceNumbers sequenceNumbers = Mockito.mock(SeekableStreamStartSequenceNumbers.class); + SeekableStreamEndSequenceNumbers endSequenceNumbers = Mockito.mock(SeekableStreamEndSequenceNumbers.class); + + Mockito.when(ioConfig.getRefreshRejectionPeriodsInMinutes()).thenReturn(null); + Mockito.when(ioConfig.getMaximumMessageTime()).thenReturn(null); + Mockito.when(ioConfig.getMinimumMessageTime()).thenReturn(null); + Mockito.when(ioConfig.getInputFormat()).thenReturn(new JsonInputFormat(null, null, null, null, null)); + Mockito.when(ioConfig.getStartSequenceNumbers()).thenReturn(sequenceNumbers); + Mockito.when(ioConfig.getEndSequenceNumbers()).thenReturn(endSequenceNumbers); + + Mockito.when(endSequenceNumbers.getPartitionSequenceNumberMap()).thenReturn(ImmutableMap.of()); + Mockito.when(sequenceNumbers.getStream()).thenReturn("test"); + + Mockito.when(task.getDataSchema()).thenReturn(schema); + Mockito.when(task.getIOConfig()).thenReturn(ioConfig); + Mockito.when(task.getTuningConfig()).thenReturn(tuningConfig); + + TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, + LockGranularity.TIME_CHUNK); + + Assert.assertEquals(ThrownAwayReason.NULL, runner.getRowRejectionReason(null)); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java index 649c0d57e464..00b2199bbb86 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java @@ -33,6 +33,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.task.RowFilter; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; @@ -104,7 +105,7 @@ public void testWithParserAndNullInputformatParseProperly() throws IOException null, null, null, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -121,7 +122,7 @@ public void testWithNullParserAndInputformatParseProperly() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -139,7 +140,7 @@ public void testWithNullParserAndNullInputformatFailToCreateParser() null, null, null, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -169,7 +170,7 @@ public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws I new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -190,7 +191,7 @@ public void parseEmptyNotEndOfShard() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -213,7 +214,7 @@ public void parseEmptyEndOfShard() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -241,7 +242,7 @@ public void testParseMalformedDataWithAllowedParseExceptions_thenNoException() t final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - row -> true, + RowFilter.allow(), rowIngestionMeters, new ParseExceptionHandler( rowIngestionMeters, @@ -279,7 +280,7 @@ public void testParseMalformedDataException() throws IOException final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ); @@ -318,7 +319,7 @@ public void testParseMalformedDataWithUnlimitedAllowedParseExceptions_thenNoExce final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - row -> true, + RowFilter.allow(), rowIngestionMeters, new ParseExceptionHandler( rowIngestionMeters, @@ -354,7 +355,7 @@ public void testWithNullParserAndNullByteEntityReaderFailToInstantiate() () -> new StreamChunkParser<>( null, null, - row -> true, + RowFilter.allow(), rowIngestionMeters, parseExceptionHandler ) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java index bff4f2e6de32..c2e3119cc186 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java @@ -74,11 +74,17 @@ public long getThrownAway() } @Override - public void incrementThrownAway() + public void incrementThrownAway(ThrownAwayReason reason) { } + @Override + public Map getThrownAwayByReason() + { + return Map.of(); + } + @Override public RowIngestionMetersTotals getTotals() { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java index 25fc3bae481a..d32f07af4a05 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java @@ -73,7 +73,16 @@ default long getProcessedBytes() void incrementUnparseable(); long getThrownAway(); - void incrementThrownAway(); + + /** + * Increments the thrown away counter for the specified reason. + */ + void incrementThrownAway(ThrownAwayReason reason); + + /** + * Returns the count of thrown away events for each reason. + */ + Map getThrownAwayByReason(); RowIngestionMetersTotals getTotals(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java index 10293e4e24ae..7ed505c4456d 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java @@ -19,15 +19,19 @@ package org.apache.druid.segment.incremental; +import java.util.EnumMap; import java.util.Map; public class SimpleRowIngestionMeters implements RowIngestionMeters { + private static final int NUM_THROWN_AWAY_REASONS = ThrownAwayReason.values().length; + private long processed; private long processedWithError; private long unparseable; private long thrownAway; private long processedBytes; + private final long[] thrownAwayByReason = new long[NUM_THROWN_AWAY_REASONS]; @Override public long getProcessed() @@ -84,9 +88,20 @@ public long getThrownAway() } @Override - public void incrementThrownAway() + public void incrementThrownAway(ThrownAwayReason reason) { thrownAway++; + thrownAwayByReason[reason.ordinal()]++; + } + + @Override + public Map getThrownAwayByReason() + { + EnumMap result = new EnumMap<>(ThrownAwayReason.class); + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + result.put(reason, thrownAwayByReason[reason.ordinal()]); + } + return result; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java b/processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java new file mode 100644 index 000000000000..91e587f5f1b3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java @@ -0,0 +1,64 @@ +/* + * 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.druid.segment.incremental; + +/** + * Reasons why an input row may be thrown away during ingestion. + */ +public enum ThrownAwayReason +{ + /** + * The row was null or the input record was empty. + */ + NULL, + + /** + * The row's timestamp is before the minimum message time (late message rejection). + */ + BEFORE_MIN_MESSAGE_TIME, + + /** + * The row's timestamp is after the maximum message time (early message rejection). + */ + AFTER_MAX_MESSAGE_TIME, + + /** + * The row was filtered out by a transformSpec filter or other row filter. + */ + FILTERED; + + /** + * Pre-computed metric dimension values, indexed by ordinal. + */ + private static final String[] METRIC_VALUES = { + "null", + "beforeMinMessageTime", + "afterMaxMessageTime", + "filtered" + }; + + /** + * Returns the value to be used as the dimension value in metrics. + */ + public String getMetricValue() + { + return METRIC_VALUES[ordinal()]; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java index 5f46129c1707..9076fcaf59d8 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java @@ -22,6 +22,8 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Map; + public class SimpleRowIngestionMetersTest { @Test @@ -32,7 +34,7 @@ public void testIncrement() rowIngestionMeters.incrementProcessedBytes(5); rowIngestionMeters.incrementProcessedWithError(); rowIngestionMeters.incrementUnparseable(); - rowIngestionMeters.incrementThrownAway(); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); Assert.assertEquals(rowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 5, 1, 1, 1)); } @@ -44,4 +46,39 @@ public void testAddRowIngestionMetersTotals() rowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); Assert.assertEquals(rowIngestionMeters.getTotals(), rowIngestionMetersTotals); } + + @Test + public void testIncrementThrownAwayWithReason() + { + SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + + Assert.assertEquals(7, rowIngestionMeters.getThrownAway()); + + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.NULL)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(3), byReason.get(ThrownAwayReason.FILTERED)); + } + + @Test + public void testGetThrownAwayByReasonReturnsAllReasons() + { + SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); + + // Even with no increments, all reasons should be present with 0 counts + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(ThrownAwayReason.values().length, byReason.size()); + for (ThrownAwayReason reason : ThrownAwayReason.values()) { + Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); + } + } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java new file mode 100644 index 000000000000..3e2ec62dd955 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java @@ -0,0 +1,51 @@ +/* + * 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.druid.segment.incremental; + +import org.junit.Assert; +import org.junit.Test; + +public class ThrownAwayReasonTest +{ + @Test + public void testOrdinalValues() + { + Assert.assertEquals(0, ThrownAwayReason.NULL.ordinal()); + Assert.assertEquals(1, ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.ordinal()); + Assert.assertEquals(2, ThrownAwayReason.AFTER_MAX_MESSAGE_TIME.ordinal()); + Assert.assertEquals(3, ThrownAwayReason.FILTERED.ordinal()); + } + + @Test + public void testMetricValues() + { + Assert.assertEquals("null", ThrownAwayReason.NULL.getMetricValue()); + Assert.assertEquals("beforeMinMessageTime", ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.getMetricValue()); + Assert.assertEquals("afterMaxMessageTime", ThrownAwayReason.AFTER_MAX_MESSAGE_TIME.getMetricValue()); + Assert.assertEquals("filtered", ThrownAwayReason.FILTERED.getMetricValue()); + } + + @Test + public void testEnumCount() + { + Assert.assertEquals(4, ThrownAwayReason.values().length); + } +} + From db6a2b974541a03de9ec4f961e89c5e54a66896f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jesse=20Tu=C4=9Flu?= Date: Fri, 19 Dec 2025 14:44:06 -0800 Subject: [PATCH 2/5] Store thrownAwayByReason in RowIngestionMetersTotals --- .../stats/DropwizardRowIngestionMeters.java | 32 +++++------ .../stats/TaskRealtimeMetricsMonitor.java | 15 +++--- .../common/task/AbstractBatchIndexTask.java | 2 +- .../FilteringCloseableInputRowIterator.java | 10 ++-- .../{RowFilter.java => InputRowFilter.java} | 24 ++++----- .../parallel/ParallelIndexSupervisorTask.java | 16 ++++++ .../SeekableStreamIndexTaskRunner.java | 26 +++++---- .../seekablestream/StreamChunkParser.java | 12 ++--- .../TaskRealtimeMetricsMonitorTest.java | 36 ++++++------- .../DropwizardRowIngestionMetersTest.java | 36 ++++++------- ...ilteringCloseableInputRowIteratorTest.java | 54 +++++++++---------- ...ilterTest.java => InputRowFilterTest.java} | 52 +++++++++--------- ...ultiPhaseParallelIndexingRowStatsTest.java | 2 +- .../SinglePhaseParallelIndexingTest.java | 7 +-- .../SeekableStreamIndexTaskRunnerTest.java | 18 +++---- .../seekablestream/StreamChunkParserTest.java | 22 ++++---- ...son.java => InputRowThrownAwayReason.java} | 47 ++++++++++------ .../incremental/NoopRowIngestionMeters.java | 8 +-- .../incremental/RowIngestionMeters.java | 4 +- .../incremental/RowIngestionMetersTotals.java | 46 +++++++++++++++- .../incremental/SimpleRowIngestionMeters.java | 21 ++++---- ...java => InputRowThrownAwayReasonTest.java} | 22 ++++---- .../druid/segment/incremental/RowMeters.java | 14 +++-- .../SimpleRowIngestionMetersTest.java | 37 ++++++------- 24 files changed, 323 insertions(+), 240 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{RowFilter.java => InputRowFilter.java} (68%) rename indexing-service/src/test/java/org/apache/druid/indexing/common/task/{RowFilterTest.java => InputRowFilterTest.java} (55%) rename processing/src/main/java/org/apache/druid/segment/incremental/{ThrownAwayReason.java => InputRowThrownAwayReason.java} (54%) rename processing/src/test/java/org/apache/druid/segment/incremental/{ThrownAwayReasonTest.java => InputRowThrownAwayReasonTest.java} (53%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java index 98b284550efe..e1c21d8d1949 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java @@ -21,10 +21,11 @@ import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; -import org.apache.druid.segment.incremental.ThrownAwayReason; +import java.util.Arrays; import java.util.EnumMap; import java.util.HashMap; import java.util.Map; @@ -35,13 +36,12 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters public static final String FIVE_MINUTE_NAME = "5m"; public static final String FIFTEEN_MINUTE_NAME = "15m"; - private static final int NUM_THROWN_AWAY_REASONS = ThrownAwayReason.values().length; + private static final int NUM_THROWN_AWAY_REASONS = InputRowThrownAwayReason.values().length; private final Meter processed; private final Meter processedBytes; private final Meter processedWithError; private final Meter unparseable; - private final Meter thrownAway; private final Meter[] thrownAwayByReason = new Meter[NUM_THROWN_AWAY_REASONS]; public DropwizardRowIngestionMeters() @@ -51,8 +51,7 @@ public DropwizardRowIngestionMeters() this.processedBytes = metricRegistry.meter(PROCESSED_BYTES); this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR); this.unparseable = metricRegistry.meter(UNPARSEABLE); - this.thrownAway = metricRegistry.meter(THROWN_AWAY); - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { this.thrownAwayByReason[reason.ordinal()] = metricRegistry.meter(THROWN_AWAY + "_" + reason.name()); } } @@ -108,21 +107,24 @@ public void incrementUnparseable() @Override public long getThrownAway() { - return thrownAway.getCount(); + long totalThrownAway = 0; + for (Meter meter : thrownAwayByReason) { + totalThrownAway += meter.getCount(); + } + return totalThrownAway; } @Override - public void incrementThrownAway(ThrownAwayReason reason) + public void incrementThrownAway(InputRowThrownAwayReason reason) { - thrownAway.mark(); thrownAwayByReason[reason.ordinal()].mark(); } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { - EnumMap result = new EnumMap<>(ThrownAwayReason.class); - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { result.put(reason, thrownAwayByReason[reason.ordinal()].getCount()); } return result; @@ -135,7 +137,7 @@ public RowIngestionMetersTotals getTotals() processed.getCount(), processedBytes.getCount(), processedWithError.getCount(), - thrownAway.getCount(), + getThrownAwayByReason(), unparseable.getCount() ); } @@ -150,21 +152,21 @@ public Map getMovingAverages() oneMinute.put(PROCESSED_BYTES, processedBytes.getOneMinuteRate()); oneMinute.put(PROCESSED_WITH_ERROR, processedWithError.getOneMinuteRate()); oneMinute.put(UNPARSEABLE, unparseable.getOneMinuteRate()); - oneMinute.put(THROWN_AWAY, thrownAway.getOneMinuteRate()); + oneMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getOneMinuteRate).reduce(0.0, Double::sum)); Map fiveMinute = new HashMap<>(); fiveMinute.put(PROCESSED, processed.getFiveMinuteRate()); fiveMinute.put(PROCESSED_BYTES, processedBytes.getFiveMinuteRate()); fiveMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFiveMinuteRate()); fiveMinute.put(UNPARSEABLE, unparseable.getFiveMinuteRate()); - fiveMinute.put(THROWN_AWAY, thrownAway.getFiveMinuteRate()); + fiveMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFiveMinuteRate).reduce(0.0, Double::sum)); Map fifteenMinute = new HashMap<>(); fifteenMinute.put(PROCESSED, processed.getFifteenMinuteRate()); fifteenMinute.put(PROCESSED_BYTES, processedBytes.getFifteenMinuteRate()); fifteenMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFifteenMinuteRate()); fifteenMinute.put(UNPARSEABLE, unparseable.getFifteenMinuteRate()); - fifteenMinute.put(THROWN_AWAY, thrownAway.getFifteenMinuteRate()); + fifteenMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFifteenMinuteRate).reduce(0.0, Double::sum)); movingAverages.put(ONE_MINUTE_NAME, oneMinute); movingAverages.put(FIVE_MINUTE_NAME, fiveMinute); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 3639294b8c88..0cbd26a3338a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -23,12 +23,11 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.AbstractMonitor; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; -import java.util.EnumMap; import java.util.Map; /** @@ -45,7 +44,6 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor private SegmentGenerationMetrics previousSegmentGenerationMetrics; private RowIngestionMetersTotals previousRowIngestionMetersTotals; - private Map previousThrownAwayByReason; public TaskRealtimeMetricsMonitor( SegmentGenerationMetrics segmentGenerationMetrics, @@ -57,8 +55,7 @@ public TaskRealtimeMetricsMonitor( this.rowIngestionMeters = rowIngestionMeters; this.builder = metricEventBuilder; previousSegmentGenerationMetrics = new SegmentGenerationMetrics(); - previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0); - previousThrownAwayByReason = new EnumMap<>(ThrownAwayReason.class); + previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, Map.of(), 0); } @Override @@ -68,21 +65,21 @@ public boolean doMonitor(ServiceEmitter emitter) RowIngestionMetersTotals rowIngestionMetersTotals = rowIngestionMeters.getTotals(); // Emit per-reason metrics with the reason dimension - final Map currentThrownAwayByReason = rowIngestionMeters.getThrownAwayByReason(); + final Map currentThrownAwayByReason = rowIngestionMetersTotals.getThrownAwayByReason(); + final Map previousThrownAwayByReason = previousRowIngestionMetersTotals.getThrownAwayByReason(); long totalThrownAway = 0; - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { final long currentCount = currentThrownAwayByReason.getOrDefault(reason, 0L); final long previousCount = previousThrownAwayByReason.getOrDefault(reason, 0L); final long delta = currentCount - previousCount; if (delta > 0) { totalThrownAway += delta; emitter.emit( - builder.setDimension(REASON_DIMENSION, reason.getMetricValue()) + builder.setDimension(REASON_DIMENSION, reason.getReason()) .setMetric("ingest/events/thrownAway", delta) ); } } - previousThrownAwayByReason = currentThrownAwayByReason; if (totalThrownAway > 0) { log.warn( "[%,d] events thrown away. Possible causes: null events, events filtered out by transformSpec, or events outside earlyMessageRejectionPeriod / lateMessageRejectionPeriod.", diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 61351448349f..ccdc99dcd059 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -230,7 +230,7 @@ public static FilteringCloseableInputRowIterator inputSourceReader( ); return new FilteringCloseableInputRowIterator( inputSourceReader.read(ingestionMeters), - RowFilter.fromPredicate(rowFilter), + InputRowFilter.fromPredicate(rowFilter), ingestionMeters, parseExceptionHandler ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java index a846a4a1f4ee..51e34756f30c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java @@ -22,22 +22,22 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.ThrownAwayReason; import java.io.IOException; import java.util.NoSuchElementException; /** * An {@link InputRow} iterator used by ingestion {@link Task}s. It can filter out rows which do not satisfy the given - * {@link RowFilter} or throw {@link ParseException} while parsing them. The relevant metric should be counted whenever + * {@link InputRowFilter} or throw {@link ParseException} while parsing them. The relevant metric should be counted whenever * it filters out rows based on the filter. ParseException handling is delegatged to {@link ParseExceptionHandler}. */ public class FilteringCloseableInputRowIterator implements CloseableIterator { private final CloseableIterator delegate; - private final RowFilter rowFilter; + private final InputRowFilter rowFilter; private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; @@ -45,7 +45,7 @@ public class FilteringCloseableInputRowIterator implements CloseableIterator delegate, - RowFilter rowFilter, + InputRowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -67,7 +67,7 @@ public boolean hasNext() // delegate.next() can throw ParseException final InputRow row = delegate.next(); // rowFilter.test() can throw ParseException, returns null if accepted, or reason if rejected - final ThrownAwayReason rejectionReason = rowFilter.test(row); + final InputRowThrownAwayReason rejectionReason = rowFilter.test(row); if (rejectionReason == null) { next = row; } else { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java similarity index 68% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java index 1e599105fe75..b9e95202fdb7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RowFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java @@ -20,7 +20,7 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; -import org.apache.druid.segment.incremental.ThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import javax.annotation.Nullable; import java.util.function.Predicate; @@ -30,30 +30,30 @@ * This is similar to {@link Predicate} but returns the rejection reason instead of just a boolean. */ @FunctionalInterface -public interface RowFilter +public interface InputRowFilter { /** * Tests whether the given row should be accepted. * * @param row the input row to test - * @return null if the row should be accepted, or the {@link ThrownAwayReason} if the row should be rejected + * @return null if the row should be accepted, or the {@link InputRowThrownAwayReason} if the row should be rejected */ @Nullable - ThrownAwayReason test(InputRow row); + InputRowThrownAwayReason test(InputRow row); /** - * Creates a {@link RowFilter} from a Predicate. When the predicate returns false, - * the rejection reason will be {@link ThrownAwayReason#FILTERED}. + * Creates a {@link InputRowFilter} from a Predicate. When the predicate returns false, + * the rejection reason will be {@link InputRowThrownAwayReason#FILTERED}. */ - static RowFilter fromPredicate(Predicate predicate) + static InputRowFilter fromPredicate(Predicate predicate) { - return row -> predicate.test(row) ? null : ThrownAwayReason.FILTERED; + return row -> predicate.test(row) ? null : InputRowThrownAwayReason.FILTERED; } /** - * Fully-permissive {@link RowFilter} used mainly for tests. + * Fully-permissive {@link InputRowFilter} used mainly for tests. */ - static RowFilter allow() + static InputRowFilter allowAll() { return row -> null; } @@ -62,10 +62,10 @@ static RowFilter allow() * Combines this filter with another filter. A row is rejected if either filter rejects it. * The rejection reason from the first rejecting filter (this filter first) is returned. */ - default RowFilter and(RowFilter other) + default InputRowFilter and(InputRowFilter other) { return row -> { - ThrownAwayReason reason = this.test(row); + InputRowThrownAwayReason reason = this.test(row); if (reason != null) { return reason; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 9faa4c2013e1..69405874c4a0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -69,6 +69,7 @@ import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.SegmentSchemaMapping; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -112,6 +113,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -1602,11 +1604,25 @@ private RowIngestionMetersTotals getTotalsFromBuildSegmentsRowStats(Object build return (RowIngestionMetersTotals) buildSegmentsRowStats; } else if (buildSegmentsRowStats instanceof Map) { Map buildSegmentsRowStatsMap = (Map) buildSegmentsRowStats; + + // Convert the thrownAwayByReason map from String keys to InputRowThrownAwayReason enum keys + Map thrownAwayByReason = null; + Object rawThrownAwayByReason = buildSegmentsRowStatsMap.get("thrownAwayByReason"); + if (rawThrownAwayByReason instanceof Map) { + thrownAwayByReason = new EnumMap<>(InputRowThrownAwayReason.class); + Map rawMap = (Map) rawThrownAwayByReason; + for (Map.Entry entry : rawMap.entrySet()) { + InputRowThrownAwayReason reason = InputRowThrownAwayReason.valueOf(entry.getKey().toString()); + thrownAwayByReason.put(reason, ((Number) entry.getValue()).longValue()); + } + } + return new RowIngestionMetersTotals( ((Number) buildSegmentsRowStatsMap.get("processed")).longValue(), ((Number) buildSegmentsRowStatsMap.get("processedBytes")).longValue(), ((Number) buildSegmentsRowStatsMap.get("processedWithError")).longValue(), ((Number) buildSegmentsRowStatsMap.get("thrownAway")).longValue(), + thrownAwayByReason, ((Number) buildSegmentsRowStatsMap.get("unparseable")).longValue() ); } else { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index a77ffb58d682..bb77ef1eed07 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -66,7 +66,7 @@ import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; -import org.apache.druid.indexing.common.task.RowFilter; +import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -79,10 +79,10 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.PendingSegmentRecord; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.ChatHandler; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; @@ -421,7 +421,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception inputRowSchema, task.getDataSchema().getTransformSpec(), toolbox.getIndexingTmpDir(), - this::getRowRejectionReason, + this::ensureRowIsNonNullAndWithinMessageTimeBounds, rowIngestionMeters, parseExceptionHandler ); @@ -2148,33 +2148,31 @@ private void refreshMinMaxMessageTime() /** * Returns the rejection reason for a row, or null if the row should be accepted. - * This method is used as a {@link RowFilter} for the {@link StreamChunkParser}. + * This method is used as a {@link InputRowFilter} for the {@link StreamChunkParser}. */ @Nullable - ThrownAwayReason getRowRejectionReason(final InputRow row) + InputRowThrownAwayReason ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable InputRow row) { if (row == null) { - return ThrownAwayReason.NULL; - } - if (minMessageTime.isAfter(row.getTimestamp())) { + return InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; + } else if (minMessageTime.isAfter(row.getTimestamp())) { if (log.isDebugEnabled()) { log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + "CurrentTimeStamp[%s] is before minimumMessageTime[%s]", row.getTimestamp(), minMessageTime ); } - return ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; - } - if (maxMessageTime.isBefore(row.getTimestamp())) { + return InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + } else if (maxMessageTime.isBefore(row.getTimestamp())) { if (log.isDebugEnabled()) { log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + "CurrentTimeStamp[%s] is after maximumMessageTime[%s]", row.getTimestamp(), maxMessageTime ); } - return ThrownAwayReason.AFTER_MAX_MESSAGE_TIME; + return InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME; } return null; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 6b1f27c8d43c..6201719af722 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -28,13 +28,13 @@ import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.common.task.FilteringCloseableInputRowIterator; -import org.apache.druid.indexing.common.task.RowFilter; +import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.transform.TransformSpec; import javax.annotation.Nullable; @@ -55,7 +55,7 @@ class StreamChunkParser private final InputRowParser parser; @Nullable private final SettableByteEntityReader byteEntityReader; - private final RowFilter rowFilter; + private final InputRowFilter rowFilter; private final RowIngestionMeters rowIngestionMeters; private final ParseExceptionHandler parseExceptionHandler; @@ -68,7 +68,7 @@ class StreamChunkParser InputRowSchema inputRowSchema, TransformSpec transformSpec, File indexingTmpDir, - RowFilter rowFilter, + InputRowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -97,7 +97,7 @@ class StreamChunkParser StreamChunkParser( @Nullable InputRowParser parser, @Nullable SettableByteEntityReader byteEntityReader, - RowFilter rowFilter, + InputRowFilter rowFilter, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler ) @@ -118,7 +118,7 @@ List parse(@Nullable List streamChunk, boolean isEndOfShar if (!isEndOfShard) { // We do not count end of shard record as thrown away event since this is a record created by Druid // Note that this only applies to Kinesis - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); } return Collections.emptyList(); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java index 5ab65a014ce1..fe4150c7607b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java @@ -27,9 +27,9 @@ import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.MonitorUtils; import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; import org.junit.Assert; import org.junit.Before; @@ -135,16 +135,16 @@ public void testMessageGapAggStats() public void testThrownAwayEmitsReasonDimension() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); - realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); List allEmittedEvents = new ArrayList<>(); ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); @@ -183,8 +183,8 @@ public void testThrownAwayEmitsReasonDimension() public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); List allEmittedEvents = new ArrayList<>(); ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); @@ -243,8 +243,8 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() createMetricEventBuilder() ); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); monitor.doMonitor(captureEmitter); long firstCallNullCount = 0; @@ -257,9 +257,9 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() Assert.assertEquals(2, firstCallNullCount); allEmittedEvents.clear(); - realMeters.incrementThrownAway(ThrownAwayReason.NULL); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); monitor.doMonitor(captureEmitter); // Find counts from second call - should be deltas only diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java index 0f2069d7ea2d..2fb526b996fd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java @@ -19,8 +19,8 @@ package org.apache.druid.indexing.common.stats; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.junit.Assert; import org.junit.Test; @@ -36,7 +36,7 @@ public void testBasicIncrements() meters.incrementProcessedBytes(100); meters.incrementProcessedWithError(); meters.incrementUnparseable(); - meters.incrementThrownAway(ThrownAwayReason.NULL); + meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); Assert.assertEquals(1, meters.getProcessed()); Assert.assertEquals(100, meters.getProcessedBytes()); @@ -57,23 +57,23 @@ public void testIncrementThrownAwayWithReason() { DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); - meters.incrementThrownAway(ThrownAwayReason.NULL); - meters.incrementThrownAway(ThrownAwayReason.NULL); - meters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - meters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - meters.incrementThrownAway(ThrownAwayReason.FILTERED); - meters.incrementThrownAway(ThrownAwayReason.FILTERED); - meters.incrementThrownAway(ThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + meters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + meters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); // Total thrownAway should be sum of all reasons Assert.assertEquals(7, meters.getThrownAway()); // Check per-reason counts - Map byReason = meters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.NULL)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(3), byReason.get(ThrownAwayReason.FILTERED)); + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowThrownAwayReason.FILTERED)); } @Test @@ -82,9 +82,9 @@ public void testGetThrownAwayByReasonReturnsAllReasons() DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); // Even with no increments, all reasons should be present with 0 counts - Map byReason = meters.getThrownAwayByReason(); - Assert.assertEquals(ThrownAwayReason.values().length, byReason.size()); - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(InputRowThrownAwayReason.values().length, byReason.size()); + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); } } @@ -95,7 +95,7 @@ public void testMovingAverages() DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); meters.incrementProcessed(); - meters.incrementThrownAway(ThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); Map movingAverages = meters.getMovingAverages(); Assert.assertNotNull(movingAverages); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java index 4d54576eb65a..bdef61bc7c6f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java @@ -28,10 +28,10 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; -import org.apache.druid.segment.incremental.ThrownAwayReason; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Before; @@ -80,7 +80,7 @@ public void testFilterOutRows() final Predicate filter = row -> (Integer) row.getRaw("dim1") == 10; final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( CloseableIterators.withEmptyBaggage(ROWS.iterator()), - RowFilter.fromPredicate(filter), + InputRowFilter.fromPredicate(filter), rowIngestionMeters, parseExceptionHandler ); @@ -127,7 +127,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -165,7 +165,7 @@ public boolean test(InputRow inputRow) final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - RowFilter.fromPredicate(filter), + InputRowFilter.fromPredicate(filter), rowIngestionMeters, parseExceptionHandler ); @@ -216,7 +216,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -262,7 +262,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -283,7 +283,7 @@ public void testCloseDelegateIsClosed() throws IOException ); final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( delegate, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -332,7 +332,7 @@ public void close() final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( parseExceptionThrowingIterator, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -356,14 +356,14 @@ public void close() public void testRowFilterWithReasons() { // RowFilter that returns different reasons based on dim1 value - final RowFilter rowFilter = row -> { + final InputRowFilter rowFilter = row -> { int dim1 = (Integer) row.getRaw("dim1"); if (dim1 == 10) { return null; // accept } else if (dim1 == 20) { - return ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + return InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; } else { - return ThrownAwayReason.FILTERED; + return InputRowThrownAwayReason.FILTERED; } }; @@ -387,11 +387,11 @@ public void testRowFilterWithReasons() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // Check per-reason counts - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.NULL)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); // dim1=20 - Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.FILTERED)); // dim1=30 + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); // dim1=20 + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.FILTERED)); // dim1=30 } @Test @@ -399,7 +399,7 @@ public void testRowFilterFromPredicate() { // Use the static helper to convert a Predicate to RowFilter final Predicate predicate = row -> (Integer) row.getRaw("dim1") == 10; - final RowFilter rowFilter = RowFilter.fromPredicate(predicate); + final InputRowFilter rowFilter = InputRowFilter.fromPredicate(predicate); final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( CloseableIterators.withEmptyBaggage(ROWS.iterator()), @@ -415,24 +415,24 @@ public void testRowFilterFromPredicate() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // All thrown away should have FILTERED reason when using fromPredicate - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.FILTERED)); - Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.NULL)); - Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(0), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.FILTERED)); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); } @Test public void testRowFilterAnd() { // First filter: reject nulls (simulated by checking dim1) - final RowFilter nullFilter = row -> row == null ? ThrownAwayReason.NULL : null; + final InputRowFilter nullFilter = row -> row == null ? InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD : null; // Second filter: reject if dim1 != 10 - final RowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? null : ThrownAwayReason.FILTERED; + final InputRowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? null : InputRowThrownAwayReason.FILTERED; // Combine filters - final RowFilter combinedFilter = nullFilter.and(valueFilter); + final InputRowFilter combinedFilter = nullFilter.and(valueFilter); final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator( CloseableIterators.withEmptyBaggage(ROWS.iterator()), @@ -448,8 +448,8 @@ public void testRowFilterAnd() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // All rejected rows should have FILTERED reason (from second filter) - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.FILTERED)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.FILTERED)); } private static InputRow newRow(DateTime timestamp, Object dim1Val, Object dim2Val) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java similarity index 55% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java index 45ea80bdb1fd..3d9810b10b3c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RowFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java @@ -24,20 +24,20 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.segment.incremental.ThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.junit.Assert; import org.junit.Test; import java.util.List; -public class RowFilterTest +public class InputRowFilterTest { private static final List DIMENSIONS = ImmutableList.of("dim1"); @Test - public void testFromPredicateAccept() + public void test_fromPredicate_whichAllowsAll() { - RowFilter filter = RowFilter.fromPredicate(row -> true); + InputRowFilter filter = InputRowFilter.fromPredicate(row -> true); InputRow row = newRow(100); Assert.assertNull(filter.test(row)); @@ -46,18 +46,18 @@ public void testFromPredicateAccept() @Test public void testFromPredicateReject() { - RowFilter filter = RowFilter.fromPredicate(row -> false); + InputRowFilter filter = InputRowFilter.fromPredicate(row -> false); InputRow row = newRow(100); - Assert.assertEquals(ThrownAwayReason.FILTERED, filter.test(row)); + Assert.assertEquals(InputRowThrownAwayReason.FILTERED, filter.test(row)); } @Test public void testAndBothAccept() { - RowFilter filter1 = row -> null; - RowFilter filter2 = row -> null; - RowFilter combined = filter1.and(filter2); + InputRowFilter filter1 = InputRowFilter.allowAll(); + InputRowFilter filter2 = InputRowFilter.allowAll(); + InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); Assert.assertNull(combined.test(row)); @@ -66,48 +66,48 @@ public void testAndBothAccept() @Test public void testAndFirstRejects() { - RowFilter filter1 = row -> ThrownAwayReason.NULL; - RowFilter filter2 = row -> null; - RowFilter combined = filter1.and(filter2); + InputRowFilter filter1 = row -> InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; + InputRowFilter filter2 = InputRowFilter.allowAll(); + InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); - Assert.assertEquals(ThrownAwayReason.NULL, combined.test(row)); + Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, combined.test(row)); } @Test public void testAndSecondRejects() { - RowFilter filter1 = row -> null; - RowFilter filter2 = row -> ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; - RowFilter combined = filter1.and(filter2); + InputRowFilter filter1 = InputRowFilter.allowAll(); + InputRowFilter filter2 = row -> InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); - Assert.assertEquals(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, combined.test(row)); + Assert.assertEquals(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, combined.test(row)); } @Test public void testAndBothRejectReturnsFirst() { - RowFilter filter1 = row -> ThrownAwayReason.NULL; - RowFilter filter2 = row -> ThrownAwayReason.FILTERED; - RowFilter combined = filter1.and(filter2); + InputRowFilter filter1 = row -> InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; + InputRowFilter filter2 = row -> InputRowThrownAwayReason.FILTERED; + InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); // Should return reason from first filter - Assert.assertEquals(ThrownAwayReason.NULL, combined.test(row)); + Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, combined.test(row)); } @Test public void testChainedAnd() { - RowFilter filter1 = row -> null; - RowFilter filter2 = row -> null; - RowFilter filter3 = row -> ThrownAwayReason.AFTER_MAX_MESSAGE_TIME; + InputRowFilter filter1 = InputRowFilter.allowAll(); + InputRowFilter filter2 = InputRowFilter.allowAll(); + InputRowFilter filter3 = row -> InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME; - RowFilter combined = filter1.and(filter2).and(filter3); + InputRowFilter combined = filter1.and(filter2).and(filter3); InputRow row = newRow(100); - Assert.assertEquals(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME, combined.test(row)); + Assert.assertEquals(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, combined.test(row)); } private static InputRow newRow(Object dim1Val) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java index 9e910a668157..cdc2e4803cdf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java @@ -172,7 +172,7 @@ public void testRangePartitionRowStats() TaskReport.ReportMap expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of(), - new RowIngestionMetersTotals(200, 5630, 0, 0, 0) + new RowIngestionMetersTotals(200, 5630, 0, 0, null, 0) ); TaskReport.ReportMap actualReports = runTaskAndGetReports(task, TaskState.SUCCESS); compareTaskReports(expectedReports, actualReports); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index cc3cdffa7479..9bd77531088f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -48,6 +48,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; @@ -508,7 +509,7 @@ public void testRunInParallelTaskReports() 1L ) ), - new RowIngestionMetersTotals(10, 335, 1, 1, 1) + new RowIngestionMetersTotals(10, 335, 1, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 1L, InputRowThrownAwayReason.UNKNOWN, 0L), 1) ); compareTaskReports(expectedReports, actualReports); } @@ -543,7 +544,7 @@ public void testRunInSequential() final ParallelIndexSupervisorTask executedTask = (ParallelIndexSupervisorTask) taskContainer.getTask(); TaskReport.ReportMap actualReports = executedTask.doGetLiveReports(true); - final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, 335, 1, 1, 1); + final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, 335, 1, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 1L, InputRowThrownAwayReason.UNKNOWN, 0L), 1); List expectedUnparseableEvents = ImmutableList.of( new ParseExceptionReport( "{ts=2017unparseable}", @@ -564,7 +565,7 @@ public void testRunInSequential() expectedReports = buildExpectedTaskReportSequential( task.getId(), expectedUnparseableEvents, - new RowIngestionMetersTotals(0, 0, 0, 0, 0), + new RowIngestionMetersTotals(0, 0, 0, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 0L, InputRowThrownAwayReason.UNKNOWN, 0L), 0), expectedTotals ); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index 0ec57c3e5def..4fb8fb3289bf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -36,7 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.incremental.ThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.indexing.DataSchema; import org.joda.time.DateTime; import org.junit.Assert; @@ -106,13 +106,13 @@ public void testWithinMinMaxTime() LockGranularity.TIME_CHUNK); Mockito.when(row.getTimestamp()).thenReturn(now); - Assert.assertNull(runner.getRowRejectionReason(row)); + Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertEquals(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, runner.getRowRejectionReason(row)); + Assert.assertEquals(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertEquals(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME, runner.getRowRejectionReason(row)); + Assert.assertEquals(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); } @Test @@ -159,17 +159,17 @@ public void testWithinMinMaxTimeNotPopulated() LockGranularity.TIME_CHUNK); Mockito.when(row.getTimestamp()).thenReturn(now); - Assert.assertNull(runner.getRowRejectionReason(row)); + Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertNull(runner.getRowRejectionReason(row)); + Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertNull(runner.getRowRejectionReason(row)); + Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); } @Test - public void testGetRowRejectionReasonForNullRow() + public void testEnsureRowRejectionReasonForNullRow() { DimensionsSpec dimensionsSpec = new DimensionsSpec( Arrays.asList( @@ -209,7 +209,7 @@ public void testGetRowRejectionReasonForNullRow() TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, LockGranularity.TIME_CHUNK); - Assert.assertEquals(ThrownAwayReason.NULL, runner.getRowRejectionReason(null)); + Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(null)); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java index 00b2199bbb86..92ed5785abb3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/StreamChunkParserTest.java @@ -33,7 +33,7 @@ import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexing.common.task.RowFilter; +import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.RE; @@ -105,7 +105,7 @@ public void testWithParserAndNullInputformatParseProperly() throws IOException null, null, null, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -122,7 +122,7 @@ public void testWithNullParserAndInputformatParseProperly() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -140,7 +140,7 @@ public void testWithNullParserAndNullInputformatFailToCreateParser() null, null, null, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -170,7 +170,7 @@ public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws I new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -191,7 +191,7 @@ public void parseEmptyNotEndOfShard() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -214,7 +214,7 @@ public void parseEmptyEndOfShard() throws IOException new InputRowSchema(TIMESTAMP_SPEC, DimensionsSpec.EMPTY, ColumnsFilter.all()), TransformSpec.NONE, temporaryFolder.newFolder(), - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -242,7 +242,7 @@ public void testParseMalformedDataWithAllowedParseExceptions_thenNoException() t final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, new ParseExceptionHandler( rowIngestionMeters, @@ -280,7 +280,7 @@ public void testParseMalformedDataException() throws IOException final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ); @@ -319,7 +319,7 @@ public void testParseMalformedDataWithUnlimitedAllowedParseExceptions_thenNoExce final StreamChunkParser chunkParser = new StreamChunkParser<>( parser, mockedByteEntityReader, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, new ParseExceptionHandler( rowIngestionMeters, @@ -355,7 +355,7 @@ public void testWithNullParserAndNullByteEntityReaderFailToInstantiate() () -> new StreamChunkParser<>( null, null, - RowFilter.allow(), + InputRowFilter.allowAll(), rowIngestionMeters, parseExceptionHandler ) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java similarity index 54% rename from processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java rename to processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java index 91e587f5f1b3..8f348c1f2f88 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/ThrownAwayReason.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java @@ -19,46 +19,63 @@ package org.apache.druid.segment.incremental; +import java.util.EnumMap; +import java.util.Map; + /** * Reasons why an input row may be thrown away during ingestion. */ -public enum ThrownAwayReason +public enum InputRowThrownAwayReason { /** * The row was null or the input record was empty. */ - NULL, + NULL_OR_EMPTY_RECORD("null"), /** * The row's timestamp is before the minimum message time (late message rejection). */ - BEFORE_MIN_MESSAGE_TIME, + BEFORE_MIN_MESSAGE_TIME("beforeMinMessageTime"), /** * The row's timestamp is after the maximum message time (early message rejection). */ - AFTER_MAX_MESSAGE_TIME, + AFTER_MAX_MESSAGE_TIME("afterMaxMessageTime"), /** * The row was filtered out by a transformSpec filter or other row filter. */ - FILTERED; + FILTERED("filtered"), + + /** + * A backwards-compatible value for tracking filter reasons for ingestion tasks using older Druid versions without filter reason tracking. + */ + UNKNOWN("unknown"); + + private final String reason; + + InputRowThrownAwayReason(String reason) + { + this.reason = reason; + } /** - * Pre-computed metric dimension values, indexed by ordinal. + * Returns string value representation of this {@link InputRowThrownAwayReason} for metric emission. */ - private static final String[] METRIC_VALUES = { - "null", - "beforeMinMessageTime", - "afterMaxMessageTime", - "filtered" - }; + public String getReason() + { + return reason; + } /** - * Returns the value to be used as the dimension value in metrics. + * Public utility for building a mutable frequency map over the possible {@link InputRowThrownAwayReason} values. */ - public String getMetricValue() + public static Map buildBaseCounterMap() { - return METRIC_VALUES[ordinal()]; + final EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { + result.put(reason, 0L); + } + return result; } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java index c2e3119cc186..0ba12f859515 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java @@ -23,13 +23,13 @@ import java.util.Map; /** - * This class is used only in {@code RealtimeIndexTask} which is deprecated now. + * This class is used only in {@code DartFrameContext}. * * Consider using {@link RowIngestionMetersFactory} instead. */ public class NoopRowIngestionMeters implements RowIngestionMeters { - private static final RowIngestionMetersTotals EMPTY_TOTALS = new RowIngestionMetersTotals(0, 0, 0, 0, 0); + private static final RowIngestionMetersTotals EMPTY_TOTALS = new RowIngestionMetersTotals(0, 0, 0, Map.of(), 0); @Override public long getProcessed() @@ -74,13 +74,13 @@ public long getThrownAway() } @Override - public void incrementThrownAway(ThrownAwayReason reason) + public void incrementThrownAway(InputRowThrownAwayReason reason) { } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { return Map.of(); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java index d32f07af4a05..1280a7027a99 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java @@ -77,12 +77,12 @@ default long getProcessedBytes() /** * Increments the thrown away counter for the specified reason. */ - void incrementThrownAway(ThrownAwayReason reason); + void incrementThrownAway(InputRowThrownAwayReason reason); /** * Returns the count of thrown away events for each reason. */ - Map getThrownAwayByReason(); + Map getThrownAwayByReason(); RowIngestionMetersTotals getTotals(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java index 2002bb24ac05..67f7be7b1d2e 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java @@ -21,7 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.common.config.Configs; +import javax.annotation.Nullable; +import java.util.Map; import java.util.Objects; public class RowIngestionMetersTotals @@ -30,6 +33,7 @@ public class RowIngestionMetersTotals private final long processedBytes; private final long processedWithError; private final long thrownAway; + private final Map thrownAwayByReason; private final long unparseable; @JsonCreator @@ -38,13 +42,32 @@ public RowIngestionMetersTotals( @JsonProperty("processedBytes") long processedBytes, @JsonProperty("processedWithError") long processedWithError, @JsonProperty("thrownAway") long thrownAway, + @JsonProperty("thrownAwayByReason") @Nullable Map thrownAwayByReason, + @JsonProperty("unparseable") long unparseable + ) + { + this( + processed, + processedBytes, + processedWithError, + Configs.valueOrDefault(thrownAwayByReason, getBackwardsCompatibleThrownAwayByReason(thrownAway)), + unparseable + ); + } + + public RowIngestionMetersTotals( + @JsonProperty("processed") long processed, + @JsonProperty("processedBytes") long processedBytes, + @JsonProperty("processedWithError") long processedWithError, + @JsonProperty("thrownAwayByReason") Map thrownAwayByReason, @JsonProperty("unparseable") long unparseable ) { this.processed = processed; this.processedBytes = processedBytes; this.processedWithError = processedWithError; - this.thrownAway = thrownAway; + this.thrownAway = thrownAwayByReason.values().stream().reduce(0L, Long::sum); + this.thrownAwayByReason = thrownAwayByReason; this.unparseable = unparseable; } @@ -72,6 +95,12 @@ public long getThrownAway() return thrownAway; } + @JsonProperty + public Map getThrownAwayByReason() + { + return thrownAwayByReason; + } + @JsonProperty public long getUnparseable() { @@ -92,13 +121,14 @@ public boolean equals(Object o) && processedBytes == that.processedBytes && processedWithError == that.processedWithError && thrownAway == that.thrownAway + && thrownAwayByReason.equals(that.thrownAwayByReason) && unparseable == that.unparseable; } @Override public int hashCode() { - return Objects.hash(processed, processedBytes, processedWithError, thrownAway, unparseable); + return Objects.hash(processed, processedBytes, processedWithError, thrownAway, thrownAwayByReason, unparseable); } @Override @@ -109,7 +139,19 @@ public String toString() ", processedBytes=" + processedBytes + ", processedWithError=" + processedWithError + ", thrownAway=" + thrownAway + + ", thrownAwayByReason=" + thrownAwayByReason + ", unparseable=" + unparseable + '}'; } + + /** + * For backwards compatibility, key by {@link InputRowThrownAwayReason} in case of lack of thrownAwayByReason input during rolling Druid upgrades. + * This can occur when tasks running on older Druid versions return ingest statistic payloads to an overlord running on a newer Druid version. + */ + private static Map getBackwardsCompatibleThrownAwayByReason(long thrownAway) + { + Map results = InputRowThrownAwayReason.buildBaseCounterMap(); + results.put(InputRowThrownAwayReason.UNKNOWN, thrownAway); + return results; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java index 7ed505c4456d..0c61eee72bf1 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java @@ -24,12 +24,11 @@ public class SimpleRowIngestionMeters implements RowIngestionMeters { - private static final int NUM_THROWN_AWAY_REASONS = ThrownAwayReason.values().length; + private static final int NUM_THROWN_AWAY_REASONS = InputRowThrownAwayReason.values().length; private long processed; private long processedWithError; private long unparseable; - private long thrownAway; private long processedBytes; private final long[] thrownAwayByReason = new long[NUM_THROWN_AWAY_REASONS]; @@ -84,21 +83,20 @@ public void incrementUnparseable() @Override public long getThrownAway() { - return thrownAway; + return getThrownAwayByReason().values().stream().reduce(0L, Long::sum); } @Override - public void incrementThrownAway(ThrownAwayReason reason) + public void incrementThrownAway(InputRowThrownAwayReason reason) { - thrownAway++; thrownAwayByReason[reason.ordinal()]++; } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { - EnumMap result = new EnumMap<>(ThrownAwayReason.class); - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + final EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { result.put(reason, thrownAwayByReason[reason.ordinal()]); } return result; @@ -111,7 +109,7 @@ public RowIngestionMetersTotals getTotals() processed, processedBytes, processedWithError, - thrownAway, + getThrownAwayByReason(), unparseable ); } @@ -127,7 +125,10 @@ public void addRowIngestionMetersTotals(RowIngestionMetersTotals rowIngestionMet this.processed += rowIngestionMetersTotals.getProcessed(); this.processedWithError += rowIngestionMetersTotals.getProcessedWithError(); this.unparseable += rowIngestionMetersTotals.getUnparseable(); - this.thrownAway += rowIngestionMetersTotals.getThrownAway(); this.processedBytes += rowIngestionMetersTotals.getProcessedBytes(); + + for (Map.Entry entry : rowIngestionMetersTotals.getThrownAwayByReason().entrySet()) { + this.thrownAwayByReason[entry.getKey().ordinal()] += entry.getValue(); + } } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java similarity index 53% rename from processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java rename to processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java index 3e2ec62dd955..aec067608bae 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/ThrownAwayReasonTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java @@ -22,30 +22,30 @@ import org.junit.Assert; import org.junit.Test; -public class ThrownAwayReasonTest +public class InputRowThrownAwayReasonTest { @Test public void testOrdinalValues() { - Assert.assertEquals(0, ThrownAwayReason.NULL.ordinal()); - Assert.assertEquals(1, ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.ordinal()); - Assert.assertEquals(2, ThrownAwayReason.AFTER_MAX_MESSAGE_TIME.ordinal()); - Assert.assertEquals(3, ThrownAwayReason.FILTERED.ordinal()); + Assert.assertEquals(0, InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD.ordinal()); + Assert.assertEquals(1, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.ordinal()); + Assert.assertEquals(2, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME.ordinal()); + Assert.assertEquals(3, InputRowThrownAwayReason.FILTERED.ordinal()); } @Test public void testMetricValues() { - Assert.assertEquals("null", ThrownAwayReason.NULL.getMetricValue()); - Assert.assertEquals("beforeMinMessageTime", ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.getMetricValue()); - Assert.assertEquals("afterMaxMessageTime", ThrownAwayReason.AFTER_MAX_MESSAGE_TIME.getMetricValue()); - Assert.assertEquals("filtered", ThrownAwayReason.FILTERED.getMetricValue()); + Assert.assertEquals("null", InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD.getReason()); + Assert.assertEquals("beforeMinMessageTime", InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.getReason()); + Assert.assertEquals("afterMaxMessageTime", InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME.getReason()); + Assert.assertEquals("filtered", InputRowThrownAwayReason.FILTERED.getReason()); } @Test - public void testEnumCount() + public void testEnumCardinality() { - Assert.assertEquals(4, ThrownAwayReason.values().length); + Assert.assertEquals(5, InputRowThrownAwayReason.values().length); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java index 02b1d290003e..6d6a06df53cd 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java @@ -19,6 +19,8 @@ package org.apache.druid.segment.incremental; +import java.util.Map; + /** * Utility class to build {@link RowIngestionMetersTotals}, used in tests. */ @@ -27,7 +29,7 @@ public class RowMeters private long processedBytes; private long processedWithError; private long unparseable; - private long thrownAway; + private final Map thrownAwayByReason = InputRowThrownAwayReason.buildBaseCounterMap(); /** * Creates a new {@link RowMeters}, that can be used to build an instance of @@ -56,14 +58,20 @@ public RowMeters unparseable(long unparseable) return this; } + public RowMeters thrownAwayByReason(InputRowThrownAwayReason thrownAwayByReason, long thrownAway) + { + this.thrownAwayByReason.put(thrownAwayByReason, this.thrownAwayByReason.get(thrownAwayByReason) + thrownAway); + return this; + } + public RowMeters thrownAway(long thrownAway) { - this.thrownAway = thrownAway; + this.thrownAwayByReason.put(InputRowThrownAwayReason.UNKNOWN, this.thrownAwayByReason.get(InputRowThrownAwayReason.UNKNOWN) + thrownAway); return this; } public RowIngestionMetersTotals totalProcessed(long processed) { - return new RowIngestionMetersTotals(processed, processedBytes, processedWithError, thrownAway, unparseable); + return new RowIngestionMetersTotals(processed, processedBytes, processedWithError, thrownAwayByReason, unparseable); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java index 9076fcaf59d8..a574eaddb67e 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java @@ -34,15 +34,16 @@ public void testIncrement() rowIngestionMeters.incrementProcessedBytes(5); rowIngestionMeters.incrementProcessedWithError(); rowIngestionMeters.incrementUnparseable(); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); - Assert.assertEquals(rowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 5, 1, 1, 1)); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + Assert.assertEquals(rowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 5, 1, Map.of( + InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 1L), 1)); } @Test public void testAddRowIngestionMetersTotals() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, 0, 1); + RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, Map.of(), 1); rowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); Assert.assertEquals(rowIngestionMeters.getTotals(), rowIngestionMetersTotals); } @@ -52,21 +53,21 @@ public void testIncrementThrownAwayWithReason() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.NULL); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); - rowIngestionMeters.incrementThrownAway(ThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); Assert.assertEquals(7, rowIngestionMeters.getThrownAway()); - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(ThrownAwayReason.NULL)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(ThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(3), byReason.get(ThrownAwayReason.FILTERED)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowThrownAwayReason.FILTERED)); } @Test @@ -75,9 +76,9 @@ public void testGetThrownAwayByReasonReturnsAllReasons() SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); // Even with no increments, all reasons should be present with 0 counts - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(ThrownAwayReason.values().length, byReason.size()); - for (ThrownAwayReason reason : ThrownAwayReason.values()) { + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(InputRowThrownAwayReason.values().length, byReason.size()); + for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); } } From f826243c1d05be492d6814b09a6202dfa07a1d2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jesse=20Tu=C4=9Flu?= Date: Fri, 19 Dec 2025 16:37:22 -0800 Subject: [PATCH 3/5] Switch to StubServiceEmitter in TaskRealtimeMetricsMonitorTest --- .../TaskRealtimeMetricsMonitorTest.java | 152 ++++++------------ 1 file changed, 48 insertions(+), 104 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java index fe4150c7607b..2e6e652caab2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java @@ -21,11 +21,9 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; -import org.apache.druid.java.util.emitter.core.Event; -import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.java.util.emitter.service.ServiceEventBuilder; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.MonitorUtils; +import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -34,19 +32,11 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Answers; -import org.mockito.ArgumentMatchers; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; - -import java.util.ArrayList; + import java.util.HashMap; import java.util.List; import java.util.Map; -@RunWith(MockitoJUnitRunner.class) public class TaskRealtimeMetricsMonitorTest { private static final Map DIMENSIONS = ImmutableMap.of( @@ -57,28 +47,18 @@ public class TaskRealtimeMetricsMonitorTest ); private static final Map TAGS = ImmutableMap.of("author", "Author Name", "version", 10); - private SegmentGenerationMetrics segmentGenerationMetrics; - @Mock(answer = Answers.RETURNS_MOCKS) + private SegmentGenerationMetrics segmentGenerationMetrics; private RowIngestionMeters rowIngestionMeters; - @Mock - private ServiceEmitter emitter; - private Map emittedEvents; + private StubServiceEmitter emitter; private TaskRealtimeMetricsMonitor target; @Before public void setUp() { - emittedEvents = new HashMap<>(); segmentGenerationMetrics = new SegmentGenerationMetrics(); - Mockito.doCallRealMethod().when(emitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); - Mockito - .doAnswer(invocation -> { - ServiceMetricEvent e = invocation.getArgument(0); - emittedEvents.put(e.getMetric(), e); - return null; - }) - .when(emitter).emit(ArgumentMatchers.any(Event.class)); + rowIngestionMeters = new SimpleRowIngestionMeters(); + emitter = new StubServiceEmitter(); target = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, rowIngestionMeters, @@ -90,7 +70,10 @@ public void setUp() public void testdoMonitorShouldEmitUserProvidedTags() { target.doMonitor(emitter); - for (ServiceMetricEvent sme : emittedEvents.values()) { + + List events = emitter.getMetricEvents("ingest/events/unparseable"); + Assert.assertFalse(events.isEmpty()); + for (ServiceMetricEvent sme : events) { Assert.assertEquals(TAGS, sme.getUserDims().get(DruidMetrics.TAGS)); } } @@ -98,12 +81,19 @@ public void testdoMonitorShouldEmitUserProvidedTags() @Test public void testdoMonitorWithoutTagsShouldNotEmitTags() { + ServiceMetricEvent.Builder builderWithoutTags = new ServiceMetricEvent.Builder(); + MonitorUtils.addDimensionsToBuilder(builderWithoutTags, DIMENSIONS); + target = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, rowIngestionMeters, - createMetricEventBuilder() + builderWithoutTags ); - for (ServiceMetricEvent sme : emittedEvents.values()) { + target.doMonitor(emitter); + + List events = emitter.getMetricEvents("ingest/events/unparseable"); + Assert.assertFalse(events.isEmpty()); + for (ServiceMetricEvent sme : events) { Assert.assertFalse(sme.getUserDims().containsKey(DruidMetrics.TAGS)); } } @@ -111,24 +101,18 @@ public void testdoMonitorWithoutTagsShouldNotEmitTags() @Test public void testMessageGapAggStats() { - target = new TaskRealtimeMetricsMonitor( - segmentGenerationMetrics, - rowIngestionMeters, - createMetricEventBuilder() - ); - target.doMonitor(emitter); - Assert.assertFalse(emittedEvents.containsKey("ingest/events/minMessageGap")); - Assert.assertFalse(emittedEvents.containsKey("ingest/events/maxMessageGap")); - Assert.assertFalse(emittedEvents.containsKey("ingest/events/avgMessageGap")); + Assert.assertTrue(emitter.getMetricEvents("ingest/events/minMessageGap").isEmpty()); + Assert.assertTrue(emitter.getMetricEvents("ingest/events/maxMessageGap").isEmpty()); + Assert.assertTrue(emitter.getMetricEvents("ingest/events/avgMessageGap").isEmpty()); - emittedEvents.clear(); + emitter.flush(); segmentGenerationMetrics.reportMessageGap(1); target.doMonitor(emitter); - Assert.assertTrue(emittedEvents.containsKey("ingest/events/minMessageGap")); - Assert.assertTrue(emittedEvents.containsKey("ingest/events/maxMessageGap")); - Assert.assertTrue(emittedEvents.containsKey("ingest/events/avgMessageGap")); + Assert.assertFalse(emitter.getMetricEvents("ingest/events/minMessageGap").isEmpty()); + Assert.assertFalse(emitter.getMetricEvents("ingest/events/maxMessageGap").isEmpty()); + Assert.assertFalse(emitter.getMetricEvents("ingest/events/avgMessageGap").isEmpty()); } @Test @@ -146,31 +130,18 @@ public void testThrownAwayEmitsReasonDimension() realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - List allEmittedEvents = new ArrayList<>(); - ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); - Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); - Mockito - .doAnswer(invocation -> { - ServiceMetricEvent e = invocation.getArgument(0); - allEmittedEvents.add(e); - return null; - }) - .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); - TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, realMeters, createMetricEventBuilder() ); - monitor.doMonitor(captureEmitter); + monitor.doMonitor(emitter); Map thrownAwayByReason = new HashMap<>(); - for (ServiceMetricEvent event : allEmittedEvents) { - if ("ingest/events/thrownAway".equals(event.getMetric())) { - Object reason = event.getUserDims().get("reason"); - thrownAwayByReason.put(reason.toString(), event.getValue().longValue()); - } + for (ServiceMetricEvent event : emitter.getMetricEvents("ingest/events/thrownAway")) { + Object reason = event.getUserDims().get("reason"); + thrownAwayByReason.put(reason.toString(), event.getValue().longValue()); } Assert.assertEquals(Long.valueOf(2), thrownAwayByReason.get("null")); @@ -186,39 +157,26 @@ public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - List allEmittedEvents = new ArrayList<>(); - ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); - Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); - Mockito - .doAnswer(invocation -> { - ServiceMetricEvent e = invocation.getArgument(0); - allEmittedEvents.add(e); - return null; - }) - .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); - TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, realMeters, createMetricEventBuilder() ); - monitor.doMonitor(captureEmitter); + monitor.doMonitor(emitter); - List emittedReasons = new ArrayList<>(); - for (ServiceMetricEvent event : allEmittedEvents) { - if ("ingest/events/thrownAway".equals(event.getMetric())) { - Object reason = event.getUserDims().get("reason"); - emittedReasons.add(reason.toString()); - } + Map thrownAwayByReason = new HashMap<>(); + for (ServiceMetricEvent event : emitter.getMetricEvents("ingest/events/thrownAway")) { + Object reason = event.getUserDims().get("reason"); + thrownAwayByReason.put(reason.toString(), event.getValue().longValue()); } // Only reasons with non-zero counts should be emitted - Assert.assertEquals(2, emittedReasons.size()); - Assert.assertTrue(emittedReasons.contains("null")); - Assert.assertTrue(emittedReasons.contains("filtered")); - Assert.assertFalse(emittedReasons.contains("beforeMinMessageTime")); - Assert.assertFalse(emittedReasons.contains("afterMaxMessageTime")); + Assert.assertEquals(2, thrownAwayByReason.size()); + Assert.assertTrue(thrownAwayByReason.containsKey("null")); + Assert.assertTrue(thrownAwayByReason.containsKey("filtered")); + Assert.assertFalse(thrownAwayByReason.containsKey("beforeMinMessageTime")); + Assert.assertFalse(thrownAwayByReason.containsKey("afterMaxMessageTime")); } @Test @@ -226,17 +184,6 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); - List allEmittedEvents = new ArrayList<>(); - ServiceEmitter captureEmitter = Mockito.mock(ServiceEmitter.class); - Mockito.doCallRealMethod().when(captureEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class)); - Mockito - .doAnswer(invocation -> { - ServiceMetricEvent e = invocation.getArgument(0); - allEmittedEvents.add(e); - return null; - }) - .when(captureEmitter).emit(ArgumentMatchers.any(Event.class)); - TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, realMeters, @@ -245,30 +192,27 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - monitor.doMonitor(captureEmitter); + monitor.doMonitor(emitter); long firstCallNullCount = 0; - for (ServiceMetricEvent event : allEmittedEvents) { - if ("ingest/events/thrownAway".equals(event.getMetric()) - && "null".equals(event.getUserDims().get("reason"))) { + for (ServiceMetricEvent event : emitter.getMetricEvents("ingest/events/thrownAway")) { + if ("null".equals(event.getUserDims().get("reason"))) { firstCallNullCount = event.getValue().longValue(); } } Assert.assertEquals(2, firstCallNullCount); - allEmittedEvents.clear(); + emitter.flush(); realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - monitor.doMonitor(captureEmitter); + monitor.doMonitor(emitter); // Find counts from second call - should be deltas only Map secondCallCounts = new HashMap<>(); - for (ServiceMetricEvent event : allEmittedEvents) { - if ("ingest/events/thrownAway".equals(event.getMetric())) { - Object reason = event.getUserDims().get("reason"); - secondCallCounts.put(reason.toString(), event.getValue().longValue()); - } + for (ServiceMetricEvent event : emitter.getMetricEvents("ingest/events/thrownAway")) { + Object reason = event.getUserDims().get("reason"); + secondCallCounts.put(reason.toString(), event.getValue().longValue()); } // Should emit only the delta (1 more NULL, 2 new FILTERED) From decc140a7c1652a3c97356dd547fcd1609dfe2d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jesse=20Tu=C4=9Flu?= Date: Fri, 19 Dec 2025 17:04:03 -0800 Subject: [PATCH 4/5] Switch from null acceptance value to InputRowFilterResult.ACCEPTED --- .../src/main/resources/defaultMetrics.json | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 33 +++++--- .../kinesis/KinesisIndexTaskTest.java | 18 +++-- .../stats/DropwizardRowIngestionMeters.java | 29 ++++--- .../stats/TaskRealtimeMetricsMonitor.java | 23 +++--- .../FilteringCloseableInputRowIterator.java | 10 +-- .../indexing/common/task/InputRowFilter.java | 27 +++---- .../parallel/ParallelIndexSupervisorTask.java | 19 +---- .../SeekableStreamIndexTaskRunner.java | 17 +++-- .../seekablestream/StreamChunkParser.java | 4 +- .../TaskRealtimeMetricsMonitorTest.java | 36 ++++----- .../DropwizardRowIngestionMetersTest.java | 38 +++++----- ...ilteringCloseableInputRowIteratorTest.java | 36 ++++----- .../indexing/common/task/IndexTaskTest.java | 50 ++++++++++-- .../common/task/InputRowFilterTest.java | 26 +++---- ...ultiPhaseParallelIndexingRowStatsTest.java | 2 +- .../SinglePhaseParallelIndexingTest.java | 12 ++- .../SeekableStreamIndexTaskRunnerTest.java | 16 ++-- ...yReason.java => InputRowFilterResult.java} | 49 +++++++++--- .../incremental/NoopRowIngestionMeters.java | 4 +- .../incremental/RowIngestionMeters.java | 8 +- .../incremental/RowIngestionMetersTotals.java | 33 ++++++-- .../incremental/SimpleRowIngestionMeters.java | 28 +++---- .../indexer/report/TaskReportSerdeTest.java | 3 + .../incremental/InputRowFilterResultTest.java | 76 +++++++++++++++++++ .../InputRowThrownAwayReasonTest.java | 51 ------------- .../druid/segment/incremental/RowMeters.java | 8 +- .../SimpleRowIngestionMetersTest.java | 43 ++++++----- 28 files changed, 411 insertions(+), 290 deletions(-) rename processing/src/main/java/org/apache/druid/segment/incremental/{InputRowThrownAwayReason.java => InputRowFilterResult.java} (53%) create mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java delete mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json index 9fd546762ace..3702039c283e 100644 --- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -104,7 +104,7 @@ "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, "ingest/events/processedWithError" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events processed with some partial errors per emission period." }, "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." }, - "ingest/events/thrownAway" : { "dimensions" : ["dataSource", "reason"], "type" : "count", "help": "Number of events rejected because they are null, filtered by transformSpec, or outside the message rejection periods. The reason dimension indicates why: null, beforeMinMessageTime, afterMaxMessageTime, or filtered."}, + "ingest/events/thrownAway" : { "dimensions" : ["dataSource", "reason"], "type" : "count", "help": "Number of events rejected because they are null, filtered by transformSpec, or outside the message rejection periods. The `reason` dimension indicates why the event was thrown away."}, "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."}, "ingest/input/bytes" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of bytes read from input sources, after decompression but prior to parsing." }, "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."}, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 96dd04d42cfe..54ef4b6ece83 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -113,6 +113,7 @@ import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.incremental.RowMeters; @@ -128,6 +129,7 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CollectionUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; @@ -717,7 +719,7 @@ public void testIncrementalHandOff() throws Exception ); long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15); - verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8)); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(8)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -851,7 +853,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ); long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15); - verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8)); + verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(8)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1165,7 +1167,7 @@ public void testRunWithMinimumMessageTime() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3)); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, 2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1214,7 +1216,7 @@ public void testRunWithMaximumMessageTime() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3)); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, 2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -1272,7 +1274,7 @@ public void testRunWithTransformSpec() throws Exception // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(4).totalProcessed(1)); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1)); // Check published metadata final List publishedDescriptors = publishedDescriptors(); @@ -1642,7 +1644,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception verifyTaskMetrics(task, RowMeters.with() .bytes(totalRecordBytes) .unparseable(3).errors(3) - .thrownAway(1).totalProcessed(4)); + .thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(4)); // Check published metadata assertEqualsExceptVersion( @@ -1660,6 +1662,9 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState()); Assert.assertNull(reportData.getErrorMsg()); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); + expectedThrownAwayByReason.put(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1667,7 +1672,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception RowIngestionMeters.PROCESSED_BYTES, (int) totalRecordBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 1 + RowIngestionMeters.THROWN_AWAY, 1, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); @@ -1745,6 +1751,8 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState()); Assert.assertNotNull(reportData.getErrorMsg()); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1752,7 +1760,8 @@ public void testMultipleParseExceptionsFailure() throws Exception RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 0 + RowIngestionMeters.THROWN_AWAY, 0, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); @@ -1887,7 +1896,7 @@ public void testRunConflicting() throws Exception verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10)) - .unparseable(3).thrownAway(1).totalProcessed(3)); + .unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3)); // Check published segments & metadata, should all be from the first task final List publishedDescriptors = publishedDescriptors(); @@ -1961,7 +1970,7 @@ public void testRunConflictingWithoutTransactions() throws Exception verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3)); verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10)) - .unparseable(3).thrownAway(1).totalProcessed(3)); + .unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3)); // Check published segments & metadata SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e")); @@ -2576,7 +2585,7 @@ public void testRunTransactionModeRollback() throws Exception long totalBytes = getTotalSizeOfRecords(0, 2) + getTotalSizeOfRecords(5, 11); verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes) - .unparseable(3).errors(1).thrownAway(1).totalProcessed(3)); + .unparseable(3).errors(1).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -3437,7 +3446,7 @@ public void testTaskWithTransformSpecDoesNotCauseCliPeonCyclicDependency() // Wait for task to exit Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); - verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(4).totalProcessed(1)); + verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1)); // Check published metadata final List publishedDescriptors = publishedDescriptors(); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 98dfea4333cc..c714953811bb 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -82,12 +82,14 @@ import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CollectionUtils; import org.easymock.EasyMock; import org.joda.time.Duration; import org.joda.time.Period; @@ -801,7 +803,7 @@ public void testRunWithMinimumMessageTime() throws Exception verifyAll(); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) - .thrownAway(2).totalProcessed(3)); + .thrownAwayByReason(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, 2).totalProcessed(3)); // Check published metadata assertEqualsExceptVersion( @@ -864,7 +866,7 @@ public void testRunWithMaximumMessageTime() throws Exception verifyAll(); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) - .thrownAway(2).totalProcessed(3)); + .thrownAwayByReason(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, 2).totalProcessed(3)); // Check published metadata and segments in deep storage assertEqualsExceptVersion( @@ -923,7 +925,7 @@ public void testRunWithTransformSpec() throws Exception verifyAll(); verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5)) - .thrownAway(4).totalProcessed(1)); + .thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1)); // Check published metadata assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors()); @@ -1194,6 +1196,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState()); Assert.assertNull(reportData.getErrorMsg()); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1201,7 +1205,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception RowIngestionMeters.PROCESSED_BYTES, 763, RowIngestionMeters.PROCESSED_WITH_ERROR, 3, RowIngestionMeters.UNPARSEABLE, 4, - RowIngestionMeters.THROWN_AWAY, 0 + RowIngestionMeters.THROWN_AWAY, 0, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); @@ -1284,6 +1289,8 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState()); Assert.assertNotNull(reportData.getErrorMsg()); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1291,7 +1298,8 @@ public void testMultipleParseExceptionsFailure() throws Exception RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes, RowIngestionMeters.PROCESSED_WITH_ERROR, 0, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 0 + RowIngestionMeters.THROWN_AWAY, 0, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java index e1c21d8d1949..0c824ff7ad47 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java @@ -21,12 +21,11 @@ import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import java.util.Arrays; -import java.util.EnumMap; import java.util.HashMap; import java.util.Map; @@ -36,13 +35,11 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters public static final String FIVE_MINUTE_NAME = "5m"; public static final String FIFTEEN_MINUTE_NAME = "15m"; - private static final int NUM_THROWN_AWAY_REASONS = InputRowThrownAwayReason.values().length; - private final Meter processed; private final Meter processedBytes; private final Meter processedWithError; private final Meter unparseable; - private final Meter[] thrownAwayByReason = new Meter[NUM_THROWN_AWAY_REASONS]; + private final Meter[] thrownAwayByReason = new Meter[InputRowFilterResult.NUM_FILTER_RESULT]; public DropwizardRowIngestionMeters() { @@ -51,7 +48,7 @@ public DropwizardRowIngestionMeters() this.processedBytes = metricRegistry.meter(PROCESSED_BYTES); this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR); this.unparseable = metricRegistry.meter(UNPARSEABLE); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { + for (InputRowFilterResult reason : InputRowFilterResult.values()) { this.thrownAwayByReason[reason.ordinal()] = metricRegistry.meter(THROWN_AWAY + "_" + reason.name()); } } @@ -108,24 +105,24 @@ public void incrementUnparseable() public long getThrownAway() { long totalThrownAway = 0; - for (Meter meter : thrownAwayByReason) { - totalThrownAway += meter.getCount(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + totalThrownAway += thrownAwayByReason[reason.ordinal()].getCount(); } return totalThrownAway; } @Override - public void incrementThrownAway(InputRowThrownAwayReason reason) + public void incrementThrownAway(InputRowFilterResult reason) { thrownAwayByReason[reason.ordinal()].mark(); } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { - EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - result.put(reason, thrownAwayByReason[reason.ordinal()].getCount()); + Map result = InputRowFilterResult.buildRejectedCounterMap(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + result.put(reason.getReason(), thrownAwayByReason[reason.ordinal()].getCount()); } return result; } @@ -152,21 +149,21 @@ public Map getMovingAverages() oneMinute.put(PROCESSED_BYTES, processedBytes.getOneMinuteRate()); oneMinute.put(PROCESSED_WITH_ERROR, processedWithError.getOneMinuteRate()); oneMinute.put(UNPARSEABLE, unparseable.getOneMinuteRate()); - oneMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getOneMinuteRate).reduce(0.0, Double::sum)); + oneMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getOneMinuteRate).sum()); Map fiveMinute = new HashMap<>(); fiveMinute.put(PROCESSED, processed.getFiveMinuteRate()); fiveMinute.put(PROCESSED_BYTES, processedBytes.getFiveMinuteRate()); fiveMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFiveMinuteRate()); fiveMinute.put(UNPARSEABLE, unparseable.getFiveMinuteRate()); - fiveMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFiveMinuteRate).reduce(0.0, Double::sum)); + fiveMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getFiveMinuteRate).sum()); Map fifteenMinute = new HashMap<>(); fifteenMinute.put(PROCESSED, processed.getFifteenMinuteRate()); fifteenMinute.put(PROCESSED_BYTES, processedBytes.getFifteenMinuteRate()); fifteenMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFifteenMinuteRate()); fifteenMinute.put(UNPARSEABLE, unparseable.getFifteenMinuteRate()); - fifteenMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFifteenMinuteRate).reduce(0.0, Double::sum)); + fifteenMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getFifteenMinuteRate).sum()); movingAverages.put(ONE_MINUTE_NAME, oneMinute); movingAverages.put(FIVE_MINUTE_NAME, fiveMinute); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 0cbd26a3338a..084097dadf72 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -23,11 +23,12 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.AbstractMonitor; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; +import java.util.HashMap; import java.util.Map; /** @@ -65,25 +66,27 @@ public boolean doMonitor(ServiceEmitter emitter) RowIngestionMetersTotals rowIngestionMetersTotals = rowIngestionMeters.getTotals(); // Emit per-reason metrics with the reason dimension - final Map currentThrownAwayByReason = rowIngestionMetersTotals.getThrownAwayByReason(); - final Map previousThrownAwayByReason = previousRowIngestionMetersTotals.getThrownAwayByReason(); - long totalThrownAway = 0; - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - final long currentCount = currentThrownAwayByReason.getOrDefault(reason, 0L); - final long previousCount = previousThrownAwayByReason.getOrDefault(reason, 0L); + final Map currentThrownAwayByReason = rowIngestionMetersTotals.getThrownAwayByReason(); + final Map previousThrownAwayByReason = previousRowIngestionMetersTotals.getThrownAwayByReason(); + final Map deltaThrownAwayByReason = new HashMap<>(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + final long currentCount = currentThrownAwayByReason.getOrDefault(reason.getReason(), 0L); + final long previousCount = previousThrownAwayByReason.getOrDefault(reason.getReason(), 0L); final long delta = currentCount - previousCount; if (delta > 0) { - totalThrownAway += delta; + deltaThrownAwayByReason.put(reason.getReason(), delta); emitter.emit( builder.setDimension(REASON_DIMENSION, reason.getReason()) .setMetric("ingest/events/thrownAway", delta) ); } } + final long totalThrownAway = deltaThrownAwayByReason.values().stream().reduce(0L, Long::sum); if (totalThrownAway > 0) { log.warn( - "[%,d] events thrown away. Possible causes: null events, events filtered out by transformSpec, or events outside earlyMessageRejectionPeriod / lateMessageRejectionPeriod.", - totalThrownAway + "[%,d] events thrown away. Breakdown: [%s]", + totalThrownAway, + deltaThrownAwayByReason ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java index 51e34756f30c..074ae6d9f098 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIterator.java @@ -22,7 +22,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -66,12 +66,12 @@ public boolean hasNext() while (next == null && delegate.hasNext()) { // delegate.next() can throw ParseException final InputRow row = delegate.next(); - // rowFilter.test() can throw ParseException, returns null if accepted, or reason if rejected - final InputRowThrownAwayReason rejectionReason = rowFilter.test(row); - if (rejectionReason == null) { + // rowFilter.test() can throw ParseException + final InputRowFilterResult filterResult = rowFilter.test(row); + if (!filterResult.isRejected()) { next = row; } else { - rowIngestionMeters.incrementThrownAway(rejectionReason); + rowIngestionMeters.incrementThrownAway(filterResult); } } break; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java index b9e95202fdb7..e961fbc5fc7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java @@ -20,14 +20,13 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; -import javax.annotation.Nullable; import java.util.function.Predicate; /** - * A filter for input rows during ingestion that can report the reason for rejection or null for acceptance. - * This is similar to {@link Predicate} but returns the rejection reason instead of just a boolean. + * A filter for input rows during ingestion that returns a {@link InputRowFilterResult} per row. + * This is similar to {@link Predicate} but returns an {@link InputRowFilterResult} instead of a boolean. */ @FunctionalInterface public interface InputRowFilter @@ -35,19 +34,17 @@ public interface InputRowFilter /** * Tests whether the given row should be accepted. * - * @param row the input row to test - * @return null if the row should be accepted, or the {@link InputRowThrownAwayReason} if the row should be rejected + * @return {@link InputRowFilterResult#ACCEPTED} if the row should be accepted, or another {@link InputRowFilterResult} value if the row should be rejected */ - @Nullable - InputRowThrownAwayReason test(InputRow row); + InputRowFilterResult test(InputRow row); /** - * Creates a {@link InputRowFilter} from a Predicate. When the predicate returns false, - * the rejection reason will be {@link InputRowThrownAwayReason#FILTERED}. + * Creates a {@link InputRowFilter} from a {@link Predicate}. + * Callers wishing to return custom rejection reason logic should implement their own {@link InputRowFilter} directly. */ static InputRowFilter fromPredicate(Predicate predicate) { - return row -> predicate.test(row) ? null : InputRowThrownAwayReason.FILTERED; + return row -> predicate.test(row) ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.FILTERED; } /** @@ -55,7 +52,7 @@ static InputRowFilter fromPredicate(Predicate predicate) */ static InputRowFilter allowAll() { - return row -> null; + return row -> InputRowFilterResult.ACCEPTED; } /** @@ -65,9 +62,9 @@ static InputRowFilter allowAll() default InputRowFilter and(InputRowFilter other) { return row -> { - InputRowThrownAwayReason reason = this.test(row); - if (reason != null) { - return reason; + InputRowFilterResult result = this.test(row); + if (result.isRejected()) { + return result; } return other.test(row); }; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 69405874c4a0..74678086d590 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -69,7 +69,6 @@ import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.SegmentSchemaMapping; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -113,7 +112,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -1604,25 +1602,14 @@ private RowIngestionMetersTotals getTotalsFromBuildSegmentsRowStats(Object build return (RowIngestionMetersTotals) buildSegmentsRowStats; } else if (buildSegmentsRowStats instanceof Map) { Map buildSegmentsRowStatsMap = (Map) buildSegmentsRowStats; - - // Convert the thrownAwayByReason map from String keys to InputRowThrownAwayReason enum keys - Map thrownAwayByReason = null; - Object rawThrownAwayByReason = buildSegmentsRowStatsMap.get("thrownAwayByReason"); - if (rawThrownAwayByReason instanceof Map) { - thrownAwayByReason = new EnumMap<>(InputRowThrownAwayReason.class); - Map rawMap = (Map) rawThrownAwayByReason; - for (Map.Entry entry : rawMap.entrySet()) { - InputRowThrownAwayReason reason = InputRowThrownAwayReason.valueOf(entry.getKey().toString()); - thrownAwayByReason.put(reason, ((Number) entry.getValue()).longValue()); - } - } - + Map thrownAwayByReason = (Map) buildSegmentsRowStatsMap.get("thrownAwayByReason"); return new RowIngestionMetersTotals( ((Number) buildSegmentsRowStatsMap.get("processed")).longValue(), ((Number) buildSegmentsRowStatsMap.get("processedBytes")).longValue(), ((Number) buildSegmentsRowStatsMap.get("processedWithError")).longValue(), ((Number) buildSegmentsRowStatsMap.get("thrownAway")).longValue(), - thrownAwayByReason, + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + thrownAwayByReason != null ? CollectionUtils.mapValues(thrownAwayByReason, Integer::longValue) : null, ((Number) buildSegmentsRowStatsMap.get("unparseable")).longValue() ); } else { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index bb77ef1eed07..d8ac6d21676d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -79,7 +79,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.PendingSegmentRecord; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -2147,14 +2147,15 @@ private void refreshMinMaxMessageTime() } /** - * Returns the rejection reason for a row, or null if the row should be accepted. + * Returns the filter result for a row. + * Returns {@link InputRowFilterResult#ACCEPTED} if the row should be accepted, + * or a rejection reason otherwise. * This method is used as a {@link InputRowFilter} for the {@link StreamChunkParser}. */ - @Nullable - InputRowThrownAwayReason ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable InputRow row) + InputRowFilterResult ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable InputRow row) { if (row == null) { - return InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; + return InputRowFilterResult.NULL_OR_EMPTY_RECORD; } else if (minMessageTime.isAfter(row.getTimestamp())) { if (log.isDebugEnabled()) { log.debug( @@ -2163,7 +2164,7 @@ InputRowThrownAwayReason ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable minMessageTime ); } - return InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + return InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME; } else if (maxMessageTime.isBefore(row.getTimestamp())) { if (log.isDebugEnabled()) { log.debug( @@ -2172,8 +2173,8 @@ InputRowThrownAwayReason ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable maxMessageTime ); } - return InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME; + return InputRowFilterResult.AFTER_MAX_MESSAGE_TIME; } - return null; + return InputRowFilterResult.ACCEPTED; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java index 6201719af722..ad9886e424af 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/StreamChunkParser.java @@ -32,7 +32,7 @@ import org.apache.druid.java.util.common.CloseableIterators; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.transform.TransformSpec; @@ -118,7 +118,7 @@ List parse(@Nullable List streamChunk, boolean isEndOfShar if (!isEndOfShard) { // We do not count end of shard record as thrown away event since this is a record created by Druid // Note that this only applies to Kinesis - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); } return Collections.emptyList(); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java index 2e6e652caab2..0bb6d508a924 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java @@ -25,7 +25,7 @@ import org.apache.druid.java.util.metrics.MonitorUtils; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; import org.apache.druid.segment.realtime.SegmentGenerationMetrics; @@ -119,16 +119,16 @@ public void testMessageGapAggStats() public void testThrownAwayEmitsReasonDimension() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - realMeters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, @@ -154,8 +154,8 @@ public void testThrownAwayEmitsReasonDimension() public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, @@ -190,8 +190,8 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() createMetricEventBuilder() ); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); monitor.doMonitor(emitter); long firstCallNullCount = 0; @@ -203,9 +203,9 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() Assert.assertEquals(2, firstCallNullCount); emitter.flush(); - realMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - realMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); monitor.doMonitor(emitter); // Find counts from second call - should be deltas only diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java index 2fb526b996fd..32bc461e501a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java @@ -19,7 +19,7 @@ package org.apache.druid.indexing.common.stats; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.junit.Assert; import org.junit.Test; @@ -36,7 +36,7 @@ public void testBasicIncrements() meters.incrementProcessedBytes(100); meters.incrementProcessedWithError(); meters.incrementUnparseable(); - meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); + meters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); Assert.assertEquals(1, meters.getProcessed()); Assert.assertEquals(100, meters.getProcessedBytes()); @@ -57,23 +57,23 @@ public void testIncrementThrownAwayWithReason() { DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); - meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - meters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - meters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - meters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + meters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + meters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); + meters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); + meters.incrementThrownAway(InputRowFilterResult.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.FILTERED); // Total thrownAway should be sum of all reasons Assert.assertEquals(7, meters.getThrownAway()); // Check per-reason counts - Map byReason = meters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowThrownAwayReason.FILTERED)); + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.FILTERED.getReason())); } @Test @@ -82,10 +82,10 @@ public void testGetThrownAwayByReasonReturnsAllReasons() DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); // Even with no increments, all reasons should be present with 0 counts - Map byReason = meters.getThrownAwayByReason(); - Assert.assertEquals(InputRowThrownAwayReason.values().length, byReason.size()); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); + Map byReason = meters.getThrownAwayByReason(); + Assert.assertEquals(InputRowFilterResult.rejectedValues().length, byReason.size()); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + Assert.assertEquals(Long.valueOf(0), byReason.get(reason.getReason())); } } @@ -95,7 +95,7 @@ public void testMovingAverages() DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); meters.incrementProcessed(); - meters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.FILTERED); Map movingAverages = meters.getMovingAverages(); Assert.assertNotNull(movingAverages); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java index bdef61bc7c6f..8f0d6638ad56 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java @@ -28,7 +28,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.SimpleRowIngestionMeters; @@ -359,11 +359,11 @@ public void testRowFilterWithReasons() final InputRowFilter rowFilter = row -> { int dim1 = (Integer) row.getRaw("dim1"); if (dim1 == 10) { - return null; // accept + return InputRowFilterResult.ACCEPTED; } else if (dim1 == 20) { - return InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + return InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME; } else { - return InputRowThrownAwayReason.FILTERED; + return InputRowFilterResult.FILTERED; } }; @@ -387,11 +387,11 @@ public void testRowFilterWithReasons() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // Check per-reason counts - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); // dim1=20 - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.FILTERED)); // dim1=30 + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); // dim1=20 + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.FILTERED.getReason())); // dim1=30 } @Test @@ -415,21 +415,21 @@ public void testRowFilterFromPredicate() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // All thrown away should have FILTERED reason when using fromPredicate - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.FILTERED)); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.FILTERED.getReason())); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); } @Test public void testRowFilterAnd() { // First filter: reject nulls (simulated by checking dim1) - final InputRowFilter nullFilter = row -> row == null ? InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD : null; + final InputRowFilter nullFilter = row -> row == null ? InputRowFilterResult.NULL_OR_EMPTY_RECORD : InputRowFilterResult.ACCEPTED; // Second filter: reject if dim1 != 10 - final InputRowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? null : InputRowThrownAwayReason.FILTERED; + final InputRowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.FILTERED; // Combine filters final InputRowFilter combinedFilter = nullFilter.and(valueFilter); @@ -448,8 +448,8 @@ public void testRowFilterAnd() Assert.assertEquals(2, rowIngestionMeters.getThrownAway()); // All rejected rows should have FILTERED reason (from second filter) - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.FILTERED)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.FILTERED.getReason())); } private static InputRow newRow(DateTime timestamp, Object dim1Val, Object dim2Val) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 2d9991102882..def62e33e530 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -85,6 +85,7 @@ import org.apache.druid.segment.handoff.NoopSegmentHandoffNotifierFactory; import org.apache.druid.segment.handoff.SegmentHandoffNotifier; import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.loading.LeastBytesUsedStorageLocationSelectorStrategy; @@ -109,6 +110,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.utils.CollectionUtils; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -1508,6 +1510,12 @@ public void testMultipleParseExceptionsSuccess() throws Exception IngestionStatsAndErrors reportData = getTaskReportData(); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedThrownAwayByReason = CollectionUtils.mapValues( + InputRowFilterResult.buildRejectedCounterMap(), + Long::intValue + ); + expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( @@ -1515,7 +1523,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception RowIngestionMeters.PROCESSED, 4, RowIngestionMeters.PROCESSED_BYTES, 657, RowIngestionMeters.UNPARSEABLE, 4, - RowIngestionMeters.THROWN_AWAY, 1 + RowIngestionMeters.THROWN_AWAY, 1, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1523,7 +1532,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception RowIngestionMeters.PROCESSED, 1, RowIngestionMeters.PROCESSED_BYTES, 657, RowIngestionMeters.UNPARSEABLE, 4, - RowIngestionMeters.THROWN_AWAY, 1 + RowIngestionMeters.THROWN_AWAY, 1, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason ) ); Assert.assertEquals(expectedMetrics, reportData.getRowStats()); @@ -1680,6 +1690,16 @@ public void testMultipleParseExceptionsFailure() throws Exception IngestionStatsAndErrors reportData = getTaskReportData(); + // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs + Map expectedDeterminePartitionsThrownAwayByReason = CollectionUtils.mapValues( + InputRowFilterResult.buildRejectedCounterMap(), + Long::intValue + ); + Map expectedBuildSegmentsThrownAwayByReason = CollectionUtils.mapValues( + InputRowFilterResult.buildRejectedCounterMap(), + Long::intValue + ); + expectedBuildSegmentsThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( @@ -1687,7 +1707,8 @@ public void testMultipleParseExceptionsFailure() throws Exception RowIngestionMeters.PROCESSED, 0, RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.UNPARSEABLE, 0, - RowIngestionMeters.THROWN_AWAY, 0 + RowIngestionMeters.THROWN_AWAY, 0, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedDeterminePartitionsThrownAwayByReason ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1695,7 +1716,8 @@ public void testMultipleParseExceptionsFailure() throws Exception RowIngestionMeters.PROCESSED, 1, RowIngestionMeters.PROCESSED_BYTES, 182, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 1 + RowIngestionMeters.THROWN_AWAY, 1, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedBuildSegmentsThrownAwayByReason ) ); @@ -1790,6 +1812,15 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc IngestionStatsAndErrors reportData = getTaskReportData(); + Map expectedDeterminePartitionsThrownAwayByReason = CollectionUtils.mapValues( + InputRowFilterResult.buildRejectedCounterMap(), + Long::intValue + ); + expectedDeterminePartitionsThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); + Map expectedBuildSegmentsThrownAwayByReason = CollectionUtils.mapValues( + InputRowFilterResult.buildRejectedCounterMap(), + Long::intValue + ); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( @@ -1797,7 +1828,8 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc RowIngestionMeters.PROCESSED, 1, RowIngestionMeters.PROCESSED_BYTES, 182, RowIngestionMeters.UNPARSEABLE, 3, - RowIngestionMeters.THROWN_AWAY, 1 + RowIngestionMeters.THROWN_AWAY, 1, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedDeterminePartitionsThrownAwayByReason ), RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1805,7 +1837,8 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc RowIngestionMeters.PROCESSED, 0, RowIngestionMeters.PROCESSED_BYTES, 0, RowIngestionMeters.UNPARSEABLE, 0, - RowIngestionMeters.THROWN_AWAY, 0 + RowIngestionMeters.THROWN_AWAY, 0, + RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedBuildSegmentsThrownAwayByReason ) ); @@ -2738,7 +2771,10 @@ private void verifySchemaAndAggFactory( Map aggregatorFactoryMap ) { - Assert.assertEquals(segmentWithSchemas.getSegments().size(), segmentWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size()); + Assert.assertEquals( + segmentWithSchemas.getSegments().size(), + segmentWithSchemas.getSegmentSchemaMapping().getSegmentIdToMetadataMap().size() + ); Assert.assertEquals(1, segmentWithSchemas.getSegmentSchemaMapping().getSchemaFingerprintToPayloadMap().size()); Assert.assertEquals( actualRowSignature, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java index 3d9810b10b3c..0cb5ef520763 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java @@ -24,7 +24,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.junit.Assert; import org.junit.Test; @@ -40,7 +40,7 @@ public void test_fromPredicate_whichAllowsAll() InputRowFilter filter = InputRowFilter.fromPredicate(row -> true); InputRow row = newRow(100); - Assert.assertNull(filter.test(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, filter.test(row)); } @Test @@ -49,7 +49,7 @@ public void testFromPredicateReject() InputRowFilter filter = InputRowFilter.fromPredicate(row -> false); InputRow row = newRow(100); - Assert.assertEquals(InputRowThrownAwayReason.FILTERED, filter.test(row)); + Assert.assertEquals(InputRowFilterResult.FILTERED, filter.test(row)); } @Test @@ -60,41 +60,41 @@ public void testAndBothAccept() InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); - Assert.assertNull(combined.test(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, combined.test(row)); } @Test public void testAndFirstRejects() { - InputRowFilter filter1 = row -> InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; + InputRowFilter filter1 = row -> InputRowFilterResult.NULL_OR_EMPTY_RECORD; InputRowFilter filter2 = InputRowFilter.allowAll(); InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); - Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, combined.test(row)); + Assert.assertEquals(InputRowFilterResult.NULL_OR_EMPTY_RECORD, combined.test(row)); } @Test public void testAndSecondRejects() { InputRowFilter filter1 = InputRowFilter.allowAll(); - InputRowFilter filter2 = row -> InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME; + InputRowFilter filter2 = row -> InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME; InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); - Assert.assertEquals(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, combined.test(row)); + Assert.assertEquals(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, combined.test(row)); } @Test public void testAndBothRejectReturnsFirst() { - InputRowFilter filter1 = row -> InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD; - InputRowFilter filter2 = row -> InputRowThrownAwayReason.FILTERED; + InputRowFilter filter1 = row -> InputRowFilterResult.NULL_OR_EMPTY_RECORD; + InputRowFilter filter2 = row -> InputRowFilterResult.FILTERED; InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); // Should return reason from first filter - Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, combined.test(row)); + Assert.assertEquals(InputRowFilterResult.NULL_OR_EMPTY_RECORD, combined.test(row)); } @Test @@ -102,12 +102,12 @@ public void testChainedAnd() { InputRowFilter filter1 = InputRowFilter.allowAll(); InputRowFilter filter2 = InputRowFilter.allowAll(); - InputRowFilter filter3 = row -> InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME; + InputRowFilter filter3 = row -> InputRowFilterResult.AFTER_MAX_MESSAGE_TIME; InputRowFilter combined = filter1.and(filter2).and(filter3); InputRow row = newRow(100); - Assert.assertEquals(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, combined.test(row)); + Assert.assertEquals(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, combined.test(row)); } private static InputRow newRow(Object dim1Val) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java index cdc2e4803cdf..9e910a668157 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingRowStatsTest.java @@ -172,7 +172,7 @@ public void testRangePartitionRowStats() TaskReport.ReportMap expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of(), - new RowIngestionMetersTotals(200, 5630, 0, 0, null, 0) + new RowIngestionMetersTotals(200, 5630, 0, 0, 0) ); TaskReport.ReportMap actualReports = runTaskAndGetReports(task, TaskState.SUCCESS); compareTaskReports(expectedReports, actualReports); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 9bd77531088f..baf1faef7e36 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -48,7 +48,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.SegmentUtils; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.indexing.DataSchema; @@ -493,6 +493,8 @@ public void testRunInParallelTaskReports() Collections.emptyList() ); TaskReport.ReportMap actualReports = task.doGetLiveReports(true); + Map expectedThrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); + expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1L); TaskReport.ReportMap expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of( @@ -509,7 +511,7 @@ public void testRunInParallelTaskReports() 1L ) ), - new RowIngestionMetersTotals(10, 335, 1, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 1L, InputRowThrownAwayReason.UNKNOWN, 0L), 1) + new RowIngestionMetersTotals(10, 335, 1, expectedThrownAwayByReason, 1) ); compareTaskReports(expectedReports, actualReports); } @@ -544,7 +546,9 @@ public void testRunInSequential() final ParallelIndexSupervisorTask executedTask = (ParallelIndexSupervisorTask) taskContainer.getTask(); TaskReport.ReportMap actualReports = executedTask.doGetLiveReports(true); - final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, 335, 1, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 1L, InputRowThrownAwayReason.UNKNOWN, 0L), 1); + Map expectedThrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); + expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1L); + final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, 335, 1, expectedThrownAwayByReason, 1); List expectedUnparseableEvents = ImmutableList.of( new ParseExceptionReport( "{ts=2017unparseable}", @@ -565,7 +569,7 @@ public void testRunInSequential() expectedReports = buildExpectedTaskReportSequential( task.getId(), expectedUnparseableEvents, - new RowIngestionMetersTotals(0, 0, 0, Map.of(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 0L, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, 0L, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, 0L, InputRowThrownAwayReason.FILTERED, 0L, InputRowThrownAwayReason.UNKNOWN, 0L), 0), + new RowIngestionMetersTotals(0, 0, 0, 0, 0), expectedTotals ); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java index 4fb8fb3289bf..eb08944d46bd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerTest.java @@ -36,7 +36,7 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.segment.incremental.InputRowThrownAwayReason; +import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.indexing.DataSchema; import org.joda.time.DateTime; import org.junit.Assert; @@ -106,13 +106,13 @@ public void testWithinMinMaxTime() LockGranularity.TIME_CHUNK); Mockito.when(row.getTimestamp()).thenReturn(now); - Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertEquals(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertEquals(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); } @Test @@ -159,13 +159,13 @@ public void testWithinMinMaxTimeNotPopulated() LockGranularity.TIME_CHUNK); Mockito.when(row.getTimestamp()).thenReturn(now); - Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.minusHours(2).minusMinutes(1)); - Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); Mockito.when(row.getTimestamp()).thenReturn(now.plusHours(2).plusMinutes(1)); - Assert.assertNull(runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); + Assert.assertEquals(InputRowFilterResult.ACCEPTED, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(row)); } @Test @@ -209,7 +209,7 @@ public void testEnsureRowRejectionReasonForNullRow() TestasbleSeekableStreamIndexTaskRunner runner = new TestasbleSeekableStreamIndexTaskRunner(task, null, LockGranularity.TIME_CHUNK); - Assert.assertEquals(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(null)); + Assert.assertEquals(InputRowFilterResult.NULL_OR_EMPTY_RECORD, runner.ensureRowIsNonNullAndWithinMessageTimeBounds(null)); } @Test diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java similarity index 53% rename from processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java rename to processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java index 8f348c1f2f88..5ada38fafba0 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/InputRowThrownAwayReason.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java @@ -19,14 +19,19 @@ package org.apache.druid.segment.incremental; -import java.util.EnumMap; +import java.util.Arrays; +import java.util.HashMap; import java.util.Map; /** - * Reasons why an input row may be thrown away during ingestion. + * Result of filtering an input row during ingestion. */ -public enum InputRowThrownAwayReason +public enum InputRowFilterResult { + /** + * The row passed the filter and should be processed. + */ + ACCEPTED("accepted"), /** * The row was null or the input record was empty. */ @@ -52,15 +57,20 @@ public enum InputRowThrownAwayReason */ UNKNOWN("unknown"); + private static final InputRowFilterResult[] REJECTED_VALUES = Arrays.stream(InputRowFilterResult.values()) + .filter(InputRowFilterResult::isRejected) + .toArray(InputRowFilterResult[]::new); + public static final int NUM_FILTER_RESULT = InputRowFilterResult.values().length; + private final String reason; - InputRowThrownAwayReason(String reason) + InputRowFilterResult(String reason) { this.reason = reason; } /** - * Returns string value representation of this {@link InputRowThrownAwayReason} for metric emission. + * Returns string value representation of this {@link InputRowFilterResult} for metric emission. */ public String getReason() { @@ -68,14 +78,33 @@ public String getReason() } /** - * Public utility for building a mutable frequency map over the possible {@link InputRowThrownAwayReason} values. + * Returns true if this result indicates the row was rejected (thrown away). + * Returns false for {@link #ACCEPTED}. */ - public static Map buildBaseCounterMap() + public boolean isRejected() { - final EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - result.put(reason, 0L); + return this != ACCEPTED; + } + + /** + * Public utility for building a mutable frequency map over the possible rejection {@link InputRowFilterResult} values. + * Keys on {@link InputRowFilterResult#getReason()} rather than the enum name as the latter is more likely to change longer-term. + * It is also easier to have stats payload keys match what is being emitted in metrics. + */ + public static Map buildRejectedCounterMap() + { + final Map result = new HashMap<>(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + result.put(reason.getReason(), 0L); } return result; } + + /** + * Returns {@link InputRowFilterResult} that are not rejection states. + */ + public static InputRowFilterResult[] rejectedValues() + { + return REJECTED_VALUES; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java index 0ba12f859515..67eb80a2dc8b 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/NoopRowIngestionMeters.java @@ -74,13 +74,13 @@ public long getThrownAway() } @Override - public void incrementThrownAway(InputRowThrownAwayReason reason) + public void incrementThrownAway(InputRowFilterResult reason) { } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { return Map.of(); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java index 1280a7027a99..502c2057ebc0 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMeters.java @@ -40,6 +40,7 @@ public interface RowIngestionMeters extends InputStats String PROCESSED_WITH_ERROR = "processedWithError"; String UNPARSEABLE = "unparseable"; String THROWN_AWAY = "thrownAway"; + String THROWN_AWAY_BY_REASON = "thrownAwayByReason"; /** * Number of bytes read by an ingestion task. @@ -75,14 +76,15 @@ default long getProcessedBytes() long getThrownAway(); /** - * Increments the thrown away counter for the specified reason. + * Increments the thrown away counter for the specified {@link InputRowFilterResult} reason. */ - void incrementThrownAway(InputRowThrownAwayReason reason); + void incrementThrownAway(InputRowFilterResult reason); /** * Returns the count of thrown away events for each reason. + * Keyed by {@link InputRowFilterResult#getReason()}. */ - Map getThrownAwayByReason(); + Map getThrownAwayByReason(); RowIngestionMetersTotals getTotals(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java index 67f7be7b1d2e..8acb21b19893 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java @@ -33,7 +33,7 @@ public class RowIngestionMetersTotals private final long processedBytes; private final long processedWithError; private final long thrownAway; - private final Map thrownAwayByReason; + private final Map thrownAwayByReason; private final long unparseable; @JsonCreator @@ -42,7 +42,7 @@ public RowIngestionMetersTotals( @JsonProperty("processedBytes") long processedBytes, @JsonProperty("processedWithError") long processedWithError, @JsonProperty("thrownAway") long thrownAway, - @JsonProperty("thrownAwayByReason") @Nullable Map thrownAwayByReason, + @JsonProperty("thrownAwayByReason") @Nullable Map thrownAwayByReason, @JsonProperty("unparseable") long unparseable ) { @@ -59,7 +59,24 @@ public RowIngestionMetersTotals( @JsonProperty("processed") long processed, @JsonProperty("processedBytes") long processedBytes, @JsonProperty("processedWithError") long processedWithError, - @JsonProperty("thrownAwayByReason") Map thrownAwayByReason, + @JsonProperty("thrownAway") long thrownAway, + @JsonProperty("unparseable") long unparseable + ) + { + this( + processed, + processedBytes, + processedWithError, + getBackwardsCompatibleThrownAwayByReason(thrownAway), + unparseable + ); + } + + public RowIngestionMetersTotals( + @JsonProperty("processed") long processed, + @JsonProperty("processedBytes") long processedBytes, + @JsonProperty("processedWithError") long processedWithError, + @JsonProperty("thrownAwayByReason") Map thrownAwayByReason, @JsonProperty("unparseable") long unparseable ) { @@ -96,7 +113,7 @@ public long getThrownAway() } @JsonProperty - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { return thrownAwayByReason; } @@ -145,13 +162,13 @@ public String toString() } /** - * For backwards compatibility, key by {@link InputRowThrownAwayReason} in case of lack of thrownAwayByReason input during rolling Druid upgrades. + * For backwards compatibility, key by {@link InputRowFilterResult} in case of lack of thrownAwayByReason input during rolling Druid upgrades. * This can occur when tasks running on older Druid versions return ingest statistic payloads to an overlord running on a newer Druid version. */ - private static Map getBackwardsCompatibleThrownAwayByReason(long thrownAway) + private static Map getBackwardsCompatibleThrownAwayByReason(long thrownAway) { - Map results = InputRowThrownAwayReason.buildBaseCounterMap(); - results.put(InputRowThrownAwayReason.UNKNOWN, thrownAway); + Map results = InputRowFilterResult.buildRejectedCounterMap(); + results.put(InputRowFilterResult.UNKNOWN.getReason(), thrownAway); return results; } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java index 0c61eee72bf1..cba7c267b076 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java @@ -19,18 +19,15 @@ package org.apache.druid.segment.incremental; -import java.util.EnumMap; import java.util.Map; public class SimpleRowIngestionMeters implements RowIngestionMeters { - private static final int NUM_THROWN_AWAY_REASONS = InputRowThrownAwayReason.values().length; - private long processed; private long processedWithError; private long unparseable; private long processedBytes; - private final long[] thrownAwayByReason = new long[NUM_THROWN_AWAY_REASONS]; + private final long[] thrownAwayByReason = new long[InputRowFilterResult.NUM_FILTER_RESULT]; @Override public long getProcessed() @@ -83,21 +80,25 @@ public void incrementUnparseable() @Override public long getThrownAway() { - return getThrownAwayByReason().values().stream().reduce(0L, Long::sum); + long total = 0; + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + total += thrownAwayByReason[reason.ordinal()]; + } + return total; } @Override - public void incrementThrownAway(InputRowThrownAwayReason reason) + public void incrementThrownAway(InputRowFilterResult reason) { - thrownAwayByReason[reason.ordinal()]++; + ++thrownAwayByReason[reason.ordinal()]; } @Override - public Map getThrownAwayByReason() + public Map getThrownAwayByReason() { - final EnumMap result = new EnumMap<>(InputRowThrownAwayReason.class); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - result.put(reason, thrownAwayByReason[reason.ordinal()]); + final Map result = InputRowFilterResult.buildRejectedCounterMap(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + result.put(reason.getReason(), thrownAwayByReason[reason.ordinal()]); } return result; } @@ -127,8 +128,9 @@ public void addRowIngestionMetersTotals(RowIngestionMetersTotals rowIngestionMet this.unparseable += rowIngestionMetersTotals.getUnparseable(); this.processedBytes += rowIngestionMetersTotals.getProcessedBytes(); - for (Map.Entry entry : rowIngestionMetersTotals.getThrownAwayByReason().entrySet()) { - this.thrownAwayByReason[entry.getKey().ordinal()] += entry.getValue(); + final Map thrownAwayByReason = rowIngestionMetersTotals.getThrownAwayByReason(); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + this.thrownAwayByReason[reason.ordinal()] += thrownAwayByReason.getOrDefault(reason.getReason(), 0L); } } } diff --git a/processing/src/test/java/org/apache/druid/indexer/report/TaskReportSerdeTest.java b/processing/src/test/java/org/apache/druid/indexer/report/TaskReportSerdeTest.java index 4ff36d731dfa..b137ffe1f357 100644 --- a/processing/src/test/java/org/apache/druid/indexer/report/TaskReportSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/indexer/report/TaskReportSerdeTest.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; import org.apache.druid.segment.incremental.RowMeters; +import org.apache.druid.utils.CollectionUtils; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -353,6 +354,7 @@ private void verifyTotalRowStats( "processedBytes", (int) determinePartitionTotalStats.getProcessedBytes(), "processedWithError", (int) determinePartitionTotalStats.getProcessedWithError(), "thrownAway", (int) determinePartitionTotalStats.getThrownAway(), + "thrownAwayByReason", CollectionUtils.mapValues(determinePartitionTotalStats.getThrownAwayByReason(), Long::intValue), "unparseable", (int) determinePartitionTotalStats.getUnparseable() ), observedTotals.get("determinePartitions") @@ -363,6 +365,7 @@ private void verifyTotalRowStats( "processedBytes", (int) buildSegmentTotalStats.getProcessedBytes(), "processedWithError", (int) buildSegmentTotalStats.getProcessedWithError(), "thrownAway", (int) buildSegmentTotalStats.getThrownAway(), + "thrownAwayByReason", CollectionUtils.mapValues(buildSegmentTotalStats.getThrownAwayByReason(), Long::intValue), "unparseable", (int) buildSegmentTotalStats.getUnparseable() ), observedTotals.get("buildSegments") diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java new file mode 100644 index 000000000000..75a5530ddf0c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java @@ -0,0 +1,76 @@ +/* + * 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.druid.segment.incremental; + +import org.junit.Assert; +import org.junit.Test; + +public class InputRowFilterResultTest +{ + @Test + public void testOrdinalValues() + { + Assert.assertEquals(0, InputRowFilterResult.ACCEPTED.ordinal()); + Assert.assertEquals(1, InputRowFilterResult.NULL_OR_EMPTY_RECORD.ordinal()); + Assert.assertEquals(2, InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.ordinal()); + Assert.assertEquals(3, InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.ordinal()); + Assert.assertEquals(4, InputRowFilterResult.FILTERED.ordinal()); + } + + @Test + public void testMetricValues() + { + Assert.assertEquals("accepted", InputRowFilterResult.ACCEPTED.getReason()); + Assert.assertEquals("null", InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason()); + Assert.assertEquals("beforeMinMessageTime", InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason()); + Assert.assertEquals("afterMaxMessageTime", InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason()); + Assert.assertEquals("filtered", InputRowFilterResult.FILTERED.getReason()); + } + + @Test + public void testEnumCardinality() + { + Assert.assertEquals(6, InputRowFilterResult.values().length); + } + + @Test + public void testIsRejected() + { + Assert.assertFalse(InputRowFilterResult.ACCEPTED.isRejected()); + Assert.assertTrue(InputRowFilterResult.NULL_OR_EMPTY_RECORD.isRejected()); + Assert.assertTrue(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.isRejected()); + Assert.assertTrue(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.isRejected()); + Assert.assertTrue(InputRowFilterResult.FILTERED.isRejected()); + Assert.assertTrue(InputRowFilterResult.UNKNOWN.isRejected()); + } + + @Test + public void testBuildRejectedCounterMapExcludesAccepted() + { + var counterMap = InputRowFilterResult.buildRejectedCounterMap(); + Assert.assertFalse(counterMap.containsKey(InputRowFilterResult.ACCEPTED.getReason())); + Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); + Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); + Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.FILTERED.getReason())); + Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.UNKNOWN.getReason())); + } +} + diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java deleted file mode 100644 index aec067608bae..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/incremental/InputRowThrownAwayReasonTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.druid.segment.incremental; - -import org.junit.Assert; -import org.junit.Test; - -public class InputRowThrownAwayReasonTest -{ - @Test - public void testOrdinalValues() - { - Assert.assertEquals(0, InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD.ordinal()); - Assert.assertEquals(1, InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.ordinal()); - Assert.assertEquals(2, InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME.ordinal()); - Assert.assertEquals(3, InputRowThrownAwayReason.FILTERED.ordinal()); - } - - @Test - public void testMetricValues() - { - Assert.assertEquals("null", InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD.getReason()); - Assert.assertEquals("beforeMinMessageTime", InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME.getReason()); - Assert.assertEquals("afterMaxMessageTime", InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME.getReason()); - Assert.assertEquals("filtered", InputRowThrownAwayReason.FILTERED.getReason()); - } - - @Test - public void testEnumCardinality() - { - Assert.assertEquals(5, InputRowThrownAwayReason.values().length); - } -} - diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java index 6d6a06df53cd..2154d722439e 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java @@ -29,7 +29,7 @@ public class RowMeters private long processedBytes; private long processedWithError; private long unparseable; - private final Map thrownAwayByReason = InputRowThrownAwayReason.buildBaseCounterMap(); + private final Map thrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); /** * Creates a new {@link RowMeters}, that can be used to build an instance of @@ -58,15 +58,15 @@ public RowMeters unparseable(long unparseable) return this; } - public RowMeters thrownAwayByReason(InputRowThrownAwayReason thrownAwayByReason, long thrownAway) + public RowMeters thrownAwayByReason(InputRowFilterResult thrownAwayByReason, long thrownAway) { - this.thrownAwayByReason.put(thrownAwayByReason, this.thrownAwayByReason.get(thrownAwayByReason) + thrownAway); + this.thrownAwayByReason.put(thrownAwayByReason.getReason(), thrownAway); return this; } public RowMeters thrownAway(long thrownAway) { - this.thrownAwayByReason.put(InputRowThrownAwayReason.UNKNOWN, this.thrownAwayByReason.get(InputRowThrownAwayReason.UNKNOWN) + thrownAway); + this.thrownAwayByReason.put(InputRowFilterResult.UNKNOWN.getReason(), thrownAway); return this; } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java index a574eaddb67e..2fce81a6f8e3 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java @@ -34,18 +34,19 @@ public void testIncrement() rowIngestionMeters.incrementProcessedBytes(5); rowIngestionMeters.incrementProcessedWithError(); rowIngestionMeters.incrementUnparseable(); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - Assert.assertEquals(rowIngestionMeters.getTotals(), new RowIngestionMetersTotals(1, 5, 1, Map.of( - InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD, 1L), 1)); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + final Map expected = InputRowFilterResult.buildRejectedCounterMap(); + expected.put(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1L); + Assert.assertEquals(new RowIngestionMetersTotals(1, 5, 1, expected, 1), rowIngestionMeters.getTotals()); } @Test public void testAddRowIngestionMetersTotals() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, Map.of(), 1); + RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, 0, 1); rowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); - Assert.assertEquals(rowIngestionMeters.getTotals(), rowIngestionMetersTotals); + Assert.assertEquals(rowIngestionMetersTotals, rowIngestionMeters.getTotals()); } @Test @@ -53,21 +54,21 @@ public void testIncrementThrownAwayWithReason() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); - rowIngestionMeters.incrementThrownAway(InputRowThrownAwayReason.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); Assert.assertEquals(7, rowIngestionMeters.getThrownAway()); - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowThrownAwayReason.NULL_OR_EMPTY_RECORD)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.BEFORE_MIN_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowThrownAwayReason.AFTER_MAX_MESSAGE_TIME)); - Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowThrownAwayReason.FILTERED)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.FILTERED.getReason())); } @Test @@ -76,10 +77,10 @@ public void testGetThrownAwayByReasonReturnsAllReasons() SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); // Even with no increments, all reasons should be present with 0 counts - Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(InputRowThrownAwayReason.values().length, byReason.size()); - for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) { - Assert.assertEquals(Long.valueOf(0), byReason.get(reason)); + Map byReason = rowIngestionMeters.getThrownAwayByReason(); + Assert.assertEquals(InputRowFilterResult.rejectedValues().length, byReason.size()); + for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { + Assert.assertEquals(Long.valueOf(0), byReason.get(reason.getReason())); } } } From 62440f303cf3e6baffdba9b6534236d461927e0f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jesse=20Tu=C4=9Flu?= Date: Mon, 22 Dec 2025 20:27:28 -0500 Subject: [PATCH 5/5] Comments --- .../indexing/kafka/KafkaIndexTaskTest.java | 6 ++-- .../kinesis/KinesisIndexTaskTest.java | 5 ++-- .../stats/DropwizardRowIngestionMeters.java | 9 ++++-- .../stats/TaskRealtimeMetricsMonitor.java | 4 +-- .../indexing/common/task/InputRowFilter.java | 4 +-- .../SeekableStreamIndexTaskRunner.java | 3 -- .../TaskRealtimeMetricsMonitorTest.java | 22 +++++++------- .../DropwizardRowIngestionMetersTest.java | 17 +++++------ ...ilteringCloseableInputRowIteratorTest.java | 17 +++++------ .../indexing/common/task/IndexTaskTest.java | 29 ++++--------------- .../common/task/InputRowFilterTest.java | 4 +-- .../SinglePhaseParallelIndexingTest.java | 6 ++-- .../org/apache/druid/query/DruidMetrics.java | 1 + .../incremental/InputRowFilterResult.java | 27 ++++++----------- .../incremental/RowIngestionMetersTotals.java | 27 +++++++++-------- .../incremental/SimpleRowIngestionMeters.java | 10 +++++-- .../incremental/InputRowFilterResultTest.java | 22 ++++---------- .../druid/segment/incremental/RowMeters.java | 3 +- .../SimpleRowIngestionMetersTest.java | 22 ++++++-------- 19 files changed, 96 insertions(+), 142 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 54ef4b6ece83..cb359896ee2f 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -129,7 +129,6 @@ import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CollectionUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; @@ -1663,8 +1662,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertNull(reportData.getErrorMsg()); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); - expectedThrownAwayByReason.put(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1); + Map expectedThrownAwayByReason = Map.of(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1752,7 +1750,7 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertNotNull(reportData.getErrorMsg()); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); + Map expectedThrownAwayByReason = Map.of(); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index c714953811bb..bd2a8aa27121 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -89,7 +89,6 @@ import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CollectionUtils; import org.easymock.EasyMock; import org.joda.time.Duration; import org.joda.time.Period; @@ -1197,7 +1196,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertNull(reportData.getErrorMsg()); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); + Map expectedThrownAwayByReason = Map.of(); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( @@ -1290,7 +1289,7 @@ public void testMultipleParseExceptionsFailure() throws Exception Assert.assertNotNull(reportData.getErrorMsg()); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue); + Map expectedThrownAwayByReason = Map.of(); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.BUILD_SEGMENTS, ImmutableMap.of( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java index 0c824ff7ad47..49a19b237995 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java @@ -39,7 +39,7 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters private final Meter processedBytes; private final Meter processedWithError; private final Meter unparseable; - private final Meter[] thrownAwayByReason = new Meter[InputRowFilterResult.NUM_FILTER_RESULT]; + private final Meter[] thrownAwayByReason = new Meter[InputRowFilterResult.numValues()]; public DropwizardRowIngestionMeters() { @@ -120,9 +120,12 @@ public void incrementThrownAway(InputRowFilterResult reason) @Override public Map getThrownAwayByReason() { - Map result = InputRowFilterResult.buildRejectedCounterMap(); + Map result = new HashMap<>(); for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { - result.put(reason.getReason(), thrownAwayByReason[reason.ordinal()].getCount()); + long count = thrownAwayByReason[reason.ordinal()].getCount(); + if (count > 0) { + result.put(reason.getReason(), count); + } } return result; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java index 084097dadf72..462fc2348123 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/stats/TaskRealtimeMetricsMonitor.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.metrics.AbstractMonitor; +import org.apache.druid.query.DruidMetrics; import org.apache.druid.segment.incremental.InputRowFilterResult; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.incremental.RowIngestionMetersTotals; @@ -37,7 +38,6 @@ public class TaskRealtimeMetricsMonitor extends AbstractMonitor { private static final EmittingLogger log = new EmittingLogger(TaskRealtimeMetricsMonitor.class); - private static final String REASON_DIMENSION = "reason"; private final SegmentGenerationMetrics segmentGenerationMetrics; private final RowIngestionMeters rowIngestionMeters; @@ -76,7 +76,7 @@ public boolean doMonitor(ServiceEmitter emitter) if (delta > 0) { deltaThrownAwayByReason.put(reason.getReason(), delta); emitter.emit( - builder.setDimension(REASON_DIMENSION, reason.getReason()) + builder.setDimension(DruidMetrics.REASON, reason.getReason()) .setMetric("ingest/events/thrownAway", delta) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java index e961fbc5fc7b..8f65b9bcbe96 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputRowFilter.java @@ -34,7 +34,7 @@ public interface InputRowFilter /** * Tests whether the given row should be accepted. * - * @return {@link InputRowFilterResult#ACCEPTED} if the row should be accepted, or another {@link InputRowFilterResult} value if the row should be rejected + * @return {@link InputRowFilterResult#ACCEPTED} only if the row should be accepted, otherwise another {@link InputRowFilterResult} value. */ InputRowFilterResult test(InputRow row); @@ -44,7 +44,7 @@ public interface InputRowFilter */ static InputRowFilter fromPredicate(Predicate predicate) { - return row -> predicate.test(row) ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.FILTERED; + return row -> predicate.test(row) ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.CUSTOM_FILTER; } /** diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index d8ac6d21676d..a448e01f9189 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -66,7 +66,6 @@ import org.apache.druid.indexing.common.actions.TaskLocks; import org.apache.druid.indexing.common.actions.TimeChunkLockAcquireAction; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; -import org.apache.druid.indexing.common.task.InputRowFilter; import org.apache.druid.indexing.input.InputRowSchemas; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; @@ -2147,10 +2146,8 @@ private void refreshMinMaxMessageTime() } /** - * Returns the filter result for a row. * Returns {@link InputRowFilterResult#ACCEPTED} if the row should be accepted, * or a rejection reason otherwise. - * This method is used as a {@link InputRowFilter} for the {@link StreamChunkParser}. */ InputRowFilterResult ensureRowIsNonNullAndWithinMessageTimeBounds(@Nullable InputRow row) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java index 0bb6d508a924..bcfebef2a6ce 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskRealtimeMetricsMonitorTest.java @@ -125,10 +125,10 @@ public void testThrownAwayEmitsReasonDimension() realMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); realMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); realMeters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, @@ -145,8 +145,8 @@ public void testThrownAwayEmitsReasonDimension() } Assert.assertEquals(Long.valueOf(2), thrownAwayByReason.get("null")); - Assert.assertEquals(Long.valueOf(3), thrownAwayByReason.get("beforeMinMessageTime")); - Assert.assertEquals(Long.valueOf(1), thrownAwayByReason.get("afterMaxMessageTime")); + Assert.assertEquals(Long.valueOf(3), thrownAwayByReason.get("beforeMinimumMessageTime")); + Assert.assertEquals(Long.valueOf(1), thrownAwayByReason.get("afterMaximumMessageTime")); Assert.assertEquals(Long.valueOf(4), thrownAwayByReason.get("filtered")); } @@ -155,7 +155,7 @@ public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() { SimpleRowIngestionMeters realMeters = new SimpleRowIngestionMeters(); realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); TaskRealtimeMetricsMonitor monitor = new TaskRealtimeMetricsMonitor( segmentGenerationMetrics, @@ -175,8 +175,8 @@ public void testThrownAwayReasonDimensionOnlyEmittedWhenNonZero() Assert.assertEquals(2, thrownAwayByReason.size()); Assert.assertTrue(thrownAwayByReason.containsKey("null")); Assert.assertTrue(thrownAwayByReason.containsKey("filtered")); - Assert.assertFalse(thrownAwayByReason.containsKey("beforeMinMessageTime")); - Assert.assertFalse(thrownAwayByReason.containsKey("afterMaxMessageTime")); + Assert.assertFalse(thrownAwayByReason.containsKey("beforeMinimumMessageTime")); + Assert.assertFalse(thrownAwayByReason.containsKey("afterMaximumMessageTime")); } @Test @@ -204,8 +204,8 @@ public void testThrownAwayReasonDeltaAcrossMonitorCalls() emitter.flush(); realMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - realMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + realMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); monitor.doMonitor(emitter); // Find counts from second call - should be deltas only diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java index 32bc461e501a..d92276700000 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMetersTest.java @@ -61,9 +61,9 @@ public void testIncrementThrownAwayWithReason() meters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); meters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); meters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); - meters.incrementThrownAway(InputRowFilterResult.FILTERED); - meters.incrementThrownAway(InputRowFilterResult.FILTERED); - meters.incrementThrownAway(InputRowFilterResult.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + meters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + meters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); // Total thrownAway should be sum of all reasons Assert.assertEquals(7, meters.getThrownAway()); @@ -73,7 +73,7 @@ public void testIncrementThrownAwayWithReason() Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); - Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.FILTERED.getReason())); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.CUSTOM_FILTER.getReason())); } @Test @@ -81,12 +81,9 @@ public void testGetThrownAwayByReasonReturnsAllReasons() { DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); - // Even with no increments, all reasons should be present with 0 counts + // With no increments, all reasons should be present with 0 counts Map byReason = meters.getThrownAwayByReason(); - Assert.assertEquals(InputRowFilterResult.rejectedValues().length, byReason.size()); - for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { - Assert.assertEquals(Long.valueOf(0), byReason.get(reason.getReason())); - } + Assert.assertTrue(byReason.isEmpty()); } @Test @@ -95,7 +92,7 @@ public void testMovingAverages() DropwizardRowIngestionMeters meters = new DropwizardRowIngestionMeters(); meters.incrementProcessed(); - meters.incrementThrownAway(InputRowFilterResult.FILTERED); + meters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); Map movingAverages = meters.getMovingAverages(); Assert.assertNotNull(movingAverages); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java index 8f0d6638ad56..e037d9adf720 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/FilteringCloseableInputRowIteratorTest.java @@ -363,7 +363,7 @@ public void testRowFilterWithReasons() } else if (dim1 == 20) { return InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME; } else { - return InputRowFilterResult.FILTERED; + return InputRowFilterResult.CUSTOM_FILTER; } }; @@ -388,10 +388,9 @@ public void testRowFilterWithReasons() // Check per-reason counts Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); + Assert.assertEquals(2, byReason.size()); Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); // dim1=20 - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); - Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.FILTERED.getReason())); // dim1=30 + Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.CUSTOM_FILTER.getReason())); // dim1=30 } @Test @@ -416,10 +415,8 @@ public void testRowFilterFromPredicate() // All thrown away should have FILTERED reason when using fromPredicate Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.FILTERED.getReason())); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); - Assert.assertEquals(Long.valueOf(0), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); + Assert.assertEquals(1, byReason.size()); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.CUSTOM_FILTER.getReason())); } @Test @@ -429,7 +426,7 @@ public void testRowFilterAnd() final InputRowFilter nullFilter = row -> row == null ? InputRowFilterResult.NULL_OR_EMPTY_RECORD : InputRowFilterResult.ACCEPTED; // Second filter: reject if dim1 != 10 - final InputRowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.FILTERED; + final InputRowFilter valueFilter = row -> (Integer) row.getRaw("dim1") == 10 ? InputRowFilterResult.ACCEPTED : InputRowFilterResult.CUSTOM_FILTER; // Combine filters final InputRowFilter combinedFilter = nullFilter.and(valueFilter); @@ -449,7 +446,7 @@ public void testRowFilterAnd() // All rejected rows should have FILTERED reason (from second filter) Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.FILTERED.getReason())); + Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.CUSTOM_FILTER.getReason())); } private static InputRow newRow(DateTime timestamp, Object dim1Val, Object dim2Val) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index def62e33e530..9a2eeb8c1a66 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -110,7 +110,6 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpec; -import org.apache.druid.utils.CollectionUtils; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; import org.hamcrest.MatcherAssert; @@ -1511,11 +1510,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception IngestionStatsAndErrors reportData = getTaskReportData(); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedThrownAwayByReason = CollectionUtils.mapValues( - InputRowFilterResult.buildRejectedCounterMap(), - Long::intValue - ); - expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); + Map expectedThrownAwayByReason = Map.of(InputRowFilterResult.CUSTOM_FILTER.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( @@ -1691,15 +1686,8 @@ public void testMultipleParseExceptionsFailure() throws Exception IngestionStatsAndErrors reportData = getTaskReportData(); // Jackson will serde numerics ≤ 32bits as Integers, rather than Longs - Map expectedDeterminePartitionsThrownAwayByReason = CollectionUtils.mapValues( - InputRowFilterResult.buildRejectedCounterMap(), - Long::intValue - ); - Map expectedBuildSegmentsThrownAwayByReason = CollectionUtils.mapValues( - InputRowFilterResult.buildRejectedCounterMap(), - Long::intValue - ); - expectedBuildSegmentsThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); + Map expectedDeterminePartitionsThrownAwayByReason = Map.of(); + Map expectedBuildSegmentsThrownAwayByReason = Map.of(InputRowFilterResult.CUSTOM_FILTER.getReason(), 1); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( @@ -1812,15 +1800,8 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc IngestionStatsAndErrors reportData = getTaskReportData(); - Map expectedDeterminePartitionsThrownAwayByReason = CollectionUtils.mapValues( - InputRowFilterResult.buildRejectedCounterMap(), - Long::intValue - ); - expectedDeterminePartitionsThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1); - Map expectedBuildSegmentsThrownAwayByReason = CollectionUtils.mapValues( - InputRowFilterResult.buildRejectedCounterMap(), - Long::intValue - ); + Map expectedDeterminePartitionsThrownAwayByReason = Map.of(InputRowFilterResult.CUSTOM_FILTER.getReason(), 1); + Map expectedBuildSegmentsThrownAwayByReason = Map.of(); Map expectedMetrics = ImmutableMap.of( RowIngestionMeters.DETERMINE_PARTITIONS, ImmutableMap.of( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java index 0cb5ef520763..53616c974e7d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/InputRowFilterTest.java @@ -49,7 +49,7 @@ public void testFromPredicateReject() InputRowFilter filter = InputRowFilter.fromPredicate(row -> false); InputRow row = newRow(100); - Assert.assertEquals(InputRowFilterResult.FILTERED, filter.test(row)); + Assert.assertEquals(InputRowFilterResult.CUSTOM_FILTER, filter.test(row)); } @Test @@ -89,7 +89,7 @@ public void testAndSecondRejects() public void testAndBothRejectReturnsFirst() { InputRowFilter filter1 = row -> InputRowFilterResult.NULL_OR_EMPTY_RECORD; - InputRowFilter filter2 = row -> InputRowFilterResult.FILTERED; + InputRowFilter filter2 = row -> InputRowFilterResult.CUSTOM_FILTER; InputRowFilter combined = filter1.and(filter2); InputRow row = newRow(100); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index baf1faef7e36..a0e3f3245f2c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -493,8 +493,7 @@ public void testRunInParallelTaskReports() Collections.emptyList() ); TaskReport.ReportMap actualReports = task.doGetLiveReports(true); - Map expectedThrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); - expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1L); + Map expectedThrownAwayByReason = Map.of(InputRowFilterResult.CUSTOM_FILTER.getReason(), 1L); TaskReport.ReportMap expectedReports = buildExpectedTaskReportParallel( task.getId(), ImmutableList.of( @@ -546,8 +545,7 @@ public void testRunInSequential() final ParallelIndexSupervisorTask executedTask = (ParallelIndexSupervisorTask) taskContainer.getTask(); TaskReport.ReportMap actualReports = executedTask.doGetLiveReports(true); - Map expectedThrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); - expectedThrownAwayByReason.put(InputRowFilterResult.FILTERED.getReason(), 1L); + Map expectedThrownAwayByReason = Map.of(InputRowFilterResult.CUSTOM_FILTER.getReason(), 1L); final RowIngestionMetersTotals expectedTotals = new RowIngestionMetersTotals(10, 335, 1, expectedThrownAwayByReason, 1); List expectedUnparseableEvents = ImmutableList.of( new ParseExceptionReport( diff --git a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java index 578d792df493..532553aef6ef 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java @@ -57,6 +57,7 @@ public class DruidMetrics public static final String STREAM = "stream"; public static final String PARTITION = "partition"; public static final String SUPERVISOR_ID = "supervisorId"; + public static final String REASON = "reason"; public static final String TAGS = "tags"; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java index 5ada38fafba0..fcb4d8aec116 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/InputRowFilterResult.java @@ -20,8 +20,6 @@ package org.apache.druid.segment.incremental; import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; /** * Result of filtering an input row during ingestion. @@ -40,17 +38,17 @@ public enum InputRowFilterResult /** * The row's timestamp is before the minimum message time (late message rejection). */ - BEFORE_MIN_MESSAGE_TIME("beforeMinMessageTime"), + BEFORE_MIN_MESSAGE_TIME("beforeMinimumMessageTime"), /** * The row's timestamp is after the maximum message time (early message rejection). */ - AFTER_MAX_MESSAGE_TIME("afterMaxMessageTime"), + AFTER_MAX_MESSAGE_TIME("afterMaximumMessageTime"), /** * The row was filtered out by a transformSpec filter or other row filter. */ - FILTERED("filtered"), + CUSTOM_FILTER("filtered"), /** * A backwards-compatible value for tracking filter reasons for ingestion tasks using older Druid versions without filter reason tracking. @@ -60,7 +58,6 @@ public enum InputRowFilterResult private static final InputRowFilterResult[] REJECTED_VALUES = Arrays.stream(InputRowFilterResult.values()) .filter(InputRowFilterResult::isRejected) .toArray(InputRowFilterResult[]::new); - public static final int NUM_FILTER_RESULT = InputRowFilterResult.values().length; private final String reason; @@ -87,24 +84,18 @@ public boolean isRejected() } /** - * Public utility for building a mutable frequency map over the possible rejection {@link InputRowFilterResult} values. - * Keys on {@link InputRowFilterResult#getReason()} rather than the enum name as the latter is more likely to change longer-term. - * It is also easier to have stats payload keys match what is being emitted in metrics. + * Returns {@link InputRowFilterResult} that are rejection states. */ - public static Map buildRejectedCounterMap() + public static InputRowFilterResult[] rejectedValues() { - final Map result = new HashMap<>(); - for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { - result.put(reason.getReason(), 0L); - } - return result; + return REJECTED_VALUES; } /** - * Returns {@link InputRowFilterResult} that are not rejection states. + * Returns total number of {@link InputRowFilterResult} values. */ - public static InputRowFilterResult[] rejectedValues() + public static int numValues() { - return REJECTED_VALUES; + return values().length; } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java index 8acb21b19893..43e1b9857056 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/RowIngestionMetersTotals.java @@ -24,6 +24,7 @@ import org.apache.druid.common.config.Configs; import javax.annotation.Nullable; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -56,11 +57,11 @@ public RowIngestionMetersTotals( } public RowIngestionMetersTotals( - @JsonProperty("processed") long processed, - @JsonProperty("processedBytes") long processedBytes, - @JsonProperty("processedWithError") long processedWithError, - @JsonProperty("thrownAway") long thrownAway, - @JsonProperty("unparseable") long unparseable + long processed, + long processedBytes, + long processedWithError, + long thrownAway, + long unparseable ) { this( @@ -73,11 +74,11 @@ public RowIngestionMetersTotals( } public RowIngestionMetersTotals( - @JsonProperty("processed") long processed, - @JsonProperty("processedBytes") long processedBytes, - @JsonProperty("processedWithError") long processedWithError, - @JsonProperty("thrownAwayByReason") Map thrownAwayByReason, - @JsonProperty("unparseable") long unparseable + long processed, + long processedBytes, + long processedWithError, + Map thrownAwayByReason, + long unparseable ) { this.processed = processed; @@ -167,8 +168,10 @@ public String toString() */ private static Map getBackwardsCompatibleThrownAwayByReason(long thrownAway) { - Map results = InputRowFilterResult.buildRejectedCounterMap(); - results.put(InputRowFilterResult.UNKNOWN.getReason(), thrownAway); + Map results = new HashMap<>(); + if (thrownAway > 0) { + results.put(InputRowFilterResult.UNKNOWN.getReason(), thrownAway); + } return results; } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java index cba7c267b076..2140b7812414 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/SimpleRowIngestionMeters.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import java.util.HashMap; import java.util.Map; public class SimpleRowIngestionMeters implements RowIngestionMeters @@ -27,7 +28,7 @@ public class SimpleRowIngestionMeters implements RowIngestionMeters private long processedWithError; private long unparseable; private long processedBytes; - private final long[] thrownAwayByReason = new long[InputRowFilterResult.NUM_FILTER_RESULT]; + private final long[] thrownAwayByReason = new long[InputRowFilterResult.numValues()]; @Override public long getProcessed() @@ -96,9 +97,12 @@ public void incrementThrownAway(InputRowFilterResult reason) @Override public Map getThrownAwayByReason() { - final Map result = InputRowFilterResult.buildRejectedCounterMap(); + final Map result = new HashMap<>(); for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { - result.put(reason.getReason(), thrownAwayByReason[reason.ordinal()]); + long count = thrownAwayByReason[reason.ordinal()]; + if (count > 0) { + result.put(reason.getReason(), count); + } } return result; } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java index 75a5530ddf0c..a8d932e4dcd9 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/InputRowFilterResultTest.java @@ -31,7 +31,7 @@ public void testOrdinalValues() Assert.assertEquals(1, InputRowFilterResult.NULL_OR_EMPTY_RECORD.ordinal()); Assert.assertEquals(2, InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.ordinal()); Assert.assertEquals(3, InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.ordinal()); - Assert.assertEquals(4, InputRowFilterResult.FILTERED.ordinal()); + Assert.assertEquals(4, InputRowFilterResult.CUSTOM_FILTER.ordinal()); } @Test @@ -39,9 +39,9 @@ public void testMetricValues() { Assert.assertEquals("accepted", InputRowFilterResult.ACCEPTED.getReason()); Assert.assertEquals("null", InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason()); - Assert.assertEquals("beforeMinMessageTime", InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason()); - Assert.assertEquals("afterMaxMessageTime", InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason()); - Assert.assertEquals("filtered", InputRowFilterResult.FILTERED.getReason()); + Assert.assertEquals("beforeMinimumMessageTime", InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason()); + Assert.assertEquals("afterMaximumMessageTime", InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason()); + Assert.assertEquals("filtered", InputRowFilterResult.CUSTOM_FILTER.getReason()); } @Test @@ -57,20 +57,8 @@ public void testIsRejected() Assert.assertTrue(InputRowFilterResult.NULL_OR_EMPTY_RECORD.isRejected()); Assert.assertTrue(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.isRejected()); Assert.assertTrue(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.isRejected()); - Assert.assertTrue(InputRowFilterResult.FILTERED.isRejected()); + Assert.assertTrue(InputRowFilterResult.CUSTOM_FILTER.isRejected()); Assert.assertTrue(InputRowFilterResult.UNKNOWN.isRejected()); } - - @Test - public void testBuildRejectedCounterMapExcludesAccepted() - { - var counterMap = InputRowFilterResult.buildRejectedCounterMap(); - Assert.assertFalse(counterMap.containsKey(InputRowFilterResult.ACCEPTED.getReason())); - Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); - Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); - Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); - Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.FILTERED.getReason())); - Assert.assertTrue(counterMap.containsKey(InputRowFilterResult.UNKNOWN.getReason())); - } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java index 2154d722439e..f155e70424dc 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/RowMeters.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import java.util.HashMap; import java.util.Map; /** @@ -29,7 +30,7 @@ public class RowMeters private long processedBytes; private long processedWithError; private long unparseable; - private final Map thrownAwayByReason = InputRowFilterResult.buildRejectedCounterMap(); + private final Map thrownAwayByReason = new HashMap<>(); /** * Creates a new {@link RowMeters}, that can be used to build an instance of diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java index 2fce81a6f8e3..9cccff0c880a 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/SimpleRowIngestionMetersTest.java @@ -35,8 +35,7 @@ public void testIncrement() rowIngestionMeters.incrementProcessedWithError(); rowIngestionMeters.incrementUnparseable(); rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); - final Map expected = InputRowFilterResult.buildRejectedCounterMap(); - expected.put(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1L); + final Map expected = Map.of(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1L); Assert.assertEquals(new RowIngestionMetersTotals(1, 5, 1, expected, 1), rowIngestionMeters.getTotals()); } @@ -44,7 +43,7 @@ public void testIncrement() public void testAddRowIngestionMetersTotals() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, 0, 1); + RowIngestionMetersTotals rowIngestionMetersTotals = new RowIngestionMetersTotals(10, 0, 1, 1, 1); rowIngestionMeters.addRowIngestionMetersTotals(rowIngestionMetersTotals); Assert.assertEquals(rowIngestionMetersTotals, rowIngestionMeters.getTotals()); } @@ -58,9 +57,9 @@ public void testIncrementThrownAwayWithReason() rowIngestionMeters.incrementThrownAway(InputRowFilterResult.NULL_OR_EMPTY_RECORD); rowIngestionMeters.incrementThrownAway(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME); rowIngestionMeters.incrementThrownAway(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME); - rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); - rowIngestionMeters.incrementThrownAway(InputRowFilterResult.FILTERED); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); + rowIngestionMeters.incrementThrownAway(InputRowFilterResult.CUSTOM_FILTER); Assert.assertEquals(7, rowIngestionMeters.getThrownAway()); @@ -68,19 +67,16 @@ public void testIncrementThrownAwayWithReason() Assert.assertEquals(Long.valueOf(2), byReason.get(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason())); Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME.getReason())); Assert.assertEquals(Long.valueOf(1), byReason.get(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME.getReason())); - Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.FILTERED.getReason())); + Assert.assertEquals(Long.valueOf(3), byReason.get(InputRowFilterResult.CUSTOM_FILTER.getReason())); } @Test - public void testGetThrownAwayByReasonReturnsAllReasons() + public void testGetThrownAwayByReasonReturnsNoRejectedReasons() { SimpleRowIngestionMeters rowIngestionMeters = new SimpleRowIngestionMeters(); - // Even with no increments, all reasons should be present with 0 counts + // With no increments, no rejected reasons should be present Map byReason = rowIngestionMeters.getThrownAwayByReason(); - Assert.assertEquals(InputRowFilterResult.rejectedValues().length, byReason.size()); - for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) { - Assert.assertEquals(Long.valueOf(0), byReason.get(reason.getReason())); - } + Assert.assertTrue(byReason.isEmpty()); } }