Skip to content

Commit decc140

Browse files
committed
Switch from null acceptance value to InputRowFilterResult.ACCEPTED
1 parent f826243 commit decc140

File tree

28 files changed

+411
-290
lines changed

28 files changed

+411
-290
lines changed

extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -104,7 +104,7 @@
104104
"ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." },
105105
"ingest/events/processedWithError" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events processed with some partial errors per emission period." },
106106
"ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." },
107-
"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."},
107+
"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."},
108108
"ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."},
109109
"ingest/input/bytes" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of bytes read from input sources, after decompression but prior to parsing." },
110110
"ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."},

extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java

Lines changed: 21 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -113,6 +113,7 @@
113113
import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
114114
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
115115
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
116+
import org.apache.druid.segment.incremental.InputRowFilterResult;
116117
import org.apache.druid.segment.incremental.RowIngestionMeters;
117118
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
118119
import org.apache.druid.segment.incremental.RowMeters;
@@ -128,6 +129,7 @@
128129
import org.apache.druid.server.security.ResourceAction;
129130
import org.apache.druid.server.security.ResourceType;
130131
import org.apache.druid.timeline.DataSegment;
132+
import org.apache.druid.utils.CollectionUtils;
131133
import org.apache.kafka.clients.producer.KafkaProducer;
132134
import org.apache.kafka.clients.producer.ProducerRecord;
133135
import org.apache.kafka.common.header.Header;
@@ -717,7 +719,7 @@ public void testIncrementalHandOff() throws Exception
717719
);
718720

719721
long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15);
720-
verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8));
722+
verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(8));
721723

722724
// Check published metadata and segments in deep storage
723725
assertEqualsExceptVersion(
@@ -851,7 +853,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
851853
);
852854

853855
long totalBytes = getTotalSizeOfRecords(0, 10) + getTotalSizeOfRecords(13, 15);
854-
verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAway(1).totalProcessed(8));
856+
verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes).unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(8));
855857

856858
// Check published metadata and segments in deep storage
857859
assertEqualsExceptVersion(
@@ -1165,7 +1167,7 @@ public void testRunWithMinimumMessageTime() throws Exception
11651167

11661168
// Wait for task to exit
11671169
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
1168-
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3));
1170+
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, 2).totalProcessed(3));
11691171

11701172
// Check published metadata and segments in deep storage
11711173
assertEqualsExceptVersion(
@@ -1214,7 +1216,7 @@ public void testRunWithMaximumMessageTime() throws Exception
12141216

12151217
// Wait for task to exit
12161218
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
1217-
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(2).totalProcessed(3));
1219+
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, 2).totalProcessed(3));
12181220

12191221
// Check published metadata and segments in deep storage
12201222
assertEqualsExceptVersion(
@@ -1272,7 +1274,7 @@ public void testRunWithTransformSpec() throws Exception
12721274

12731275
// Wait for task to exit
12741276
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
1275-
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(4).totalProcessed(1));
1277+
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1));
12761278

12771279
// Check published metadata
12781280
final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
@@ -1642,7 +1644,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception
16421644
verifyTaskMetrics(task, RowMeters.with()
16431645
.bytes(totalRecordBytes)
16441646
.unparseable(3).errors(3)
1645-
.thrownAway(1).totalProcessed(4));
1647+
.thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(4));
16461648

16471649
// Check published metadata
16481650
assertEqualsExceptVersion(
@@ -1660,14 +1662,18 @@ public void testMultipleParseExceptionsSuccess() throws Exception
16601662
Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState());
16611663
Assert.assertNull(reportData.getErrorMsg());
16621664

1665+
// Jackson will serde numerics ≤ 32bits as Integers, rather than Longs
1666+
Map<String, Integer> expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue);
1667+
expectedThrownAwayByReason.put(InputRowFilterResult.NULL_OR_EMPTY_RECORD.getReason(), 1);
16631668
Map<String, Object> expectedMetrics = ImmutableMap.of(
16641669
RowIngestionMeters.BUILD_SEGMENTS,
16651670
ImmutableMap.of(
16661671
RowIngestionMeters.PROCESSED, 4,
16671672
RowIngestionMeters.PROCESSED_BYTES, (int) totalRecordBytes,
16681673
RowIngestionMeters.PROCESSED_WITH_ERROR, 3,
16691674
RowIngestionMeters.UNPARSEABLE, 3,
1670-
RowIngestionMeters.THROWN_AWAY, 1
1675+
RowIngestionMeters.THROWN_AWAY, 1,
1676+
RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason
16711677
)
16721678
);
16731679
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1745,14 +1751,17 @@ public void testMultipleParseExceptionsFailure() throws Exception
17451751
Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState());
17461752
Assert.assertNotNull(reportData.getErrorMsg());
17471753

1754+
// Jackson will serde numerics ≤ 32bits as Integers, rather than Longs
1755+
Map<String, Integer> expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue);
17481756
Map<String, Object> expectedMetrics = ImmutableMap.of(
17491757
RowIngestionMeters.BUILD_SEGMENTS,
17501758
ImmutableMap.of(
17511759
RowIngestionMeters.PROCESSED, 3,
17521760
RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes,
17531761
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
17541762
RowIngestionMeters.UNPARSEABLE, 3,
1755-
RowIngestionMeters.THROWN_AWAY, 0
1763+
RowIngestionMeters.THROWN_AWAY, 0,
1764+
RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason
17561765
)
17571766
);
17581767
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1887,7 +1896,7 @@ public void testRunConflicting() throws Exception
18871896

18881897
verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
18891898
verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10))
1890-
.unparseable(3).thrownAway(1).totalProcessed(3));
1899+
.unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3));
18911900

18921901
// Check published segments & metadata, should all be from the first task
18931902
final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();
@@ -1961,7 +1970,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
19611970

19621971
verifyTaskMetrics(task1, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
19631972
verifyTaskMetrics(task2, RowMeters.with().bytes(getTotalSizeOfRecords(3, 10))
1964-
.unparseable(3).thrownAway(1).totalProcessed(3));
1973+
.unparseable(3).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3));
19651974

19661975
// Check published segments & metadata
19671976
SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e"));
@@ -2576,7 +2585,7 @@ public void testRunTransactionModeRollback() throws Exception
25762585

25772586
long totalBytes = getTotalSizeOfRecords(0, 2) + getTotalSizeOfRecords(5, 11);
25782587
verifyTaskMetrics(task, RowMeters.with().bytes(totalBytes)
2579-
.unparseable(3).errors(1).thrownAway(1).totalProcessed(3));
2588+
.unparseable(3).errors(1).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 1).totalProcessed(3));
25802589

25812590
// Check published metadata and segments in deep storage
25822591
assertEqualsExceptVersion(
@@ -3437,7 +3446,7 @@ public void testTaskWithTransformSpecDoesNotCauseCliPeonCyclicDependency()
34373446

34383447
// Wait for task to exit
34393448
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
3440-
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAway(4).totalProcessed(1));
3449+
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(0, 5)).thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1));
34413450

34423451
// Check published metadata
34433452
final List<SegmentDescriptor> publishedDescriptors = publishedDescriptors();

extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java

Lines changed: 13 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -82,12 +82,14 @@
8282
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
8383
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
8484
import org.apache.druid.segment.TestHelper;
85+
import org.apache.druid.segment.incremental.InputRowFilterResult;
8586
import org.apache.druid.segment.incremental.RowIngestionMeters;
8687
import org.apache.druid.segment.incremental.RowMeters;
8788
import org.apache.druid.segment.indexing.DataSchema;
8889
import org.apache.druid.segment.transform.ExpressionTransform;
8990
import org.apache.druid.segment.transform.TransformSpec;
9091
import org.apache.druid.timeline.DataSegment;
92+
import org.apache.druid.utils.CollectionUtils;
9193
import org.easymock.EasyMock;
9294
import org.joda.time.Duration;
9395
import org.joda.time.Period;
@@ -801,7 +803,7 @@ public void testRunWithMinimumMessageTime() throws Exception
801803
verifyAll();
802804

803805
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5))
804-
.thrownAway(2).totalProcessed(3));
806+
.thrownAwayByReason(InputRowFilterResult.BEFORE_MIN_MESSAGE_TIME, 2).totalProcessed(3));
805807

806808
// Check published metadata
807809
assertEqualsExceptVersion(
@@ -864,7 +866,7 @@ public void testRunWithMaximumMessageTime() throws Exception
864866
verifyAll();
865867

866868
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5))
867-
.thrownAway(2).totalProcessed(3));
869+
.thrownAwayByReason(InputRowFilterResult.AFTER_MAX_MESSAGE_TIME, 2).totalProcessed(3));
868870

869871
// Check published metadata and segments in deep storage
870872
assertEqualsExceptVersion(
@@ -923,7 +925,7 @@ public void testRunWithTransformSpec() throws Exception
923925
verifyAll();
924926

925927
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSize(RECORDS, 0, 5))
926-
.thrownAway(4).totalProcessed(1));
928+
.thrownAwayByReason(InputRowFilterResult.NULL_OR_EMPTY_RECORD, 4).totalProcessed(1));
927929

928930
// Check published metadata
929931
assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors());
@@ -1194,14 +1196,17 @@ public void testMultipleParseExceptionsSuccess() throws Exception
11941196
Assert.assertEquals(IngestionState.COMPLETED, reportData.getIngestionState());
11951197
Assert.assertNull(reportData.getErrorMsg());
11961198

1199+
// Jackson will serde numerics ≤ 32bits as Integers, rather than Longs
1200+
Map<String, Integer> expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue);
11971201
Map<String, Object> expectedMetrics = ImmutableMap.of(
11981202
RowIngestionMeters.BUILD_SEGMENTS,
11991203
ImmutableMap.of(
12001204
RowIngestionMeters.PROCESSED, 4,
12011205
RowIngestionMeters.PROCESSED_BYTES, 763,
12021206
RowIngestionMeters.PROCESSED_WITH_ERROR, 3,
12031207
RowIngestionMeters.UNPARSEABLE, 4,
1204-
RowIngestionMeters.THROWN_AWAY, 0
1208+
RowIngestionMeters.THROWN_AWAY, 0,
1209+
RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason
12051210
)
12061211
);
12071212
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1284,14 +1289,17 @@ public void testMultipleParseExceptionsFailure() throws Exception
12841289
Assert.assertEquals(IngestionState.BUILD_SEGMENTS, reportData.getIngestionState());
12851290
Assert.assertNotNull(reportData.getErrorMsg());
12861291

1292+
// Jackson will serde numerics ≤ 32bits as Integers, rather than Longs
1293+
Map<String, Integer> expectedThrownAwayByReason = CollectionUtils.mapValues(InputRowFilterResult.buildRejectedCounterMap(), Long::intValue);
12871294
Map<String, Object> expectedMetrics = ImmutableMap.of(
12881295
RowIngestionMeters.BUILD_SEGMENTS,
12891296
ImmutableMap.of(
12901297
RowIngestionMeters.PROCESSED, 3,
12911298
RowIngestionMeters.PROCESSED_BYTES, (int) totalBytes,
12921299
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
12931300
RowIngestionMeters.UNPARSEABLE, 3,
1294-
RowIngestionMeters.THROWN_AWAY, 0
1301+
RowIngestionMeters.THROWN_AWAY, 0,
1302+
RowIngestionMeters.THROWN_AWAY_BY_REASON, expectedThrownAwayByReason
12951303
)
12961304
);
12971305
Assert.assertEquals(expectedMetrics, reportData.getRowStats());

indexing-service/src/main/java/org/apache/druid/indexing/common/stats/DropwizardRowIngestionMeters.java

Lines changed: 13 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -21,12 +21,11 @@
2121

2222
import com.codahale.metrics.Meter;
2323
import com.codahale.metrics.MetricRegistry;
24-
import org.apache.druid.segment.incremental.InputRowThrownAwayReason;
24+
import org.apache.druid.segment.incremental.InputRowFilterResult;
2525
import org.apache.druid.segment.incremental.RowIngestionMeters;
2626
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
2727

2828
import java.util.Arrays;
29-
import java.util.EnumMap;
3029
import java.util.HashMap;
3130
import java.util.Map;
3231

@@ -36,13 +35,11 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters
3635
public static final String FIVE_MINUTE_NAME = "5m";
3736
public static final String FIFTEEN_MINUTE_NAME = "15m";
3837

39-
private static final int NUM_THROWN_AWAY_REASONS = InputRowThrownAwayReason.values().length;
40-
4138
private final Meter processed;
4239
private final Meter processedBytes;
4340
private final Meter processedWithError;
4441
private final Meter unparseable;
45-
private final Meter[] thrownAwayByReason = new Meter[NUM_THROWN_AWAY_REASONS];
42+
private final Meter[] thrownAwayByReason = new Meter[InputRowFilterResult.NUM_FILTER_RESULT];
4643

4744
public DropwizardRowIngestionMeters()
4845
{
@@ -51,7 +48,7 @@ public DropwizardRowIngestionMeters()
5148
this.processedBytes = metricRegistry.meter(PROCESSED_BYTES);
5249
this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR);
5350
this.unparseable = metricRegistry.meter(UNPARSEABLE);
54-
for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) {
51+
for (InputRowFilterResult reason : InputRowFilterResult.values()) {
5552
this.thrownAwayByReason[reason.ordinal()] = metricRegistry.meter(THROWN_AWAY + "_" + reason.name());
5653
}
5754
}
@@ -108,24 +105,24 @@ public void incrementUnparseable()
108105
public long getThrownAway()
109106
{
110107
long totalThrownAway = 0;
111-
for (Meter meter : thrownAwayByReason) {
112-
totalThrownAway += meter.getCount();
108+
for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) {
109+
totalThrownAway += thrownAwayByReason[reason.ordinal()].getCount();
113110
}
114111
return totalThrownAway;
115112
}
116113

117114
@Override
118-
public void incrementThrownAway(InputRowThrownAwayReason reason)
115+
public void incrementThrownAway(InputRowFilterResult reason)
119116
{
120117
thrownAwayByReason[reason.ordinal()].mark();
121118
}
122119

123120
@Override
124-
public Map<InputRowThrownAwayReason, Long> getThrownAwayByReason()
121+
public Map<String, Long> getThrownAwayByReason()
125122
{
126-
EnumMap<InputRowThrownAwayReason, Long> result = new EnumMap<>(InputRowThrownAwayReason.class);
127-
for (InputRowThrownAwayReason reason : InputRowThrownAwayReason.values()) {
128-
result.put(reason, thrownAwayByReason[reason.ordinal()].getCount());
123+
Map<String, Long> result = InputRowFilterResult.buildRejectedCounterMap();
124+
for (InputRowFilterResult reason : InputRowFilterResult.rejectedValues()) {
125+
result.put(reason.getReason(), thrownAwayByReason[reason.ordinal()].getCount());
129126
}
130127
return result;
131128
}
@@ -152,21 +149,21 @@ public Map<String, Object> getMovingAverages()
152149
oneMinute.put(PROCESSED_BYTES, processedBytes.getOneMinuteRate());
153150
oneMinute.put(PROCESSED_WITH_ERROR, processedWithError.getOneMinuteRate());
154151
oneMinute.put(UNPARSEABLE, unparseable.getOneMinuteRate());
155-
oneMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getOneMinuteRate).reduce(0.0, Double::sum));
152+
oneMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getOneMinuteRate).sum());
156153

157154
Map<String, Object> fiveMinute = new HashMap<>();
158155
fiveMinute.put(PROCESSED, processed.getFiveMinuteRate());
159156
fiveMinute.put(PROCESSED_BYTES, processedBytes.getFiveMinuteRate());
160157
fiveMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFiveMinuteRate());
161158
fiveMinute.put(UNPARSEABLE, unparseable.getFiveMinuteRate());
162-
fiveMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFiveMinuteRate).reduce(0.0, Double::sum));
159+
fiveMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getFiveMinuteRate).sum());
163160

164161
Map<String, Object> fifteenMinute = new HashMap<>();
165162
fifteenMinute.put(PROCESSED, processed.getFifteenMinuteRate());
166163
fifteenMinute.put(PROCESSED_BYTES, processedBytes.getFifteenMinuteRate());
167164
fifteenMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFifteenMinuteRate());
168165
fifteenMinute.put(UNPARSEABLE, unparseable.getFifteenMinuteRate());
169-
fifteenMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).map(Meter::getFifteenMinuteRate).reduce(0.0, Double::sum));
166+
fifteenMinute.put(THROWN_AWAY, Arrays.stream(thrownAwayByReason).mapToDouble(Meter::getFifteenMinuteRate).sum());
170167

171168
movingAverages.put(ONE_MINUTE_NAME, oneMinute);
172169
movingAverages.put(FIVE_MINUTE_NAME, fiveMinute);

0 commit comments

Comments
 (0)