@@ -446,4 +546,4 @@ See the following topics for more information:
* [Supervisor API](../api-reference/supervisor-api.md) for how to manage and monitor supervisors using the API.
* [Supervisor](../ingestion/supervisor.md) for supervisor status and capacity planning.
* [Loading from Apache Kafka](../tutorials/tutorial-kafka.md) for a tutorial on streaming data from Apache Kafka.
-* [Kafka input format](../ingestion/data-formats.md#kafka) to learn about the `kafka` input format.
\ No newline at end of file
+* [Kafka input format](../ingestion/data-formats.md#kafka) to learn about the `kafka` input format.
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 51041952cfae..d95db0e063bb 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -46,25 +46,33 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
|`query/time`|Milliseconds taken to complete a query.|Native Query: `dataSource`, `type`, `interval`, `hasFilters`, `duration`, `context`, `remoteAddress`, `id`.|< 1s|
+|`router/http/numRequestsQueued`|Total number of outbound HTTP requests currently queued across all destinations on the Router's HTTP client.|None.|0; sustained high values indicate backpressure toward downstream Brokers.|
### Broker
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
|`query/time`|Milliseconds taken to complete a query.|Common: `dataSource`, `type`, `interval`, `hasFilters`, `duration`, `context`, `remoteAddress`, `id`.
Aggregation Queries: `numMetrics`, `numComplexMetrics`.
GroupBy: `numDimensions`.
TopN: `threshold`, `dimension`.
|< 1s|
+|`query/planningTime`|Milliseconds taken to complete query planning at Broker before query is fanned out to the Data nodes.|Common: `dataSource`, `type`, `interval`, `hasFilters`, `duration`, `context`, `remoteAddress`, `id`.
Aggregation Queries: `numMetrics`, `numComplexMetrics`.
GroupBy: `numDimensions`.
TopN: `threshold`, `dimension`.
|< 1s|
|`query/bytes`|The total number of bytes returned to the requesting client in the query response from the broker. Other services report the total bytes for their portion of the query. |Common: `dataSource`, `type`, `interval`, `hasFilters`, `duration`, `context`, `remoteAddress`, `id`.
Aggregation Queries: `numMetrics`, `numComplexMetrics`.
GroupBy: `numDimensions`.
TopN: `threshold`, `dimension`.
| |
|`query/node/time`|Milliseconds taken to query individual historical/realtime processes.|`id`, `status`, `server`|< 1s|
|`query/node/bytes`|Number of bytes returned from querying individual historical/realtime processes.|`id`, `status`, `server`| |
|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime processes.|`id`, `status`, `server`|< 1s|
|`query/node/backpressure`|Milliseconds that the channel to this process has spent suspended due to backpressure.|`id`, `status`, `server`.| |
+|`query/rows/scanned`|Number of rows scanned by the Data nodes to address the query.|`id`, `status`, `server`.| |
|`query/count`|Number of total queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/success/count`|Number of queries successfully processed.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
+|`query/merge/buffersUsed`|Number of merge buffers used up to merge the results of group by queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/segments/count`|This metric is not enabled by default. See the `QueryMetrics` Interface for reference regarding enabling this metric. Number of segments that will be touched by the query. In the broker, it makes a plan to distribute the query to realtime tasks and historicals based on a snapshot of segment distribution state. If there are some segments moved after this snapshot is created, certain historicals and realtime tasks can report those segments as missing to the broker. The broker will resend the query to the new servers that serve those segments after move. In this case, those segments can be counted more than once in this metric.||Varies|
|`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|`lane`, `dataSource`, `type`|0|
+|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`|< 1s|
+|`sqlQuery/planningTimeMs`|Milliseconds taken to plan a SQL to native query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`| |
+|`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`| |
+|`httpClient/channelAcquire/timeNs`|Time in nanoseconds spent by the httpclient to acquire the channel.| |
|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`|< 1s|
|`sqlQuery/planningTimeMs`|Milliseconds taken to plan a SQL to native query.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`| |
|`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`| |
@@ -100,6 +108,7 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/success/count`|Number of queries successfully processed.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
+|`query/merge/buffersUsed`|Number of merge buffers used up to merge the results of group by queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.| |
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
@@ -117,6 +126,8 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
+|`query/merge/buffersUsed`|Number of merge buffers used up to merge the results of group by queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
+|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
### Jetty
@@ -240,6 +251,7 @@ batch ingestion emit the following metrics. These metrics are deltas for each em
|`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`, or `windowPeriod`.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0|
+|`ingest/events/filtered`|Number of events rejected by header-based filtering before parsing.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|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/aliyun-oss-extensions/pom.xml b/extensions-contrib/aliyun-oss-extensions/pom.xml
index 3ed01c7fa7bb..57392cf42fe5 100644
--- a/extensions-contrib/aliyun-oss-extensions/pom.xml
+++ b/extensions-contrib/aliyun-oss-extensions/pom.xml
@@ -43,6 +43,12 @@
com.aliyun.oss
aliyun-sdk-oss
3.11.3
+
+
+ javax.xml.bind
+ jaxb-api
+
+
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json b/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json
index 8ac86ed46fc7..d492419bb37a 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json
+++ b/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json
@@ -46,6 +46,10 @@
"dataSource",
"type"
],
+ "query/planningTime": [
+ "dataSource",
+ "type"
+ ],
"query/wait/time": [
"dataSource",
"type"
diff --git a/extensions-contrib/confluent-extensions/pom.xml b/extensions-contrib/confluent-extensions/pom.xml
new file mode 100644
index 000000000000..ab27aa893700
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/pom.xml
@@ -0,0 +1,75 @@
+
+
+
+
+
+ 4.0.0
+
+ io.confluent.druid.extensions
+ confluent-extensions
+ confluent-extensions
+ confluent-extensions
+
+
+ druid
+ org.apache.druid
+ 30.0.1
+ ../../pom.xml
+
+
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ provided
+
+
+ com.google.code.findbugs
+ jsr305
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ provided
+
+
+ com.google.guava
+ guava
+ provided
+
+
+ com.google.inject
+ guice
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ provided
+
+
+
+ junit
+ junit
+ test
+
+
+ org.mockito
+ mockito-core
+ test
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ test
+ test-jar
+
+
+
diff --git a/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/ConfluentExtensionsModule.java b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/ConfluentExtensionsModule.java
new file mode 100644
index 000000000000..3b67c6c1be8c
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/ConfluentExtensionsModule.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import io.confluent.druid.transform.EnrichResourceNameTransform;
+import io.confluent.druid.transform.ExtractTenantTopicTransform;
+import io.confluent.druid.transform.ExtractTenantTransform;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.Collections;
+import java.util.List;
+
+public class ConfluentExtensionsModule implements DruidModule
+{
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.singletonList(
+ new SimpleModule("ConfluentTransformsModule")
+ .registerSubtypes(
+ new NamedType(ExtractTenantTransform.class, "extractTenant"),
+ new NamedType(ExtractTenantTopicTransform.class, "extractTenantTopic"),
+ new NamedType(EnrichResourceNameTransform.class, "enrichResourceName")
+ )
+ );
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ }
+}
diff --git a/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/EnrichResourceNameTransform.java b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/EnrichResourceNameTransform.java
new file mode 100644
index 000000000000..05053ef07093
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/EnrichResourceNameTransform.java
@@ -0,0 +1,340 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.data.input.Row;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+import org.apache.druid.query.lookup.LookupExtractor;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+import org.apache.druid.segment.transform.RowFunction;
+import org.apache.druid.segment.transform.Transform;
+
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+public class EnrichResourceNameTransform implements Transform
+{
+ private final String name;
+ private final String metricNameDimension;
+ private final Set kafkaMetricPrefixes;
+ private final String kafkaResourceIdDimension;
+ private final String kafkaResourceIdDerivedDimension;
+ private final Set tableflowMetricPrefixes;
+ private final String tableflowResourceIdDimension;
+ private final Set connectMetricPrefixes;
+ private final String connectResourceIdDimension;
+ private final Set clientConnectorMetricPrefixes;
+ private final String clientConnectorResourceIdDimension;
+ private final Set ksqlMetricPrefixes;
+ private final String ksqlResourceIdDimension;
+ private final Set schemaRegistryMetricPrefixes;
+ private final String schemaRegistryResourceIdDimension;
+ private final Set fcpMetricPrefixes;
+ private final String fcpResourceIdDimension;
+ private final String lookupName;
+ private final LookupExtractorFactoryContainerProvider lookupProvider;
+
+ private static final EmittingLogger log = new EmittingLogger(EnrichResourceNameTransform.class);
+
+ public EnrichResourceNameTransform(
+ @JsonProperty("name") final String name,
+ @JsonProperty("metricNameDimension") final String metricNameDimension,
+ @JsonProperty("kafkaMetricPrefixes") final Set kafkaMetricPrefixes,
+ @JsonProperty("kafkaResourceIdDimension") final String kafkaResourceIdDimension,
+ @JsonProperty("kafkaResourceIdDerivedDimension") final String kafkaResourceIdDerivedDimension,
+ @JsonProperty("tableflowMetricPrefixes") final Set tableflowMetricPrefixes,
+ @JsonProperty("tableflowResourceIdDimension") final String tableflowResourceIdDimension,
+ @JsonProperty("connectMetricPrefixes") final Set connectMetricPrefixes,
+ @JsonProperty("connectResourceIdDimension") final String connectResourceIdDimension,
+ @JsonProperty("clientConnectorMetricPrefixes") final Set clientConnectorMetricPrefixes,
+ @JsonProperty("clientConnectorResourceIdDimension") final String clientConnectorResourceIdDimension,
+ @JsonProperty("ksqlMetricPrefixes") final Set ksqlMetricPrefixes,
+ @JsonProperty("ksqlResourceIdDimension") final String ksqlResourceIdDimension,
+ @JsonProperty("schemaRegistryMetricPrefixes") final Set schemaRegistryMetricPrefixes,
+ @JsonProperty("schemaRegistryResourceIdDimension") final String schemaRegistryResourceIdDimension,
+ @JsonProperty("fcpMetricPrefixes") final Set fcpMetricPrefixes,
+ @JsonProperty("fcpResourceIdDimension") final String fcpResourceIdDimension,
+ @JsonProperty("lookupName") final String lookupName,
+ @JacksonInject LookupExtractorFactoryContainerProvider lookupProvider
+ )
+ {
+ this.name = Preconditions.checkNotNull(name, "Specify output-column name");
+ this.metricNameDimension = Preconditions.checkNotNull(metricNameDimension, "Specify metric-name column : metricNameDimension");
+ this.kafkaMetricPrefixes = kafkaMetricPrefixes != null ? kafkaMetricPrefixes : new HashSet<>();
+ this.kafkaResourceIdDimension = kafkaResourceIdDimension != null ? kafkaResourceIdDimension : "";
+ this.kafkaResourceIdDerivedDimension = kafkaResourceIdDerivedDimension != null ? kafkaResourceIdDerivedDimension : "";
+ this.tableflowMetricPrefixes = tableflowMetricPrefixes != null ? tableflowMetricPrefixes : new HashSet<>();
+ this.tableflowResourceIdDimension = tableflowResourceIdDimension != null ? tableflowResourceIdDimension : "";
+ this.connectMetricPrefixes = connectMetricPrefixes != null ? connectMetricPrefixes : new HashSet<>();
+ this.connectResourceIdDimension = connectResourceIdDimension != null ? connectResourceIdDimension : "";
+ this.clientConnectorMetricPrefixes = clientConnectorMetricPrefixes != null ? clientConnectorMetricPrefixes : new HashSet<>();
+ this.clientConnectorResourceIdDimension = clientConnectorResourceIdDimension != null ? clientConnectorResourceIdDimension : "";
+ this.ksqlMetricPrefixes = ksqlMetricPrefixes != null ? ksqlMetricPrefixes : new HashSet<>();
+ this.ksqlResourceIdDimension = ksqlResourceIdDimension != null ? ksqlResourceIdDimension : "";
+ this.schemaRegistryMetricPrefixes = schemaRegistryMetricPrefixes != null ? schemaRegistryMetricPrefixes : new HashSet<>();
+ this.schemaRegistryResourceIdDimension = schemaRegistryResourceIdDimension != null ? schemaRegistryResourceIdDimension : "";
+ this.fcpMetricPrefixes = fcpMetricPrefixes != null ? fcpMetricPrefixes : new HashSet<>();
+ this.fcpResourceIdDimension = fcpResourceIdDimension != null ? fcpResourceIdDimension : "";
+ this.lookupName = Preconditions.checkNotNull(lookupName, "Specify lookup-name");
+ this.lookupProvider = Preconditions.checkNotNull(lookupProvider, "Specify lookupProvider");
+ }
+
+ @JsonProperty
+ @Override
+ public String getName()
+ {
+ return name;
+ }
+
+ @JsonProperty
+ public String getMetricNameDimension()
+ {
+ return metricNameDimension;
+ }
+
+ @JsonProperty
+ public String getKafkaResourceIdDimension()
+ {
+ return kafkaResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getKafkaResourceIdDerivedDimension()
+ {
+ return kafkaResourceIdDerivedDimension;
+ }
+
+ @JsonProperty
+ public String getTableflowResourceIdDimension()
+ {
+ return tableflowResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getConnectResourceIdDimension()
+ {
+ return connectResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getClientConnectorResourceIdDimension()
+ {
+ return clientConnectorResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getKsqlResourceIdDimension()
+ {
+ return ksqlResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getSchemaRegistryResourceIdDimension()
+ {
+ return schemaRegistryResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getFcpResourceIdDimension()
+ {
+ return fcpResourceIdDimension;
+ }
+
+ @JsonProperty
+ public String getLookupName()
+ {
+ return lookupName;
+ }
+
+ @JsonProperty
+ public Set getKafkaMetricPrefixes()
+ {
+ return kafkaMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getTableflowMetricPrefixes()
+ {
+ return tableflowMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getConnectMetricPrefixes()
+ {
+ return connectMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getClientConnectorMetricPrefixes()
+ {
+ return clientConnectorMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getKsqlMetricPrefixes()
+ {
+ return ksqlMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getSchemaRegistryMetricPrefixes()
+ {
+ return schemaRegistryMetricPrefixes;
+ }
+
+ @JsonProperty
+ public Set getFcpMetricPrefixes()
+ {
+ return fcpMetricPrefixes;
+ }
+
+ @Override
+ public RowFunction getRowFunction()
+ {
+ return row -> {
+ try {
+ Optional container = lookupProvider.get(lookupName);
+ if (!container.isPresent()) {
+ return null;
+ }
+ LookupExtractor lookup = container.get().getLookupExtractorFactory().get();
+ String metricName = row.getRaw(metricNameDimension).toString();
+
+ if (metricName != null) {
+ // Check if metric name starts with any kafka prefix
+ for (String prefix : kafkaMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, kafkaResourceIdDimension, kafkaResourceIdDerivedDimension);
+ }
+ }
+ // Check if metric name starts with any connect prefix
+ for (String prefix : connectMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, connectResourceIdDimension, null);
+ }
+ }
+ // Check if metric name starts with any client-connector prefix
+ for (String prefix : clientConnectorMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, clientConnectorResourceIdDimension, null);
+ }
+ }
+ // Check if metric name starts with any ksql prefix
+ for (String prefix : ksqlMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, ksqlResourceIdDimension, null);
+ }
+ }
+ // Check if metric name starts with any schema registry prefix
+ for (String prefix : schemaRegistryMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, schemaRegistryResourceIdDimension, null);
+ }
+ }
+ // Check if metric name starts with any flink-compute-pool prefix
+ for (String prefix : fcpMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, fcpResourceIdDimension, null);
+ }
+ }
+ // Check if metric name starts with any tableflow prefix
+ for (String prefix : tableflowMetricPrefixes) {
+ if (metricName.startsWith(prefix)) {
+ return enrichNameFromLookup(row, lookup, tableflowResourceIdDimension, null);
+ }
+ }
+ }
+ }
+ catch (Exception ex) {
+ log.warn("Failed to enrich name dut to exception %s", ex.getMessage());
+ }
+ return null;
+ };
+ }
+
+ private String enrichNameFromLookup(Row row, LookupExtractor lookup, String resourceIdDimension, String resourceIdDerivedDimension)
+ {
+ Object resourceIdObject = row.getRaw(resourceIdDimension);
+
+ String resourceId = resourceIdObject != null ? resourceIdObject.toString() : null;
+ if (resourceId == null) {
+ if (resourceIdDerivedDimension != null) {
+ Object resourceIdDerivedObject = row.getRaw(resourceIdDerivedDimension);
+ if (resourceIdDerivedObject != null) {
+ resourceId = TenantUtils.extractTenant(resourceIdDerivedObject.toString());
+ }
+ }
+ if (resourceId == null) {
+ return null;
+ }
+ }
+
+ return lookup.apply(resourceId);
+ }
+
+ @Override
+ public Set getRequiredColumns()
+ {
+ Set columns = new HashSet<>();
+ columns.add(this.name);
+ columns.add(this.metricNameDimension);
+ columns.add(this.kafkaResourceIdDimension);
+ columns.add(this.tableflowResourceIdDimension);
+ columns.add(this.connectResourceIdDimension);
+ columns.add(this.clientConnectorResourceIdDimension);
+ columns.add(this.ksqlResourceIdDimension);
+ columns.add(this.schemaRegistryResourceIdDimension);
+ columns.add(this.fcpResourceIdDimension);
+ return columns;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof EnrichResourceNameTransform)) {
+ return false;
+ }
+ EnrichResourceNameTransform that = (EnrichResourceNameTransform) o;
+ return name.equals(that.name) &&
+ Objects.equals(kafkaMetricPrefixes, that.kafkaMetricPrefixes) &&
+ Objects.equals(tableflowMetricPrefixes, that.tableflowMetricPrefixes) &&
+ Objects.equals(connectMetricPrefixes, that.connectMetricPrefixes) &&
+ Objects.equals(clientConnectorMetricPrefixes, that.clientConnectorMetricPrefixes) &&
+ Objects.equals(ksqlMetricPrefixes, that.ksqlMetricPrefixes) &&
+ Objects.equals(schemaRegistryMetricPrefixes, that.schemaRegistryMetricPrefixes) &&
+ Objects.equals(fcpMetricPrefixes, that.fcpMetricPrefixes);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(name, kafkaMetricPrefixes, tableflowMetricPrefixes, connectMetricPrefixes,
+ clientConnectorMetricPrefixes, ksqlMetricPrefixes, schemaRegistryMetricPrefixes, fcpMetricPrefixes);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "EnrichResourceNameTransform{" +
+ "name='" + name + '\'' +
+ ", kafkaMetricPrefixes=" + kafkaMetricPrefixes +
+ ", tableflowMetricPrefixes=" + tableflowMetricPrefixes +
+ ", connectMetricPrefixes=" + connectMetricPrefixes +
+ ", clientConnectorMetricPrefixes=" + clientConnectorMetricPrefixes +
+ ", ksqlMetricPrefixes=" + ksqlMetricPrefixes +
+ ", schemaRegistryMetricPrefixes=" + schemaRegistryMetricPrefixes +
+ ", fcpMetricPrefixes=" + fcpMetricPrefixes +
+ '}';
+ }
+}
diff --git a/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTopicTransform.java b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTopicTransform.java
new file mode 100644
index 000000000000..914d1cebc3cb
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTopicTransform.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.segment.transform.RowFunction;
+import org.apache.druid.segment.transform.Transform;
+
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+public class ExtractTenantTopicTransform implements Transform
+{
+ private final String fieldName;
+ private final String name;
+
+ public ExtractTenantTopicTransform(
+ @JsonProperty("name") final String name,
+ @JsonProperty("fieldName") final String fieldName
+ )
+ {
+ this.name = Preconditions.checkNotNull(name, "name");
+ this.fieldName = Preconditions.checkNotNull(fieldName, "fieldName");
+ }
+
+ @JsonProperty
+ @Override
+ public String getName()
+ {
+ return name;
+ }
+
+ @JsonProperty
+ public String getFieldName()
+ {
+ return fieldName;
+ }
+
+ @Override
+ public RowFunction getRowFunction()
+ {
+ return row -> {
+ Object existing = row.getRaw(name);
+ // do not overwrite existing values if present
+ if (existing != null) {
+ return existing;
+ }
+
+ Object value = row.getRaw(fieldName);
+ return value == null ? null : TenantUtils.extractTenantTopic(value.toString());
+ };
+ }
+
+ @Override
+ public Set getRequiredColumns()
+ {
+ Set columns = new HashSet();
+ columns.add(this.name);
+ columns.add(this.fieldName);
+ return columns;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof ExtractTenantTopicTransform)) {
+ return false;
+ }
+ ExtractTenantTopicTransform that = (ExtractTenantTopicTransform) o;
+ return fieldName.equals(that.fieldName) &&
+ name.equals(that.name);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(fieldName, name);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ExtractTenantTopicTransform{" +
+ "fieldName='" + fieldName + '\'' +
+ ", name='" + name + '\'' +
+ '}';
+ }
+}
diff --git a/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTransform.java b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTransform.java
new file mode 100644
index 000000000000..4b6ad09d6400
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/ExtractTenantTransform.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.segment.transform.RowFunction;
+import org.apache.druid.segment.transform.Transform;
+
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+public class ExtractTenantTransform implements Transform
+{
+ private final String fieldName;
+ private final String name;
+
+ public ExtractTenantTransform(
+ @JsonProperty("name") final String name,
+ @JsonProperty("fieldName") final String fieldName
+ )
+ {
+ this.name = Preconditions.checkNotNull(name, "name");
+ this.fieldName = Preconditions.checkNotNull(fieldName, "fieldName");
+ }
+
+ @JsonProperty
+ @Override
+ public String getName()
+ {
+ return name;
+ }
+
+ @JsonProperty
+ public String getFieldName()
+ {
+ return fieldName;
+ }
+
+ @Override
+ public RowFunction getRowFunction()
+ {
+ return row -> {
+ Object existing = row.getRaw(name);
+ // do not overwrite existing values if present
+ if (existing != null) {
+ return existing;
+ }
+
+ Object value = row.getRaw(fieldName);
+ return value == null ? null : TenantUtils.extractTenant(value.toString());
+ };
+ }
+
+ @Override
+ public Set getRequiredColumns()
+ {
+ Set columns = new HashSet();
+ columns.add(this.name);
+ columns.add(this.fieldName);
+ return columns;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof ExtractTenantTransform)) {
+ return false;
+ }
+ ExtractTenantTransform that = (ExtractTenantTransform) o;
+ return fieldName.equals(that.fieldName) &&
+ name.equals(that.name);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(fieldName, name);
+ }
+
+ @Override
+ public String toString()
+ {
+ return "ExtractTenantTransform{" +
+ "fieldName='" + fieldName + '\'' +
+ ", name='" + name + '\'' +
+ '}';
+ }
+}
diff --git a/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/TenantUtils.java b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/TenantUtils.java
new file mode 100644
index 000000000000..1a4e8c66df24
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/java/io/confluent/druid/transform/TenantUtils.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import javax.annotation.Nullable;
+
+public class TenantUtils
+{
+ private static final char DELIMITER = '_';
+
+ @Nullable
+ public static String extractTenant(String prefixedTopic)
+ {
+ int i = prefixedTopic.indexOf(DELIMITER);
+ return i > 0 ? prefixedTopic.substring(0, i) : null;
+ }
+
+ @Nullable
+ public static String extractTenantTopic(String prefixedTopic)
+ {
+ int i = prefixedTopic.indexOf(DELIMITER);
+ return i > 0 ? prefixedTopic.substring(i + 1) : null;
+ }
+}
diff --git a/extensions-contrib/confluent-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/confluent-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100644
index 000000000000..f14e0fe0915b
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,3 @@
+# Copyright 2020 Confluent Inc.
+
+io.confluent.druid.ConfluentExtensionsModule
diff --git a/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/EnrichResourceNameTransformTest.java b/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/EnrichResourceNameTransformTest.java
new file mode 100644
index 000000000000..3b7cfbad0728
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/EnrichResourceNameTransformTest.java
@@ -0,0 +1,847 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.data.input.impl.MapInputRowParser;
+import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.query.lookup.LookupExtractor;
+import org.apache.druid.query.lookup.LookupExtractorFactory;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
+import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
+import org.apache.druid.segment.transform.TransformSpec;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class EnrichResourceNameTransformTest
+{
+ @Mock
+ private LookupExtractorFactoryContainerProvider mockLookupProvider;
+
+ @Mock
+ private LookupExtractorFactoryContainer mockLookupContainer;
+
+ @Mock
+ private LookupExtractorFactory mockLookupExtractorFactory;
+
+ @Mock
+ private LookupExtractor mockLookupExtractor;
+
+ private static final MapInputRowParser PARSER = new MapInputRowParser(
+ new TimeAndDimsParseSpec(
+ new TimestampSpec("t", "auto", DateTimes.of("2020-01-01")),
+ new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("metric_name", "connector_id", "tenant", "logical_cluster_id", "compute_pool")))
+ )
+ );
+
+ @Before
+ public void setUp()
+ {
+ // Setup mock lookup behavior
+ when(mockLookupProvider.get("resource_display_name_test_lookup"))
+ .thenReturn(Optional.of(mockLookupContainer));
+ when(mockLookupContainer.getLookupExtractorFactory())
+ .thenReturn(mockLookupExtractorFactory);
+
+ // Setup lookup data
+ when(mockLookupExtractor.apply("lkc-abc123")).thenReturn("My Kafka Cluster");
+ when(mockLookupExtractor.apply("lcc-xyz789")).thenReturn("My Connect Cluster");
+ when(mockLookupExtractor.apply("lcc-client-connector-123")).thenReturn("My Client Connector");
+ when(mockLookupExtractor.apply("tableflow-123")).thenReturn("My Tableflow");
+ when(mockLookupExtractor.apply("lsrc-def456")).thenReturn("My Schema Registry");
+ when(mockLookupExtractor.apply("ksql-ghi789")).thenReturn("My KSQL Cluster");
+ when(mockLookupExtractor.apply("fcp-jkl012")).thenReturn("My Flink Compute Pool");
+ }
+
+ @Test
+ public void testKafkaMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-", "kafka_"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Kafka metric with tenant lookup
+ Map kafkaRowData = ImmutableMap.builder()
+ .put("metric_name", "kafka-producer-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .put("kafka_resource_derived", "lkc-abc123_topic1")
+ .build();
+
+ InputRow kafkaRow = parser.parseBatch(kafkaRowData).get(0);
+ Assert.assertNotNull(kafkaRow);
+ Assert.assertEquals("My Kafka Cluster", kafkaRow.getRaw("resource_name"));
+ }
+
+
+ @Test
+ public void testKafkaMetricWithDerivedResource()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Kafka metric with only derived resource ID (kafka_resource is null)
+ Map kafkaRowData = ImmutableMap.builder()
+ .put("metric_name", "kafka-producer-metrics")
+ .put("kafka_resource_derived", "lkc-abc123_topic1")
+ .build();
+
+ InputRow kafkaRow = parser.parseBatch(kafkaRowData).get(0);
+ Assert.assertNotNull(kafkaRow);
+ Assert.assertEquals("My Kafka Cluster", kafkaRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testConnectMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of("connect-", "kafka-connect-"),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Connect metric with connector_id lookup
+ Map connectRowData = ImmutableMap.builder()
+ .put("metric_name", "connect-kafka-source-metrics")
+ .put("connect_resource", "lcc-xyz789")
+ .build();
+
+ InputRow connectRow = parser.parseBatch(connectRowData).get(0);
+ Assert.assertNotNull(connectRow);
+ Assert.assertEquals("My Connect Cluster", connectRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testSchemaRegistryMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of("schema_registry-"),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Schema Registry metric with tenant lookup
+ Map schemaRegistryRowData = ImmutableMap.builder()
+ .put("metric_name", "schema_registry-subject-metrics")
+ .put("schema_registry_resource", "lsrc-def456")
+ .build();
+
+ InputRow schemaRegistryRow = parser.parseBatch(schemaRegistryRowData).get(0);
+ Assert.assertNotNull(schemaRegistryRow);
+ Assert.assertEquals("My Schema Registry", schemaRegistryRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testKSQLMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of("ksql-"),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test KSQL metric with logical_cluster_id lookup
+ Map ksqlRowData = ImmutableMap.builder()
+ .put("metric_name", "ksql-query-metrics")
+ .put("ksql_resource", "ksql-ghi789")
+ .build();
+
+ InputRow ksqlRow = parser.parseBatch(ksqlRowData).get(0);
+ Assert.assertNotNull(ksqlRow);
+ Assert.assertEquals("My KSQL Cluster", ksqlRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testFCPMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of("fcp-"),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test FCP metric with compute_pool lookup
+ Map fcpRowData = ImmutableMap.builder()
+ .put("metric_name", "fcp-compute-metrics")
+ .put("fcp_resource", "fcp-jkl012")
+ .build();
+
+ InputRow fcpRow = parser.parseBatch(fcpRowData).get(0);
+ Assert.assertNotNull(fcpRow);
+ Assert.assertEquals("My Flink Compute Pool", fcpRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testNoPrefixMatch()
+ {
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test metric that doesn't match any prefix
+ Map rowData = ImmutableMap.builder()
+ .put("metric_name", "unknown-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .build();
+
+ InputRow row = parser.parseBatch(rowData).get(0);
+ Assert.assertNotNull(row);
+ Assert.assertNull(row.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testLookupNotFound()
+ {
+ when(mockLookupProvider.get("nonexistent_lookup"))
+ .thenReturn(Optional.empty());
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "nonexistent_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test with nonexistent lookup
+ Map rowData = ImmutableMap.builder()
+ .put("metric_name", "kafka-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .build();
+
+ InputRow row = parser.parseBatch(rowData).get(0);
+ Assert.assertNotNull(row);
+ Assert.assertNull(row.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testLookupReturnsNull()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test with resource ID that doesn't exist in lookup
+ Map rowData = ImmutableMap.builder()
+ .put("metric_name", "kafka-metrics")
+ .put("kafka_resource", "unknown-id")
+ .build();
+
+ InputRow row = parser.parseBatch(rowData).get(0);
+ Assert.assertNotNull(row);
+ Assert.assertNull(row.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testMultiplePrefixMatching()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-", "kafka_"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test both kafka- and kafka_ prefixes
+ Map kafkaDashData = ImmutableMap.builder()
+ .put("metric_name", "kafka-producer-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .build();
+
+ InputRow kafkaDashRow = parser.parseBatch(kafkaDashData).get(0);
+ Assert.assertNotNull(kafkaDashRow);
+ Assert.assertEquals("My Kafka Cluster", kafkaDashRow.getRaw("resource_name"));
+
+ Map kafkaUnderscoreData = ImmutableMap.builder()
+ .put("metric_name", "kafka_consumer-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .build();
+
+ InputRow kafkaUnderscoreRow = parser.parseBatch(kafkaUnderscoreData).get(0);
+ Assert.assertNotNull(kafkaUnderscoreRow);
+ Assert.assertEquals("My Kafka Cluster", kafkaUnderscoreRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testGetRequiredColumns()
+ {
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ Set requiredColumns = transform.getRequiredColumns();
+ Assert.assertTrue(requiredColumns.contains("resource_name"));
+ Assert.assertTrue(requiredColumns.contains("metric_name"));
+ Assert.assertTrue(requiredColumns.contains("kafka_resource"));
+ Assert.assertTrue(requiredColumns.contains("tableflow_resource"));
+ Assert.assertTrue(requiredColumns.contains("connect_resource"));
+ Assert.assertTrue(requiredColumns.contains("client_connector_resource"));
+ Assert.assertTrue(requiredColumns.contains("ksql_resource"));
+ Assert.assertTrue(requiredColumns.contains("schema_registry_resource"));
+ Assert.assertTrue(requiredColumns.contains("fcp_resource"));
+ }
+
+ @Test
+ public void testClientConnectorMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of("client-connector-", "client_connector-"),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Client Connector metric with client connector resource ID lookup
+ Map clientConnectorRowData = ImmutableMap.builder()
+ .put("metric_name", "client-connector-metrics")
+ .put("client_connector_resource", "lcc-client-connector-123")
+ .build();
+
+ InputRow clientConnectorRow = parser.parseBatch(clientConnectorRowData).get(0);
+ Assert.assertNotNull(clientConnectorRow);
+ Assert.assertEquals("My Client Connector", clientConnectorRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testTableflowMetricEnrichment()
+ {
+ when(mockLookupExtractorFactory.get()).thenReturn(mockLookupExtractor);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of("tableflow-"),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+
+ // Test Tableflow metric with tableflow resource ID lookup
+ Map tableflowRowData = ImmutableMap.builder()
+ .put("metric_name", "tableflow-metrics")
+ .put("tableflow_resource", "tableflow-123")
+ .build();
+
+ InputRow tableflowRow = parser.parseBatch(tableflowRowData).get(0);
+ Assert.assertNotNull(tableflowRow);
+ Assert.assertEquals("My Tableflow", tableflowRow.getRaw("resource_name"));
+ }
+
+ @Test
+ public void testConstructorWithNullResourceIdDimensions()
+ {
+ // Test that constructor accepts null values for resource ID dimensions (defaults to empty strings)
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ null, // kafkaResourceIdDimension
+ null, // kafkaResourceIdDerivedDimension
+ ImmutableSet.of(),
+ null, // tableflowResourceIdDimension
+ ImmutableSet.of(),
+ null, // connectResourceIdDimension
+ ImmutableSet.of(),
+ null, // clientConnectorResourceIdDimension
+ ImmutableSet.of(),
+ null, // ksqlResourceIdDimension
+ ImmutableSet.of(),
+ null, // schemaRegistryResourceIdDimension
+ ImmutableSet.of(),
+ null, // fcpResourceIdDimension
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ Assert.assertNotNull(transform);
+ Assert.assertEquals("", transform.getKafkaResourceIdDimension());
+ Assert.assertEquals("", transform.getKafkaResourceIdDerivedDimension());
+ Assert.assertEquals("", transform.getTableflowResourceIdDimension());
+ Assert.assertEquals("", transform.getConnectResourceIdDimension());
+ Assert.assertEquals("", transform.getClientConnectorResourceIdDimension());
+ Assert.assertEquals("", transform.getKsqlResourceIdDimension());
+ Assert.assertEquals("", transform.getSchemaRegistryResourceIdDimension());
+ Assert.assertEquals("", transform.getFcpResourceIdDimension());
+ }
+
+ @Test
+ public void testConstructorWithNullMetricPrefixSets()
+ {
+ // Test that constructor accepts null values for metric prefix sets (defaults to empty HashSets)
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ null, // kafkaMetricPrefixes
+ "kafka_resource",
+ "kafka_resource_derived",
+ null, // tableflowMetricPrefixes
+ "tableflow_resource",
+ null, // connectMetricPrefixes
+ "connect_resource",
+ null, // clientConnectorMetricPrefixes
+ "client_connector_resource",
+ null, // ksqlMetricPrefixes
+ "ksql_resource",
+ null, // schemaRegistryMetricPrefixes
+ "schema_registry_resource",
+ null, // fcpMetricPrefixes
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ Assert.assertNotNull(transform);
+ Assert.assertNotNull(transform.getKafkaMetricPrefixes());
+ Assert.assertTrue(transform.getKafkaMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getTableflowMetricPrefixes());
+ Assert.assertTrue(transform.getTableflowMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getConnectMetricPrefixes());
+ Assert.assertTrue(transform.getConnectMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getClientConnectorMetricPrefixes());
+ Assert.assertTrue(transform.getClientConnectorMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getKsqlMetricPrefixes());
+ Assert.assertTrue(transform.getKsqlMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getSchemaRegistryMetricPrefixes());
+ Assert.assertTrue(transform.getSchemaRegistryMetricPrefixes().isEmpty());
+ Assert.assertNotNull(transform.getFcpMetricPrefixes());
+ Assert.assertTrue(transform.getFcpMetricPrefixes().isEmpty());
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testConstructorWithNullName()
+ {
+ // Test that constructor throws exception when name is null
+ new EnrichResourceNameTransform(
+ null, // name
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testConstructorWithNullMetricNameDimension()
+ {
+ // Test that constructor throws exception when metricNameDimension is null
+ new EnrichResourceNameTransform(
+ "resource_name",
+ null, // metricNameDimension
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testConstructorWithNullLookupName()
+ {
+ // Test that constructor throws exception when lookupName is null
+ new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ null, // lookupName
+ mockLookupProvider
+ );
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testConstructorWithNullLookupProvider()
+ {
+ // Test that constructor throws exception when lookupProvider is null
+ new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of(),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ null // lookupProvider
+ );
+ }
+
+ @Test
+ public void testInterruptedExceptionHandling()
+ {
+ // This can occur when the lookup provider is not yet started or thread is interrupted
+ // (e.g., during JVM shutdown or ingestion task cancellation by overlord)
+ InterruptedException interruptedException = new InterruptedException("Thread interrupted");
+ RuntimeException runtimeException = new RuntimeException(interruptedException);
+
+ when(mockLookupProvider.get("resource_display_name_test_lookup"))
+ .thenThrow(runtimeException);
+
+ EnrichResourceNameTransform transform = new EnrichResourceNameTransform(
+ "resource_name",
+ "metric_name",
+ ImmutableSet.of("kafka-"),
+ "kafka_resource",
+ "kafka_resource_derived",
+ ImmutableSet.of(),
+ "tableflow_resource",
+ ImmutableSet.of(),
+ "connect_resource",
+ ImmutableSet.of(),
+ "client_connector_resource",
+ ImmutableSet.of(),
+ "ksql_resource",
+ ImmutableSet.of(),
+ "schema_registry_resource",
+ ImmutableSet.of(),
+ "fcp_resource",
+ "resource_display_name_test_lookup",
+ mockLookupProvider
+ );
+
+ TransformSpec transformSpec = new TransformSpec(null, ImmutableList.of(transform));
+ InputRowParser> parser = transformSpec.decorate(PARSER);
+ Map rowData = ImmutableMap.builder()
+ .put("metric_name", "kafka-producer-metrics")
+ .put("kafka_resource", "lkc-abc123")
+ .build();
+ InputRow row = parser.parseBatch(rowData).get(0);
+ Assert.assertNotNull(row);
+ Assert.assertNull(row.getRaw("resource_name"));
+ }
+
+}
diff --git a/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/ExtractTransformTest.java b/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/ExtractTransformTest.java
new file mode 100644
index 000000000000..2ca5390e76b9
--- /dev/null
+++ b/extensions-contrib/confluent-extensions/src/test/java/io/confluent/druid/transform/ExtractTransformTest.java
@@ -0,0 +1,161 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ */
+
+package io.confluent.druid.transform;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import io.confluent.druid.ConfluentExtensionsModule;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.data.input.impl.MapInputRowParser;
+import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.segment.TestHelper;
+import org.apache.druid.segment.transform.TransformSpec;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class ExtractTransformTest
+{
+
+ private static final MapInputRowParser PARSER = new MapInputRowParser(
+ new TimeAndDimsParseSpec(
+ new TimestampSpec("t", "auto", DateTimes.of("2020-01-01")),
+ new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("topic", "tenant")))
+ )
+ );
+
+ private static final Map ROW1 = ImmutableMap.builder()
+ .put("topic", "lkc-abc123_mytopic")
+ .build();
+
+ private static final Map ROW2 = ImmutableMap.builder()
+ .put("tenant", "lkc-xyz789")
+ .put("tenant_topic", "topic0")
+ .put("topic", "lkc-abc123_mytopic")
+ .build();
+
+ private static final Map ROW3 = ImmutableMap.builder()
+ .put("topic", "invalid-topic")
+ .build();
+
+ private static final Map ROW4 = ImmutableMap.builder()
+ .build();
+
+
+ @Test
+ public void testExtraction()
+ {
+ final TransformSpec transformSpec = new TransformSpec(
+ null,
+ ImmutableList.of(
+ new ExtractTenantTransform("tenant", "topic"),
+ new ExtractTenantTopicTransform("tenant_topic", "topic")
+ )
+ );
+
+ final InputRowParser> parser = transformSpec.decorate(PARSER);
+ final InputRow row = parser.parseBatch(ROW1).get(0);
+
+ Assert.assertNotNull(row);
+ Assert.assertEquals(ImmutableList.of("topic", "tenant"), row.getDimensions());
+ Assert.assertEquals(ImmutableList.of("lkc-abc123"), row.getDimension("tenant"));
+ Assert.assertEquals(ImmutableList.of("mytopic"), row.getDimension("tenant_topic"));
+ }
+
+ @Test
+ public void testInternal()
+ {
+ Assert.assertEquals(null, TenantUtils.extractTenantTopic("__consumer_offsets"));
+ Assert.assertEquals(null, TenantUtils.extractTenant("__consumer_offsets"));
+ Assert.assertEquals(null, TenantUtils.extractTenantTopic("other.topic"));
+ Assert.assertEquals(null, TenantUtils.extractTenant("other.topic"));
+ }
+
+ @Test
+ public void testPreserveExistingFields()
+ {
+ final TransformSpec transformSpec = new TransformSpec(
+ null,
+ ImmutableList.of(
+ new ExtractTenantTransform("tenant", "topic"),
+ new ExtractTenantTopicTransform("tenant_topic", "topic")
+ )
+ );
+
+ final InputRowParser> parser = transformSpec.decorate(PARSER);
+ final InputRow row = parser.parseBatch(ROW2).get(0);
+
+ Assert.assertNotNull(row);
+ Assert.assertEquals(ImmutableList.of("topic", "tenant"), row.getDimensions());
+ Assert.assertEquals(ImmutableList.of("lkc-xyz789"), row.getDimension("tenant"));
+ Assert.assertEquals(ImmutableList.of("topic0"), row.getDimension("tenant_topic"));
+ }
+
+ @Test
+ public void testInvalidTopics()
+ {
+ final TransformSpec transformSpec = new TransformSpec(
+ null,
+ ImmutableList.of(
+ new ExtractTenantTransform("tenant", "topic"),
+ new ExtractTenantTopicTransform("tenant_topic", "topic")
+ )
+ );
+
+ final InputRowParser> parser = transformSpec.decorate(PARSER);
+ final InputRow row = parser.parseBatch(ROW3).get(0);
+
+ Assert.assertNotNull(row);
+ Assert.assertEquals(ImmutableList.of("topic", "tenant"), row.getDimensions());
+ Assert.assertNull(row.getRaw("tenant"));
+ Assert.assertNull(row.getRaw("tenant_topic"));
+ }
+
+ @Test
+ public void testNullTopic()
+ {
+ final TransformSpec transformSpec = new TransformSpec(
+ null,
+ ImmutableList.of(
+ new ExtractTenantTransform("tenant", "topic"),
+ new ExtractTenantTopicTransform("tenant_topic", "topic")
+ )
+ );
+
+ final InputRowParser> parser = transformSpec.decorate(PARSER);
+ final InputRow row = parser.parseBatch(ROW4).get(0);
+
+ Assert.assertNotNull(row);
+ Assert.assertEquals(ImmutableList.of("topic", "tenant"), row.getDimensions());
+ Assert.assertNull(row.getRaw("tenant"));
+ Assert.assertNull(row.getRaw("tenant_topic"));
+ }
+
+ @Test
+ public void testSerde() throws Exception
+ {
+ final TransformSpec transformSpec = new TransformSpec(
+ null,
+ ImmutableList.of(
+ new ExtractTenantTopicTransform("tenant_topic", "topic"),
+ new ExtractTenantTransform("tenant", "topic")
+ )
+ );
+
+ final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
+ jsonMapper.registerModules(new ConfluentExtensionsModule().getJacksonModules());
+
+ Assert.assertEquals(
+ transformSpec,
+ jsonMapper.readValue(jsonMapper.writeValueAsString(transformSpec), TransformSpec.class)
+ );
+ }
+}
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
index c5faefc65ca6..4e4ec8686830 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
@@ -28,6 +28,7 @@
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.timeseries.DefaultTimeseriesQueryMetrics;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
@@ -39,6 +40,7 @@
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime;
+import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
@@ -100,8 +102,10 @@ public void testTimeseriesWithDistinctCountAgg() throws Exception
)
.build();
+ ResponseContext responseContext = ResponseContext.createEmpty();
final Iterable> results =
- engine.process(query, new IncrementalIndexStorageAdapter(index), new DefaultTimeseriesQueryMetrics()).toList();
+ engine.process(query, new IncrementalIndexStorageAdapter(index), new DefaultTimeseriesQueryMetrics(), responseContext).toList();
+ Assert.assertEquals(3L, (long) responseContext.getRowScanCount());
List> expectedResults = Collections.singletonList(
new Result<>(
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
index c61a793ff837..0544c20a49cb 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
@@ -29,6 +29,7 @@
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.query.topn.TopNQueryEngine;
@@ -41,6 +42,7 @@
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime;
import org.junit.After;
+import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -128,9 +130,10 @@ public void testTopNWithDistinctCountAgg() throws Exception
new DistinctCountAggregatorFactory("UV", visitor_id, null)
)
.build();
-
+ ResponseContext responseContext = ResponseContext.createEmpty();
final Iterable> results =
- engine.query(query, new IncrementalIndexStorageAdapter(index), null).toList();
+ engine.query(query, new IncrementalIndexStorageAdapter(index), null, responseContext).toList();
+ Assert.assertEquals(3L, (long) responseContext.getRowScanCount());
List> expectedResults = Collections.singletonList(
new Result<>(
diff --git a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json
index 849162c5ff33..54990a8d88dc 100644
--- a/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json
+++ b/extensions-contrib/dropwizard-emitter/src/main/resources/defaultMetricDimensions.json
@@ -7,6 +7,14 @@
"type": "timer",
"timeUnit": "MILLISECONDS"
},
+ "query/planningTime": {
+ "dimensions": [
+ "dataSource",
+ "type"
+ ],
+ "type": "timer",
+ "timeUnit": "MILLISECONDS"
+ },
"query/node/time": {
"dimensions": [
"server"
@@ -155,6 +163,12 @@
],
"type": "counter"
},
+ "ingest/events/filtered": {
+ "dimensions": [
+ "dataSource"
+ ],
+ "type": "counter"
+ },
"ingest/rows/output": {
"dimensions": [
"dataSource"
@@ -539,4 +553,4 @@
],
"type": "gauge"
}
-}
\ No newline at end of file
+}
diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml
index 673ed0abaa34..0ce47c6b3727 100644
--- a/extensions-contrib/druid-deltalake-extensions/pom.xml
+++ b/extensions-contrib/druid-deltalake-extensions/pom.xml
@@ -95,7 +95,7 @@
com.fasterxml.jackson.core
jackson-databind
- 2.12.7.1
+ ${jackson.version}
it.unimi.dsi
diff --git a/extensions-contrib/graphite-emitter/src/main/resources/defaultWhiteListMap.json b/extensions-contrib/graphite-emitter/src/main/resources/defaultWhiteListMap.json
index 44bd5ef8db9e..1a6bfbe6346a 100644
--- a/extensions-contrib/graphite-emitter/src/main/resources/defaultWhiteListMap.json
+++ b/extensions-contrib/graphite-emitter/src/main/resources/defaultWhiteListMap.json
@@ -33,6 +33,10 @@
"dataSource",
"type"
],
+ "query/planningTime": [
+ "dataSource",
+ "type"
+ ],
"query/wait/time": [
"dataSource",
"type"
diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml
index 232d11c9587c..4ca08c715bff 100644
--- a/extensions-contrib/kafka-emitter/pom.xml
+++ b/extensions-contrib/kafka-emitter/pom.xml
@@ -38,6 +38,16 @@
org.apache.kafka
kafka-clients
${apache.kafka.version}
+
+
+ org.lz4
+ lz4-java
+
+
+
+
+ at.yawk.lz4
+ lz4-java
org.apache.druid
@@ -91,7 +101,11 @@
slf4j-api
provided
-
+
+ joda-time
+ joda-time
+ provided
+
junit
junit
@@ -126,15 +140,60 @@
hamcrest-core
test
+
+ com.google.protobuf
+ protobuf-java
+
+
+ com.google.protobuf
+ protobuf-java-util
+
-
+
+ kr.motd.maven
+ os-maven-plugin
+ 1.7.0
+
+
+ initialize
+
+ detect
+
+
+
+
+
+ org.xolstice.maven.plugins
+ protobuf-maven-plugin
+ 0.6.1
+
+
+
+ compile
+
+
+
+
+ com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}
+
+
org.owasp
- dependency-check-maven
+ dependency-check-maven
+
+ true
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
- true
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
diff --git a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java
index a0cb1a9afe1e..f53044917e6d 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java
@@ -19,12 +19,13 @@
package org.apache.druid.emitter.kafka;
-import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.Timestamp;
+import com.google.protobuf.util.Timestamps;
import org.apache.druid.emitter.kafka.KafkaEmitterConfig.EventType;
+import org.apache.druid.emitter.proto.DruidSegmentEvent;
import org.apache.druid.java.util.common.MemoryBoundLinkedBlockingQueue;
-import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.core.Emitter;
@@ -39,6 +40,7 @@
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.StringSerializer;
import java.util.Properties;
@@ -62,12 +64,12 @@ public class KafkaEmitter implements Emitter
private final AtomicLong invalidLost;
private final KafkaEmitterConfig config;
- private final Producer producer;
+ private final Producer producer;
private final ObjectMapper jsonMapper;
- private final MemoryBoundLinkedBlockingQueue metricQueue;
- private final MemoryBoundLinkedBlockingQueue alertQueue;
- private final MemoryBoundLinkedBlockingQueue requestQueue;
- private final MemoryBoundLinkedBlockingQueue segmentMetadataQueue;
+ private final MemoryBoundLinkedBlockingQueue metricQueue;
+ private final MemoryBoundLinkedBlockingQueue alertQueue;
+ private final MemoryBoundLinkedBlockingQueue requestQueue;
+ private final MemoryBoundLinkedBlockingQueue segmentMetadataQueue;
private final ScheduledExecutorService scheduler;
protected int sendInterval = DEFAULT_SEND_INTERVAL_SECONDS;
@@ -105,7 +107,7 @@ private Callback setProducerCallback(AtomicLong lostCounter)
}
@VisibleForTesting
- protected Producer setKafkaProducer()
+ protected Producer setKafkaProducer()
{
ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
try {
@@ -114,7 +116,7 @@ protected Producer setKafkaProducer()
Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.getBootstrapServers());
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
- props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+ props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
props.put(ProducerConfig.RETRIES_CONFIG, DEFAULT_RETRIES);
props.putAll(config.getKafkaProducerConfig());
props.putAll(config.getKafkaProducerSecrets().getConfig());
@@ -174,9 +176,9 @@ private void sendSegmentMetadataToKafka()
sendToKafka(config.getSegmentMetadataTopic(), segmentMetadataQueue, setProducerCallback(segmentMetadataLost));
}
- private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue recordQueue, Callback callback)
+ private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue recordQueue, Callback callback)
{
- MemoryBoundLinkedBlockingQueue.ObjectContainer objectToSend;
+ MemoryBoundLinkedBlockingQueue.ObjectContainer objectToSend;
try {
while (true) {
objectToSend = recordQueue.take();
@@ -200,11 +202,10 @@ public void emit(final Event event)
EventMap map = event.toMap();
map = addExtraDimensionsToEvent(map);
- String resultJson = jsonMapper.writeValueAsString(map);
-
- MemoryBoundLinkedBlockingQueue.ObjectContainer objectContainer = new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(
- resultJson,
- StringUtils.toUtf8(resultJson).length
+ byte[] resultBytes = jsonMapper.writeValueAsBytes(map);
+ MemoryBoundLinkedBlockingQueue.ObjectContainer objectContainer = new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(
+ resultBytes,
+ resultBytes.length
);
Set eventTypes = config.getEventTypes();
@@ -221,14 +222,32 @@ public void emit(final Event event)
requestLost.incrementAndGet();
}
} else if (event instanceof SegmentMetadataEvent) {
- if (!eventTypes.contains(EventType.SEGMENT_METADATA) || !segmentMetadataQueue.offer(objectContainer)) {
+ if (!eventTypes.contains(EventType.SEGMENT_METADATA)) {
segmentMetadataLost.incrementAndGet();
+ } else {
+ switch (config.getSegmentMetadataTopicFormat()) {
+ case PROTOBUF:
+ resultBytes = convertMetadataEventToProto((SegmentMetadataEvent) event, segmentMetadataLost);
+ objectContainer = new MemoryBoundLinkedBlockingQueue.ObjectContainer<>(
+ resultBytes,
+ resultBytes.length
+ );
+ break;
+ case JSON:
+ // Do Nothing. We already have the JSON object stored in objectContainer
+ break;
+ default:
+ throw new UnsupportedOperationException("segmentMetadata.topic.format has an invalid value " + config.getSegmentMetadataTopicFormat().toString());
+ }
+ if (!segmentMetadataQueue.offer(objectContainer)) {
+ segmentMetadataLost.incrementAndGet();
+ }
}
} else {
invalidLost.incrementAndGet();
}
}
- catch (JsonProcessingException e) {
+ catch (Exception e) {
invalidLost.incrementAndGet();
log.warn(e, "Exception while serializing event");
}
@@ -250,6 +269,32 @@ private EventMap addExtraDimensionsToEvent(EventMap map)
return map;
}
+ private byte[] convertMetadataEventToProto(SegmentMetadataEvent event, AtomicLong segmentMetadataLost)
+ {
+ try {
+ Timestamp createdTimeTs = Timestamps.fromMillis(event.getCreatedTime().getMillis());
+ Timestamp startTimeTs = Timestamps.fromMillis(event.getStartTime().getMillis());
+ Timestamp endTimeTs = Timestamps.fromMillis(event.getEndTime().getMillis());
+
+ DruidSegmentEvent.Builder druidSegmentEventBuilder = DruidSegmentEvent.newBuilder()
+ .setDataSource(event.getDataSource())
+ .setCreatedTime(createdTimeTs)
+ .setStartTime(startTimeTs)
+ .setEndTime(endTimeTs)
+ .setVersion(event.getVersion())
+ .setIsCompacted(event.isCompacted());
+ if (config.getClusterName() != null) {
+ druidSegmentEventBuilder.setClusterName(config.getClusterName());
+ }
+ DruidSegmentEvent druidSegmentEvent = druidSegmentEventBuilder.build();
+ return druidSegmentEvent.toByteArray();
+ }
+ catch (Exception e) {
+ log.warn(e, "Exception while serializing SegmentMetadataEvent");
+ throw e;
+ }
+ }
+
@Override
public void flush()
{
diff --git a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterConfig.java b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterConfig.java
index 8ae253290350..159fa6ea562f 100644
--- a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterConfig.java
+++ b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterConfig.java
@@ -59,6 +59,25 @@ public static EventType fromString(String name)
}
}
+ public enum SegmentMetadataTopicFormat
+ {
+ JSON,
+ PROTOBUF;
+
+ @JsonValue
+ @Override
+ public String toString()
+ {
+ return StringUtils.toLowerCase(this.name());
+ }
+
+ @JsonCreator
+ public static SegmentMetadataTopicFormat fromString(String name)
+ {
+ return valueOf(StringUtils.toUpperCase(name));
+ }
+ }
+
public static final Set DEFAULT_EVENT_TYPES = ImmutableSet.of(EventType.ALERTS, EventType.METRICS);
@JsonProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
private final String bootstrapServers;
@@ -72,7 +91,9 @@ public static EventType fromString(String name)
private final String requestTopic;
@Nullable @JsonProperty("segmentMetadata.topic")
private final String segmentMetadataTopic;
- @Nullable @JsonProperty
+ @Nullable @JsonProperty("segmentMetadata.topic.format")
+ private final SegmentMetadataTopicFormat segmentMetadataTopicFormat;
+ @JsonProperty
private final String clusterName;
@Nullable @JsonProperty("extra.dimensions")
private final Map extraDimensions;
@@ -89,7 +110,8 @@ public KafkaEmitterConfig(
@Nullable @JsonProperty("alert.topic") String alertTopic,
@Nullable @JsonProperty("request.topic") String requestTopic,
@Nullable @JsonProperty("segmentMetadata.topic") String segmentMetadataTopic,
- @Nullable @JsonProperty("clusterName") String clusterName,
+ @Nullable @JsonProperty("segmentMetadata.topic.format") SegmentMetadataTopicFormat segmentMetadataTopicFormat,
+ @JsonProperty("clusterName") String clusterName,
@Nullable @JsonProperty("extra.dimensions") Map extraDimensions,
@JsonProperty("producer.config") @Nullable Map kafkaProducerConfig,
@JsonProperty("producer.hiddenProperties") @Nullable DynamicConfigProvider kafkaProducerSecrets
@@ -134,6 +156,7 @@ public KafkaEmitterConfig(
this.alertTopic = alertTopic;
this.requestTopic = requestTopic;
this.segmentMetadataTopic = segmentMetadataTopic;
+ this.segmentMetadataTopicFormat = segmentMetadataTopicFormat == null ? SegmentMetadataTopicFormat.JSON : segmentMetadataTopicFormat;
this.clusterName = clusterName;
this.extraDimensions = extraDimensions;
this.kafkaProducerConfig = kafkaProducerConfig == null ? ImmutableMap.of() : kafkaProducerConfig;
@@ -205,6 +228,12 @@ public String getSegmentMetadataTopic()
return segmentMetadataTopic;
}
+ @JsonProperty
+ public SegmentMetadataTopicFormat getSegmentMetadataTopicFormat()
+ {
+ return segmentMetadataTopicFormat;
+ }
+
@JsonProperty
public Map getKafkaProducerConfig()
{
@@ -253,6 +282,10 @@ public boolean equals(Object o)
return false;
}
+ if (getSegmentMetadataTopicFormat() != null ? !getSegmentMetadataTopicFormat().equals(that.getSegmentMetadataTopicFormat()) : that.getSegmentMetadataTopicFormat() != null) {
+ return false;
+ }
+
if (getClusterName() != null ? !getClusterName().equals(that.getClusterName()) : that.getClusterName() != null) {
return false;
}
@@ -271,6 +304,7 @@ public int hashCode()
result = 31 * result + (getAlertTopic() != null ? getAlertTopic().hashCode() : 0);
result = 31 * result + (getRequestTopic() != null ? getRequestTopic().hashCode() : 0);
result = 31 * result + (getSegmentMetadataTopic() != null ? getSegmentMetadataTopic().hashCode() : 0);
+ result = 31 * result + (getSegmentMetadataTopicFormat() != null ? getSegmentMetadataTopicFormat().hashCode() : 0);
result = 31 * result + (getClusterName() != null ? getClusterName().hashCode() : 0);
result = 31 * result + (getExtraDimensions() != null ? getExtraDimensions().hashCode() : 0);
result = 31 * result + getKafkaProducerConfig().hashCode();
@@ -288,6 +322,7 @@ public String toString()
", alert.topic='" + alertTopic + '\'' +
", request.topic='" + requestTopic + '\'' +
", segmentMetadata.topic='" + segmentMetadataTopic + '\'' +
+ ", segmentMetadata.topic.format='" + segmentMetadataTopicFormat + '\'' +
", clusterName='" + clusterName + '\'' +
", extra.dimensions='" + extraDimensions + '\'' +
", producer.config=" + kafkaProducerConfig + '\'' +
diff --git a/extensions-contrib/kafka-emitter/src/main/proto/DruidSegmentEvent.proto b/extensions-contrib/kafka-emitter/src/main/proto/DruidSegmentEvent.proto
new file mode 100644
index 000000000000..810ab64f92dd
--- /dev/null
+++ b/extensions-contrib/kafka-emitter/src/main/proto/DruidSegmentEvent.proto
@@ -0,0 +1,30 @@
+syntax = "proto3";
+import "google/protobuf/timestamp.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.druid.emitter.proto";
+option java_outer_classname = "DruidSegmentEventMessage";
+
+/* Druid segment Event used by Druid to publish first level segment information.
+ * The message will be consumed by segment processing app. */
+message DruidSegmentEvent {
+ string dataSource = 1;
+
+ // When this event was created
+ google.protobuf.Timestamp createdTime = 2;
+
+ // Start time of the segment
+ google.protobuf.Timestamp startTime = 3;
+
+ // End time of the segment
+ google.protobuf.Timestamp endTime = 4;
+
+ // Segment version
+ string version = 5;
+
+ // Cluster name
+ string clusterName = 6;
+
+ // Is the segment compacted or not
+ bool isCompacted = 7;
+}
diff --git a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java
index 24b743a06611..69fb9b2e3d94 100644
--- a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java
+++ b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java
@@ -31,7 +31,6 @@
import org.hamcrest.MatcherAssert;
import org.junit.Assert;
import org.junit.Test;
-
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
@@ -54,6 +53,7 @@ public void testSerDeserKafkaEmitterConfig() throws IOException
"alertTest",
"requestTest",
"metadataTest",
+ null,
"clusterNameTest",
ImmutableMap.of("env", "preProd"),
ImmutableMap.builder()
@@ -76,6 +76,7 @@ public void testSerDeserKafkaEmitterConfigNullRequestTopic() throws IOException
"alertTest",
null,
"metadataTest",
+ null,
"clusterNameTest",
null,
ImmutableMap.builder()
@@ -100,6 +101,7 @@ public void testSerDeserKafkaEmitterConfigNullMetricsTopic() throws IOException
null,
null,
"metadataTest",
+ null,
"clusterNameTest",
null,
ImmutableMap.builder()
@@ -123,6 +125,7 @@ public void testSerDeNotRequiredKafkaProducerConfigOrKafkaSecretProducer() throw
null,
"metadataTest",
null,
+ null,
ImmutableMap.of("env", "preProd"),
null,
null
@@ -179,6 +182,7 @@ public void testNullBootstrapServers()
null,
null,
null,
+ null,
null
)
),
@@ -203,6 +207,7 @@ public void testNullMetricTopic()
null,
null,
null,
+ null,
null
)
),
@@ -227,6 +232,7 @@ public void testNullAlertTopic()
null,
null,
null,
+ null,
null
)
),
@@ -253,6 +259,7 @@ public void testNullRequestTopic()
null,
null,
null,
+ null,
null
)
),
@@ -279,6 +286,7 @@ public void testNullSegmentMetadataTopic()
null,
null,
null,
+ null,
null
)
),
diff --git a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
index a9f5e14fbeaa..ae16a13f45b9 100644
--- a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
+++ b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterTest.java
@@ -43,7 +43,6 @@
import org.junit.Before;
import org.junit.Test;
-import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -62,7 +61,7 @@
public class KafkaEmitterTest
{
- private KafkaProducer producer;
+ private KafkaProducer producer;
@Before
public void setup()
@@ -146,6 +145,7 @@ public void testServiceMetricEvents() throws InterruptedException, JsonProcessin
"alerts",
"requests",
"segments",
+ null,
"clusterName",
ImmutableMap.of("clusterId", "cluster-101"),
null,
@@ -192,6 +192,7 @@ public void testAllEvents() throws InterruptedException, JsonProcessingException
"requests",
"segments",
null,
+ "clusterName",
ImmutableMap.of("clusterId", "cluster-101", "env", "staging"),
null,
null
@@ -241,6 +242,7 @@ public void testDefaultEvents() throws InterruptedException, JsonProcessingExcep
"alerts",
"requests",
"segment_metadata",
+ null,
"clusterName",
null,
null,
@@ -289,6 +291,7 @@ public void testAlertsPlusUnsubscribedEvents() throws InterruptedException, Json
"alerts",
"requests",
"segment_metadata",
+ null,
"clusterName",
null,
null,
@@ -349,6 +352,7 @@ public void testAllEventsWithCommonTopic() throws InterruptedException, JsonProc
null,
null,
null,
+ null,
null
);
@@ -397,6 +401,7 @@ public void testUnknownEvents() throws InterruptedException, JsonProcessingExcep
"topic",
null,
null,
+ null,
"cluster-102",
ImmutableMap.of("clusterName", "cluster-101", "env", "staging"), // clusterName again, extraDimensions should take precedence
null,
@@ -447,7 +452,7 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
final ImmutableMap extraDimensions = ImmutableMap.of("clusterId", "cluster-101");
final Map> feedToAllEventsBeforeDrop = trackExpectedEventsPerFeed(
inputEvents,
- null,
+ "clusterName",
extraDimensions
);
@@ -473,7 +478,7 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
int totalBufferSize = 0;
for (final List feedEvents : feedToAllEventsBeforeDrop.values()) {
for (int idx = 0; idx < feedEvents.size() - bufferEventsDrop; idx++) {
- totalBufferSize += MAPPER.writeValueAsString(feedEvents.get(idx)).getBytes(StandardCharsets.UTF_8).length;
+ totalBufferSize += MAPPER.writeValueAsBytes(feedEvents.get(idx)).length;
}
}
@@ -490,7 +495,8 @@ public void testDropEventsWhenQueueFull() throws JsonProcessingException, Interr
"alerts",
"requests",
"segments",
- null,
+ KafkaEmitterConfig.SegmentMetadataTopicFormat.PROTOBUF,
+ "clusterName",
extraDimensions,
ImmutableMap.of(ProducerConfig.BUFFER_MEMORY_CONFIG, String.valueOf(totalBufferSize)),
null
@@ -522,7 +528,7 @@ private KafkaEmitter initKafkaEmitter(
)
{
@Override
- protected Producer setKafkaProducer()
+ protected Producer setKafkaProducer()
{
// override send interval to 1 second
sendInterval = 1;
@@ -564,9 +570,9 @@ private Map> trackActualEventsPerFeed(
// A concurrent hashmap because the producer callback can trigger concurrently and can override the map initialization
final ConcurrentHashMap> feedToActualEvents = new ConcurrentHashMap<>();
when(producer.send(any(), any())).then((invocation) -> {
- final ProducerRecord, ?> producerRecord = invocation.getArgument(0);
- final String value = String.valueOf(producerRecord.value());
- final EventMap eventMap = MAPPER.readValue(value, EventMap.class);
+ final ProducerRecord, byte[]> producerRecord = invocation.getArgument(0);
+ final EventMap eventMap = MAPPER.readValue(producerRecord.value(), EventMap.class);
+
feedToActualEvents.computeIfAbsent(
(String) eventMap.get("feed"), k -> new ArrayList<>()
).add(eventMap);
diff --git a/extensions-contrib/kubernetes-overlord-extensions/pom.xml b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
index 0936f4e6066b..a32e771ee8f8 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/pom.xml
+++ b/extensions-contrib/kubernetes-overlord-extensions/pom.xml
@@ -34,16 +34,15 @@
-
-
-
- org.yaml
- snakeyaml
- 1.33
-
-
+
+
+
+ com.squareup.okio
+ okio
+ 3.6.0
+
+
@@ -115,7 +114,7 @@
io.fabric8
kubernetes-model-core
- 6.7.2
+ 6.8.0
jakarta.validation
@@ -125,19 +124,39 @@
io.fabric8
kubernetes-model-batch
- 6.7.2
+ 6.8.0
io.fabric8
kubernetes-client-api
- 6.7.2
+ 6.8.0
io.fabric8
kubernetes-client
- 6.7.2
+ 6.8.0
+
+
+ com.squareup.okhttp3
+ okhttp
+
+
+ com.squareup.okhttp3
+ logging-interceptor
+
+
runtime
+
+ com.squareup.okhttp3
+ logging-interceptor
+ 4.12.0
+
+
+ com.squareup.okhttp3
+ okhttp
+ 4.12.0
+
@@ -145,6 +164,12 @@
junit
test
+
+ com.squareup.okhttp3
+ mockwebserver
+ 4.12.0
+ test
+
org.easymock
easymock
@@ -160,6 +185,16 @@
io.fabric8
mockwebserver
0.2.2
+
+
+ com.squareup.okhttp3
+ okhttp
+
+
+ com.squareup.okhttp3
+ mockwebserver
+
+
test
@@ -264,6 +299,17 @@
true
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ com.squareup.okhttp3:okhttp:jar:4.12.0
+ com.squareup.okhttp3:logging-interceptor:jar:4.12.0
+
+
+
diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
index 241b4d9fc68f..4f4f06592c85 100644
--- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
+++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/taskadapter/DruidPeonClientIntegrationTest.java
@@ -52,7 +52,6 @@
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
-import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
@@ -146,15 +145,10 @@ public void testDeployingSomethingToKind(@TempDir Path tempDir) throws Exception
List expectedLogs = IntStream.range(1, 1001).boxed().collect(Collectors.toList());
List actualLogs = new ArrayList<>();
Thread thread = new Thread(() -> {
- try {
- actualLogs.addAll(IOUtils.readLines(peonLogs, "UTF-8")
- .stream()
- .map(Integer::parseInt)
- .collect(Collectors.toList()));
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
+ actualLogs.addAll(IOUtils.readLines(peonLogs, "UTF-8")
+ .stream()
+ .map(Integer::parseInt)
+ .collect(Collectors.toList()));
});
thread.start();
diff --git a/extensions-contrib/opencensus-extensions/pom.xml b/extensions-contrib/opencensus-extensions/pom.xml
new file mode 100644
index 000000000000..990c6f02db90
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/pom.xml
@@ -0,0 +1,197 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.druid.extensions.contrib
+ druid-opencensus-extensions
+ druid-opencensus-extensions
+ druid-opencensus-extensions
+
+
+ druid
+ org.apache.druid
+ 30.0.1
+ ../../pom.xml
+
+
+
+
+ io.opencensus
+ opencensus-proto
+ 0.2.0
+
+
+
+ com.google.guava
+ guava
+
+
+ io.grpc
+ grpc-protobuf
+
+
+
+
+ io.grpc
+ grpc-protobuf
+ 1.60.0
+
+
+ org.apache.druid
+ druid-indexing-service
+ ${project.parent.version}
+ provided
+
+
+ io.opentelemetry.proto
+ opentelemetry-proto
+
+
+ org.apache.druid.extensions.contrib
+ druid-opentelemetry-extensions
+ ${project.parent.version}
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ provided
+
+
+ com.google.protobuf
+ protobuf-java
+
+
+ com.google.guava
+ guava
+ provided
+
+
+ com.google.inject
+ guice
+ provided
+
+
+ com.google.code.findbugs
+ jsr305
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ provided
+
+
+
+ junit
+ junit
+ test
+
+
+ org.apache.druid.extensions
+ druid-kafka-indexing-service
+ ${project.parent.version}
+ test
+
+
+ org.apache.kafka
+ kafka-clients
+ ${apache.kafka.version}
+ test
+
+
+ org.lz4
+ lz4-java
+
+
+
+
+ at.yawk.lz4
+ lz4-java
+ test
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ provided
+
+
+ org.apache.curator
+ curator-client
+ 5.4.0
+ provided
+
+
+
+ org.openjdk.jmh
+ jmh-core
+ 1.27
+ test
+
+
+ org.openjdk.jmh
+ jmh-generator-annprocess
+ 1.27
+ test
+
+
+ org.mockito
+ mockito-all
+ 1.9.5
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ io.grpc:grpc-protobuf
+
+ io.grpc:grpc-netty-shaded
+ com.google.guava:guava
+
+ at.yawk.lz4:lz4-java
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-resources-plugin
+ 3.0.2
+
+
+ desc
+
+
+
+
+
+
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/KafkaUtils.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/KafkaUtils.java
new file mode 100644
index 000000000000..f12589e5adbe
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/KafkaUtils.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.data.input;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.util.Objects;
+
+public class KafkaUtils
+{
+ /**
+ * Creates a MethodHandle that – when invoked on a KafkaRecordEntity - returns the given header value
+ * for the underlying KafkaRecordEntity
+ *
+ * The method handle is roughly equivalent to the following function
+ *
+ * (KafkaRecordEntity input) -> {
+ * Header h = input.getRecord().headers().lastHeader(header)
+ * if (h != null) {
+ * return h.value();
+ * } else {
+ * return null;
+ * }
+ * }
+ *
+ * Since KafkaRecordEntity only exists in the kafka-indexing-service plugin classloader,
+ * we need to look up the relevant classes in the classloader where the InputEntity was instantiated.
+ *
+ * The handle returned by this method should be cached for the classloader it was invoked with.
+ *
+ * If the lookup fails for whatever reason, the method handle will always return null;
+ *
+ * @param classLoader the kafka-indexing-service classloader
+ * @param header the header value to look up
+ * @return a MethodHandle
+ */
+ public static MethodHandle lookupGetHeaderMethod(ClassLoader classLoader, String header)
+ {
+ try {
+ Class entityType = Class.forName("org.apache.druid.data.input.kafka.KafkaRecordEntity", true, classLoader);
+ Class recordType = Class.forName("org.apache.kafka.clients.consumer.ConsumerRecord", true, classLoader);
+ Class headersType = Class.forName("org.apache.kafka.common.header.Headers", true, classLoader);
+ Class headerType = Class.forName("org.apache.kafka.common.header.Header", true, classLoader);
+
+ final MethodHandles.Lookup lookup = MethodHandles.lookup();
+ MethodHandle nonNullTest = lookup.findStatic(Objects.class, "nonNull",
+ MethodType.methodType(boolean.class, Object.class)
+ ).asType(MethodType.methodType(boolean.class, headerType));
+
+ final MethodHandle getRecordMethod = lookup.findVirtual(
+ entityType,
+ "getRecord",
+ MethodType.methodType(recordType)
+ );
+ final MethodHandle headersMethod = lookup.findVirtual(recordType, "headers", MethodType.methodType(headersType));
+ final MethodHandle lastHeaderMethod = lookup.findVirtual(
+ headersType,
+ "lastHeader",
+ MethodType.methodType(headerType, String.class)
+ );
+ final MethodHandle valueMethod = lookup.findVirtual(headerType, "value", MethodType.methodType(byte[].class));
+
+ return MethodHandles.filterReturnValue(
+ MethodHandles.filterReturnValue(
+ MethodHandles.filterReturnValue(getRecordMethod, headersMethod),
+ MethodHandles.insertArguments(lastHeaderMethod, 1, header)
+ ),
+ // return null byte array if header is not present
+ MethodHandles.guardWithTest(
+ nonNullTest,
+ valueMethod,
+ // match valueMethod signature by dropping the header instance argument
+ MethodHandles.dropArguments(MethodHandles.constant(byte[].class, null), 0, headerType)
+ )
+ );
+ }
+ catch (ReflectiveOperationException e) {
+ // if lookup fails in the classloader where the InputEntity is defined, then the source may not be
+ // the kafka-indexing-service classloader, or method signatures did not match.
+ // In that case we return a method handle always returning null
+ return noopMethodHandle();
+ }
+ }
+
+ static MethodHandle noopMethodHandle()
+ {
+ return MethodHandles.dropArguments(MethodHandles.constant(byte[].class, null), 0, InputEntity.class);
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/HybridProtobufReader.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/HybridProtobufReader.java
new file mode 100644
index 000000000000..83c5c20299aa
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/HybridProtobufReader.java
@@ -0,0 +1,137 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.KafkaUtils;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.opentelemetry.protobuf.OpenTelemetryMetricsProtobufReader;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandle;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class HybridProtobufReader implements InputEntityReader
+{
+ private static final String VERSION_HEADER_KEY = "v";
+ private static final int OPENTELEMETRY_FORMAT_VERSION = 1;
+
+ private final DimensionsSpec dimensionsSpec;
+ private final SettableByteEntity extends ByteEntity> source;
+ private final String metricDimension;
+ private final String valueDimension;
+ private final String metricLabelPrefix;
+ private final String resourceLabelPrefix;
+
+ private volatile MethodHandle getHeaderMethod = null;
+
+ enum ProtobufReader
+ {
+ OPENCENSUS,
+ OPENTELEMETRY
+ }
+
+ public HybridProtobufReader(
+ DimensionsSpec dimensionsSpec,
+ SettableByteEntity extends ByteEntity> source,
+ String metricDimension,
+ String valueDimension,
+ String metricLabelPrefix,
+ String resourceLabelPrefix
+ )
+ {
+ this.dimensionsSpec = dimensionsSpec;
+ this.source = source;
+ this.metricDimension = metricDimension;
+ this.valueDimension = valueDimension;
+ this.metricLabelPrefix = metricLabelPrefix;
+ this.resourceLabelPrefix = resourceLabelPrefix;
+ }
+
+ @Override
+ public CloseableIterator read() throws IOException
+ {
+ return newReader(whichReader()).read();
+ }
+
+ public InputEntityReader newReader(ProtobufReader which)
+ {
+ switch (which) {
+ case OPENTELEMETRY:
+ return new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ source,
+ metricDimension,
+ valueDimension,
+ metricLabelPrefix,
+ resourceLabelPrefix
+ );
+ case OPENCENSUS:
+ default:
+ return new OpenCensusProtobufReader(
+ dimensionsSpec,
+ source,
+ metricDimension,
+ metricLabelPrefix,
+ resourceLabelPrefix
+ );
+ }
+ }
+
+ public ProtobufReader whichReader()
+ {
+ // assume InputEntity is always defined in a single classloader (the kafka-indexing-service classloader)
+ // so we only have to look it up once. To be completely correct we should cache the method based on classloader
+ if (getHeaderMethod == null) {
+ getHeaderMethod = KafkaUtils.lookupGetHeaderMethod(
+ source.getEntity().getClass().getClassLoader(),
+ VERSION_HEADER_KEY
+ );
+ }
+
+ try {
+ byte[] versionHeader = (byte[]) getHeaderMethod.invoke(source.getEntity());
+ if (versionHeader != null) {
+ int version =
+ ByteBuffer.wrap(versionHeader).order(ByteOrder.LITTLE_ENDIAN).getInt();
+ if (version == OPENTELEMETRY_FORMAT_VERSION) {
+ return ProtobufReader.OPENTELEMETRY;
+ }
+ }
+ }
+ catch (Throwable t) {
+ // assume input is opencensus if something went wrong
+ }
+ return ProtobufReader.OPENCENSUS;
+ }
+
+ @Override
+ public CloseableIterator sample() throws IOException
+ {
+ return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java
new file mode 100644
index 000000000000..66a58c0eb28e
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufExtensionsModule.java
@@ -0,0 +1,50 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.Collections;
+import java.util.List;
+
+public class OpenCensusProtobufExtensionsModule implements DruidModule
+{
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.singletonList(
+ new SimpleModule("OpenCensusProtobufInputRowParserModule")
+ .registerSubtypes(
+ new NamedType(OpenCensusProtobufInputRowParser.class, "opencensus-protobuf"),
+ new NamedType(OpenCensusProtobufInputFormat.class, "opencensus-protobuf")
+ )
+ );
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputFormat.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputFormat.java
new file mode 100644
index 000000000000..f06d6bb9deb5
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputFormat.java
@@ -0,0 +1,126 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.data.input.InputEntity;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.util.Objects;
+
+public class OpenCensusProtobufInputFormat implements InputFormat
+{
+ private static final String DEFAULT_METRIC_DIMENSION = "name";
+ private static final String DEFAULT_RESOURCE_PREFIX = "resource.";
+ private static final String DEFAULT_VALUE_DIMENSION = "value";
+
+ private final String metricDimension;
+ private final String valueDimension;
+ private final String metricLabelPrefix;
+ private final String resourceLabelPrefix;
+
+ public OpenCensusProtobufInputFormat(
+ @JsonProperty("metricDimension") String metricDimension,
+ @JsonProperty("valueDimension") @Nullable String valueDimension,
+ @JsonProperty("metricLabelPrefix") String metricLabelPrefix,
+ @JsonProperty("resourceLabelPrefix") String resourceLabelPrefix
+ )
+ {
+ this.metricDimension = metricDimension != null ? metricDimension : DEFAULT_METRIC_DIMENSION;
+ this.valueDimension = valueDimension != null ? valueDimension : DEFAULT_VALUE_DIMENSION;
+ this.metricLabelPrefix = StringUtils.nullToEmptyNonDruidDataString(metricLabelPrefix);
+ this.resourceLabelPrefix = resourceLabelPrefix != null ? resourceLabelPrefix : DEFAULT_RESOURCE_PREFIX;
+ }
+
+ @Override
+ public boolean isSplittable()
+ {
+ return false;
+ }
+
+ @Override
+ public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
+ {
+ // Sampler passes a KafkaRecordEntity directly, while the normal code path wraps the same entity in a
+ // SettableByteEntity
+ SettableByteEntity extends ByteEntity> settableEntity;
+ if (source instanceof SettableByteEntity) {
+ settableEntity = (SettableByteEntity extends ByteEntity>) source;
+ } else {
+ SettableByteEntity wrapper = new SettableByteEntity<>();
+ wrapper.setEntity((ByteEntity) source);
+ settableEntity = wrapper;
+ }
+ return new HybridProtobufReader(
+ inputRowSchema.getDimensionsSpec(),
+ settableEntity,
+ metricDimension,
+ valueDimension,
+ metricLabelPrefix,
+ resourceLabelPrefix
+ );
+ }
+
+ @JsonProperty
+ public String getMetricDimension()
+ {
+ return metricDimension;
+ }
+
+ @JsonProperty
+ public String getMetricLabelPrefix()
+ {
+ return metricLabelPrefix;
+ }
+
+ @JsonProperty
+ public String getResourceLabelPrefix()
+ {
+ return resourceLabelPrefix;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof OpenCensusProtobufInputFormat)) {
+ return false;
+ }
+ OpenCensusProtobufInputFormat that = (OpenCensusProtobufInputFormat) o;
+ return Objects.equals(metricDimension, that.metricDimension)
+ && Objects.equals(metricLabelPrefix, that.metricLabelPrefix)
+ && Objects.equals(resourceLabelPrefix, that.resourceLabelPrefix);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(metricDimension, metricLabelPrefix, resourceLabelPrefix);
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java
new file mode 100644
index 000000000000..e39ca60764b6
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParser.java
@@ -0,0 +1,140 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Strings;
+import org.apache.druid.data.input.ByteBufferInputRowParser;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.ParseSpec;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * use {@link OpenCensusProtobufInputFormat} instead
+ */
+@Deprecated
+public class OpenCensusProtobufInputRowParser implements ByteBufferInputRowParser
+{
+ private static final Logger LOG = new Logger(OpenCensusProtobufInputRowParser.class);
+
+ private static final String DEFAULT_METRIC_DIMENSION = "name";
+ private static final String DEFAULT_RESOURCE_PREFIX = "";
+
+ private final ParseSpec parseSpec;
+
+ private final String metricDimension;
+ private final String metricLabelPrefix;
+ private final String resourceLabelPrefix;
+
+ @JsonCreator
+ public OpenCensusProtobufInputRowParser(
+ @JsonProperty("parseSpec") ParseSpec parseSpec,
+ @JsonProperty("metricDimension") String metricDimension,
+ @JsonProperty("metricLabelPrefix") String metricPrefix,
+ @JsonProperty("resourceLabelPrefix") String resourcePrefix
+ )
+ {
+ this.parseSpec = parseSpec;
+ this.metricDimension = Strings.isNullOrEmpty(metricDimension) ? DEFAULT_METRIC_DIMENSION : metricDimension;
+ this.metricLabelPrefix = StringUtils.nullToEmptyNonDruidDataString(metricPrefix);
+ this.resourceLabelPrefix = resourcePrefix != null ? resourcePrefix : DEFAULT_RESOURCE_PREFIX;
+
+ LOG.info("Creating OpenCensus Protobuf parser with spec:" + parseSpec);
+ }
+
+ @Override
+ public ParseSpec getParseSpec()
+ {
+ return parseSpec;
+ }
+
+ @JsonProperty
+ public String getMetricDimension()
+ {
+ return metricDimension;
+ }
+
+ @JsonProperty
+ public String getMetricLabelPrefix()
+ {
+ return metricLabelPrefix;
+ }
+
+ @JsonProperty
+ public String getResourceLabelPrefix()
+ {
+ return resourceLabelPrefix;
+ }
+
+ @Override
+ public OpenCensusProtobufInputRowParser withParseSpec(ParseSpec parseSpec)
+ {
+ return new OpenCensusProtobufInputRowParser(
+ parseSpec,
+ metricDimension,
+ metricLabelPrefix,
+ resourceLabelPrefix);
+ }
+
+ @Override
+ public List parseBatch(ByteBuffer input)
+ {
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(input));
+ return new OpenCensusProtobufReader(
+ parseSpec.getDimensionsSpec(),
+ settableByteEntity,
+ metricDimension,
+ metricLabelPrefix,
+ resourceLabelPrefix
+ ).readAsList();
+ }
+
+ @Override
+ public boolean equals(final Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof OpenCensusProtobufInputRowParser)) {
+ return false;
+ }
+ final OpenCensusProtobufInputRowParser that = (OpenCensusProtobufInputRowParser) o;
+ return Objects.equals(parseSpec, that.parseSpec) &&
+ Objects.equals(metricDimension, that.metricDimension) &&
+ Objects.equals(metricLabelPrefix, that.metricLabelPrefix) &&
+ Objects.equals(resourceLabelPrefix, that.resourceLabelPrefix);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(parseSpec, metricDimension, metricLabelPrefix, resourceLabelPrefix);
+ }
+
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReader.java b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReader.java
new file mode 100644
index 000000000000..65e2fca9efff
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReader.java
@@ -0,0 +1,234 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.Timestamp;
+import io.opencensus.proto.metrics.v1.LabelKey;
+import io.opencensus.proto.metrics.v1.Metric;
+import io.opencensus.proto.metrics.v1.Point;
+import io.opencensus.proto.metrics.v1.TimeSeries;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+import org.apache.druid.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class OpenCensusProtobufReader implements InputEntityReader
+{
+ private static final String SEPARATOR = "-";
+ private static final String VALUE_COLUMN = "value";
+
+ private final DimensionsSpec dimensionsSpec;
+ private final SettableByteEntity extends ByteEntity> source;
+ private final String metricDimension;
+ private final String metricLabelPrefix;
+ private final String resourceLabelPrefix;
+
+ public OpenCensusProtobufReader(
+ DimensionsSpec dimensionsSpec,
+ SettableByteEntity extends ByteEntity> source,
+ String metricDimension,
+ String metricLabelPrefix,
+ String resourceLabelPrefix
+ )
+ {
+ this.dimensionsSpec = dimensionsSpec;
+ this.source = source;
+ this.metricDimension = metricDimension;
+ this.metricLabelPrefix = metricLabelPrefix;
+ this.resourceLabelPrefix = resourceLabelPrefix;
+ }
+
+ private interface LabelContext
+ {
+ void addRow(long millis, String metricName, Object value);
+ }
+
+ @Override
+ public CloseableIterator read()
+ {
+ Supplier> supplier = Suppliers.memoize(() -> readAsList().iterator());
+ return CloseableIterators.withEmptyBaggage(new Iterator() {
+ @Override
+ public boolean hasNext()
+ {
+ return supplier.get().hasNext();
+ }
+ @Override
+ public InputRow next()
+ {
+ return supplier.get().next();
+ }
+ });
+ }
+
+ List readAsList()
+ {
+ ByteBuffer buffer = source.getEntity().getBuffer();
+ try {
+ List rows = parseMetric(Metric.parseFrom(buffer));
+ return rows;
+ }
+ catch (InvalidProtocolBufferException e) {
+ throw new ParseException(null, e, "Protobuf message could not be parsed");
+ }
+ finally {
+ // Explicitly move the position assuming that all the remaining bytes have been consumed because the protobuf
+ // parser does not update the position itself
+ // In case of an exception, the buffer is moved to the end to avoid parsing it in a loop.
+ buffer.position(buffer.limit());
+ }
+ }
+
+ private List parseMetric(final Metric metric)
+ {
+ // Process metric descriptor labels map keys.
+ List descriptorLabels = new ArrayList<>(metric.getMetricDescriptor().getLabelKeysCount());
+ for (LabelKey s : metric.getMetricDescriptor().getLabelKeysList()) {
+ descriptorLabels.add(this.metricLabelPrefix + s.getKey());
+ }
+
+ // Process resource labels map.
+ Map resourceLabelsMap = CollectionUtils.mapKeys(
+ metric.getResource().getLabelsMap(),
+ key -> this.resourceLabelPrefix + key
+ );
+
+ final List schemaDimensions = dimensionsSpec.getDimensionNames();
+
+ final List dimensions;
+ if (!schemaDimensions.isEmpty()) {
+ dimensions = schemaDimensions;
+ } else {
+ Set recordDimensions = new HashSet<>(descriptorLabels);
+
+ // Add resource map key set to record dimensions.
+ recordDimensions.addAll(resourceLabelsMap.keySet());
+
+ // MetricDimension, VALUE dimensions will not be present in labelKeysList or Metric.Resource
+ // map as they are derived dimensions, which get populated while parsing data for timeSeries
+ // hence add them to recordDimensions.
+ recordDimensions.add(metricDimension);
+ recordDimensions.add(VALUE_COLUMN);
+
+ dimensions = Lists.newArrayList(
+ Sets.difference(recordDimensions, dimensionsSpec.getDimensionExclusions())
+ );
+ }
+
+ final int capacity = resourceLabelsMap.size()
+ + descriptorLabels.size()
+ + 2; // metric name + value columns
+
+ List rows = new ArrayList<>();
+ for (TimeSeries ts : metric.getTimeseriesList()) {
+ final LabelContext labelContext = (millis, metricName, value) -> {
+ // Add common resourceLabels.
+ Map event = Maps.newHashMapWithExpectedSize(capacity);
+ event.putAll(resourceLabelsMap);
+ // Add metric labels
+ for (int i = 0; i < metric.getMetricDescriptor().getLabelKeysCount(); i++) {
+ event.put(descriptorLabels.get(i), ts.getLabelValues(i).getValue());
+ }
+ // add metric name and value
+ event.put(metricDimension, metricName);
+ event.put(VALUE_COLUMN, value);
+ rows.add(new MapBasedInputRow(millis, dimensions, event));
+ };
+
+ for (Point point : ts.getPointsList()) {
+ addPointRows(point, metric, labelContext);
+ }
+ }
+ return rows;
+ }
+
+ private void addPointRows(Point point, Metric metric, LabelContext labelContext)
+ {
+ Timestamp timestamp = point.getTimestamp();
+ long millis = Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos()).toEpochMilli();
+ String metricName = metric.getMetricDescriptor().getName();
+
+ switch (point.getValueCase()) {
+ case DOUBLE_VALUE:
+ labelContext.addRow(millis, metricName, point.getDoubleValue());
+ break;
+
+ case INT64_VALUE:
+ labelContext.addRow(millis, metricName, point.getInt64Value());
+ break;
+
+ case SUMMARY_VALUE:
+ // count
+ labelContext.addRow(
+ millis,
+ metricName + SEPARATOR + "count",
+ point.getSummaryValue().getCount().getValue()
+ );
+ // sum
+ labelContext.addRow(
+ millis,
+ metricName + SEPARATOR + "sum",
+ point.getSummaryValue().getSnapshot().getSum().getValue()
+ );
+ break;
+
+ // TODO : How to handle buckets and percentiles
+ case DISTRIBUTION_VALUE:
+ // count
+ labelContext.addRow(millis, metricName + SEPARATOR + "count", point.getDistributionValue().getCount());
+ // sum
+ labelContext.addRow(
+ millis,
+ metricName + SEPARATOR + "sum",
+ point.getDistributionValue().getSum()
+ );
+ break;
+ default:
+ }
+ }
+
+ @Override
+ public CloseableIterator sample()
+ {
+ return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100755
index 000000000000..54b4400fd2cf
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.druid.data.input.opencensus.protobuf.OpenCensusProtobufExtensionsModule
\ No newline at end of file
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/KafkaUtilsTest.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/KafkaUtilsTest.java
new file mode 100644
index 000000000000..88d918ce09e9
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/KafkaUtilsTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.data.input;
+
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.record.TimestampType;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.lang.invoke.MethodHandle;
+import java.nio.ByteBuffer;
+
+public class KafkaUtilsTest
+{
+
+ private static final byte[] BYTES = ByteBuffer.allocate(Integer.BYTES).putInt(42).array();
+
+ @Test
+ public void testNoopMethodHandle() throws Throwable
+ {
+ Assert.assertNull(
+ KafkaUtils.noopMethodHandle().invoke(new ByteEntity(new byte[]{}))
+ );
+ }
+
+ @Test
+ public void testKafkaRecordEntity() throws Throwable
+ {
+ final MethodHandle handle = KafkaUtils.lookupGetHeaderMethod(KafkaUtilsTest.class.getClassLoader(), "version");
+ KafkaRecordEntity input = new KafkaRecordEntity(
+ new ConsumerRecord<>(
+ "test",
+ 0,
+ 0,
+ 0,
+ TimestampType.CREATE_TIME,
+ -1L,
+ -1,
+ -1,
+ null,
+ new byte[]{},
+ new RecordHeaders(ImmutableList.of(new Header()
+ {
+ @Override
+ public String key()
+ {
+ return "version";
+ }
+
+ @Override
+ public byte[] value()
+ {
+ return BYTES;
+ }
+ }))
+ )
+ );
+ Assert.assertArrayEquals(BYTES, (byte[]) handle.invoke(input));
+ }
+
+ @Test(expected = ClassCastException.class)
+ public void testNonKafkaEntity() throws Throwable
+ {
+ final MethodHandle handle = KafkaUtils.lookupGetHeaderMethod(KafkaUtilsTest.class.getClassLoader(), "version");
+ handle.invoke(new ByteEntity(new byte[]{}));
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusBenchmark.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusBenchmark.java
new file mode 100644
index 000000000000..871ce0321b8f
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusBenchmark.java
@@ -0,0 +1,118 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.Timestamp;
+import io.opencensus.proto.metrics.v1.LabelKey;
+import io.opencensus.proto.metrics.v1.LabelValue;
+import io.opencensus.proto.metrics.v1.Metric;
+import io.opencensus.proto.metrics.v1.MetricDescriptor;
+import io.opencensus.proto.metrics.v1.Point;
+import io.opencensus.proto.metrics.v1.TimeSeries;
+import io.opencensus.proto.resource.v1.Resource;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.JSONParseSpec;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+@Fork(1)
+public class OpenCensusBenchmark
+{
+ private static final Instant INSTANT = Instant.parse("2019-07-12T09:30:01.123Z");
+ private static final Timestamp TIMESTAMP = Timestamp.newBuilder()
+ .setSeconds(INSTANT.getEpochSecond())
+ .setNanos(INSTANT.getNano()).build();
+
+ private static final JSONParseSpec PARSE_SPEC = new JSONParseSpec(
+ new TimestampSpec("timestamp", "millis", null),
+ new DimensionsSpec(Collections.emptyList()),
+ new JSONPathSpec(
+ true,
+ Lists.newArrayList(
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "name", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "value", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "foo_key", "")
+ )
+ ), null, null
+ );
+
+ private static final OpenCensusProtobufInputRowParser PARSER = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ private static final ByteBuffer BUFFER = ByteBuffer.wrap(createMetric().toByteArray());
+
+ static Metric createMetric()
+ {
+ final MetricDescriptor.Builder descriptorBuilder = MetricDescriptor.newBuilder()
+ .setName("io.confluent.domain/such/good/metric/wow")
+ .setUnit("ms")
+ .setType(MetricDescriptor.Type.CUMULATIVE_DOUBLE);
+
+
+ final TimeSeries.Builder tsBuilder = TimeSeries.newBuilder()
+ .setStartTimestamp(TIMESTAMP)
+ .addPoints(Point.newBuilder().setDoubleValue(42.0).build());
+ for (int i = 0; i < 10; i++) {
+ descriptorBuilder.addLabelKeys(LabelKey.newBuilder()
+ .setKey("foo_key_" + i)
+ .build());
+ tsBuilder.addLabelValues(LabelValue.newBuilder()
+ .setHasValue(true)
+ .setValue("foo_value")
+ .build());
+ }
+
+ final Map resourceLabels = new HashMap<>();
+ for (int i = 0; i < 5; i++) {
+ resourceLabels.put("resoure.label_" + i, "val_" + i);
+ }
+
+ return Metric.newBuilder()
+ .setMetricDescriptor(descriptorBuilder.build())
+ .setResource(
+ Resource.newBuilder()
+ .setType("env")
+ .putAllLabels(resourceLabels)
+ .build())
+ .addTimeseries(tsBuilder.build())
+ .build();
+ }
+
+ @Benchmark()
+ public void measureSerde(Blackhole blackhole)
+ {
+ // buffer must be reset / duplicated each time to ensure each iteration reads the entire buffer from the beginning
+ for (InputRow row : PARSER.parseBatch(BUFFER.duplicate())) {
+ blackhole.consume(row);
+ }
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusInputFormatTest.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusInputFormatTest.java
new file mode 100644
index 000000000000..7aeba5462612
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusInputFormatTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.InputFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class OpenCensusInputFormatTest
+{
+ @Test
+ public void testSerde() throws Exception
+ {
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name", null, "descriptor.", "custom.");
+
+ final ObjectMapper jsonMapper = new ObjectMapper();
+ jsonMapper.registerModules(new OpenCensusProtobufExtensionsModule().getJacksonModules());
+
+ final OpenCensusProtobufInputFormat actual = (OpenCensusProtobufInputFormat) jsonMapper.readValue(
+ jsonMapper.writeValueAsString(inputFormat),
+ InputFormat.class
+ );
+ Assert.assertEquals(inputFormat, actual);
+ Assert.assertEquals("metric.name", actual.getMetricDimension());
+ Assert.assertEquals("descriptor.", actual.getMetricLabelPrefix());
+ Assert.assertEquals("custom.", actual.getResourceLabelPrefix());
+ }
+
+ @Test
+ public void testDefaults()
+ {
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat(null, null, null, null);
+
+ Assert.assertEquals("name", inputFormat.getMetricDimension());
+ Assert.assertEquals("", inputFormat.getMetricLabelPrefix());
+ Assert.assertEquals("resource.", inputFormat.getResourceLabelPrefix());
+ }
+}
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java
new file mode 100644
index 000000000000..a9c696cd27cd
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufInputRowParserTest.java
@@ -0,0 +1,477 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.protobuf.DoubleValue;
+import com.google.protobuf.Int64Value;
+import com.google.protobuf.Timestamp;
+import io.opencensus.proto.metrics.v1.DistributionValue;
+import io.opencensus.proto.metrics.v1.LabelKey;
+import io.opencensus.proto.metrics.v1.LabelValue;
+import io.opencensus.proto.metrics.v1.Metric;
+import io.opencensus.proto.metrics.v1.MetricDescriptor;
+import io.opencensus.proto.metrics.v1.MetricDescriptor.Type;
+import io.opencensus.proto.metrics.v1.Point;
+import io.opencensus.proto.metrics.v1.SummaryValue;
+import io.opencensus.proto.metrics.v1.TimeSeries;
+import io.opencensus.proto.resource.v1.Resource;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.InputRowParser;
+import org.apache.druid.data.input.impl.JSONParseSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
+import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.List;
+
+public class OpenCensusProtobufInputRowParserTest
+{
+ private static final Instant INSTANT = Instant.parse("2019-07-12T09:30:01.123Z");
+ private static final Timestamp TIMESTAMP = Timestamp.newBuilder()
+ .setSeconds(INSTANT.getEpochSecond())
+ .setNanos(INSTANT.getNano()).build();
+
+ static final JSONParseSpec PARSE_SPEC = new JSONParseSpec(
+ new TimestampSpec("timestamp", "millis", null),
+ new DimensionsSpec(Collections.emptyList()),
+ new JSONPathSpec(
+ true,
+ Lists.newArrayList(
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "name", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "value", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "foo_key", "")
+ )
+ ), null, null
+ );
+
+ static final JSONParseSpec PARSE_SPEC_WITH_DIMENSIONS = new JSONParseSpec(
+ new TimestampSpec("timestamp", "millis", null),
+ new DimensionsSpec(ImmutableList.of(
+ new StringDimensionSchema("foo_key"),
+ new StringDimensionSchema("env_key")
+ )),
+ new JSONPathSpec(
+ true,
+ Lists.newArrayList(
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "name", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "value", ""),
+ new JSONPathFieldSpec(JSONPathFieldType.ROOT, "foo_key", "")
+ )
+ ), null, null
+ );
+
+ @Rule
+ public ExpectedException expectedException = ExpectedException.none();
+
+ @Test
+ public void testSerde() throws Exception
+ {
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(
+ OpenCensusProtobufInputRowParserTest.PARSE_SPEC,
+ "metric.name",
+ "descriptor.",
+ "custom."
+ );
+
+ final ObjectMapper jsonMapper = new ObjectMapper();
+ jsonMapper.registerModules(new OpenCensusProtobufExtensionsModule().getJacksonModules());
+
+ final OpenCensusProtobufInputRowParser actual = (OpenCensusProtobufInputRowParser) jsonMapper.readValue(
+ jsonMapper.writeValueAsString(parser),
+ InputRowParser.class
+ );
+ Assert.assertEquals(parser, actual);
+ Assert.assertEquals("metric.name", actual.getMetricDimension());
+ Assert.assertEquals("descriptor.", actual.getMetricLabelPrefix());
+ Assert.assertEquals("custom.", actual.getResourceLabelPrefix());
+ }
+
+
+ @Test
+ public void testDefaults()
+ {
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(
+ OpenCensusProtobufInputRowParserTest.PARSE_SPEC,
+ null, null, null
+ );
+
+ Assert.assertEquals("name", parser.getMetricDimension());
+ Assert.assertEquals("", parser.getMetricLabelPrefix());
+ Assert.assertEquals("", parser.getResourceLabelPrefix());
+ }
+
+ @Test
+ public void testDoubleGaugeParse()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ Metric metric = doubleGaugeMetric(TIMESTAMP);
+
+ InputRow row = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray())).get(0);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+
+ assertDimensionEquals(row, "name", "metric_gauge_double");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+
+ Assert.assertEquals(2000, row.getMetric("value").doubleValue(), 0.0);
+ }
+
+ @Test
+ public void testIntGaugeParse()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ Metric metric = intGaugeMetric(TIMESTAMP);
+
+ InputRow row = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray())).get(0);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+
+ assertDimensionEquals(row, "name", "metric_gauge_int64");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+ Assert.assertEquals(1000, row.getMetric("value").intValue());
+ }
+
+ @Test
+ public void testSummaryParse()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ Metric metric = summaryMetric(TIMESTAMP);
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+ assertDimensionEquals(row, "name", "metric_summary-count");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ Assert.assertEquals(40, row.getMetric("value").doubleValue(), 0.0);
+
+ row = rows.get(1);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+ assertDimensionEquals(row, "name", "metric_summary-sum");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ Assert.assertEquals(10, row.getMetric("value").doubleValue(), 0.0);
+ }
+
+ @Test
+ public void testDistributionParse()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ Metric metric = distributionMetric(TIMESTAMP);
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+ assertDimensionEquals(row, "name", "metric_distribution-count");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ Assert.assertEquals(100, row.getMetric("value").intValue());
+
+ row = rows.get(1);
+ Assert.assertEquals(INSTANT.toEpochMilli(), row.getTimestampFromEpoch());
+ assertDimensionEquals(row, "name", "metric_distribution-sum");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ Assert.assertEquals(500, row.getMetric("value").doubleValue(), 0.0);
+ }
+
+ @Test
+ public void testDimensionsParseWithParseSpecDimensions()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC_WITH_DIMENSIONS, null, null, "");
+
+ Metric metric = summaryMetric(TIMESTAMP);
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(2, row.getDimensions().size());
+ assertDimensionEquals(row, "env_key", "env_val");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+ row = rows.get(1);
+ Assert.assertEquals(2, row.getDimensions().size());
+ assertDimensionEquals(row, "env_key", "env_val");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+ }
+
+ @Test
+ public void testDimensionsParseWithoutPARSE_SPECDimensions()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, "");
+
+ Metric metric = summaryMetric(TIMESTAMP);
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-count");
+ assertDimensionEquals(row, "env_key", "env_val");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+ row = rows.get(1);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-sum");
+ assertDimensionEquals(row, "env_key", "env_val");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+
+ }
+
+ @Test
+ public void testMetricNameOverride()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, "dimension_name", null, "");
+
+ Metric metric = summaryMetric(Timestamp.getDefaultInstance());
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "dimension_name", "metric_summary-count");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ assertDimensionEquals(row, "env_key", "env_val");
+
+ row = rows.get(1);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "dimension_name", "metric_summary-sum");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ assertDimensionEquals(row, "env_key", "env_val");
+ }
+
+ @Test
+ public void testDefaultPrefix()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, null, null);
+
+ Metric metric = summaryMetric(Timestamp.getDefaultInstance());
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-count");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ assertDimensionEquals(row, "env_key", "env_val");
+
+ row = rows.get(1);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-sum");
+ assertDimensionEquals(row, "foo_key", "foo_value");
+ assertDimensionEquals(row, "env_key", "env_val");
+ }
+
+ @Test
+ public void testCustomPrefix()
+ {
+ //configure parser with desc file
+ OpenCensusProtobufInputRowParser parser = new OpenCensusProtobufInputRowParser(PARSE_SPEC, null, "descriptor.", "custom.");
+
+ Metric metric = summaryMetric(Timestamp.getDefaultInstance());
+
+ List rows = parser.parseBatch(ByteBuffer.wrap(metric.toByteArray()));
+
+ Assert.assertEquals(2, rows.size());
+
+ InputRow row = rows.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-count");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ assertDimensionEquals(row, "custom.env_key", "env_val");
+
+ row = rows.get(1);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "name", "metric_summary-sum");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ assertDimensionEquals(row, "custom.env_key", "env_val");
+ }
+
+ private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ {
+ List values = row.getDimension(dimension);
+
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(expected, values.get(0));
+ }
+
+ static Metric doubleGaugeMetric(Timestamp timestamp)
+ {
+ return getMetric(
+ "metric_gauge_double",
+ "metric_gauge_double_description",
+ Type.GAUGE_DOUBLE,
+ Point.newBuilder()
+ .setTimestamp(timestamp)
+ .setDoubleValue(2000)
+ .build(),
+ timestamp);
+ }
+
+ static Metric intGaugeMetric(Timestamp timestamp)
+ {
+ return getMetric(
+ "metric_gauge_int64",
+ "metric_gauge_int64_description",
+ MetricDescriptor.Type.GAUGE_INT64,
+ Point.newBuilder()
+ .setTimestamp(timestamp)
+ .setInt64Value(1000)
+ .build(),
+ timestamp);
+ }
+
+ static Metric summaryMetric(Timestamp timestamp)
+ {
+
+ SummaryValue.Snapshot snapshot = SummaryValue.Snapshot.newBuilder()
+ .setSum(DoubleValue.newBuilder().setValue(10).build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(50.0)
+ .setValue(10)
+ .build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(75.0)
+ .setValue(20)
+ .build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(95.0)
+ .setValue(30)
+ .build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(98.0)
+ .setValue(40)
+ .build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(99.0)
+ .setValue(50)
+ .build())
+ .addPercentileValues(SummaryValue.Snapshot.ValueAtPercentile.newBuilder()
+ .setPercentile(99.9)
+ .setValue(60)
+ .build())
+ .build();
+
+
+ SummaryValue summaryValue = SummaryValue.newBuilder()
+ .setCount(Int64Value.newBuilder().setValue(40).build())
+ .setSnapshot(snapshot)
+ .build();
+
+ return getMetric(
+ "metric_summary",
+ "metric_summary_description",
+ MetricDescriptor.Type.SUMMARY,
+ Point.newBuilder()
+ .setTimestamp(timestamp)
+ .setSummaryValue(summaryValue)
+ .build(),
+ timestamp);
+ }
+
+ static Metric distributionMetric(Timestamp timestamp)
+ {
+ DistributionValue distributionValue = DistributionValue.newBuilder()
+ .setCount(100)
+ .setSum(500)
+ .build();
+
+ return getMetric(
+ "metric_distribution",
+ "metric_distribution_description",
+ MetricDescriptor.Type.GAUGE_DISTRIBUTION,
+ Point.newBuilder()
+ .setTimestamp(timestamp)
+ .setDistributionValue(distributionValue)
+ .build(),
+ timestamp);
+ }
+
+ static Metric getMetric(String name, String description, MetricDescriptor.Type type, Point point, Timestamp timestamp)
+ {
+ Metric dist = Metric.newBuilder()
+ .setMetricDescriptor(
+ MetricDescriptor.newBuilder()
+ .setName(name)
+ .setDescription(description)
+ .setUnit("ms")
+ .setType(type)
+ .addLabelKeys(
+ LabelKey.newBuilder()
+ .setKey("foo_key")
+ .build())
+ .build())
+ .setResource(
+ Resource.newBuilder()
+ .setType("env")
+ .putAllLabels(Collections.singletonMap("env_key", "env_val"))
+ .build())
+ .addTimeseries(
+ TimeSeries.newBuilder()
+ .setStartTimestamp(timestamp)
+ .addLabelValues(
+ LabelValue.newBuilder()
+ .setHasValue(true)
+ .setValue("foo_value")
+ .build())
+ .addPoints(point)
+ .build())
+ .build();
+
+ return dist;
+ }
+
+}
diff --git a/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReaderTest.java b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReaderTest.java
new file mode 100644
index 000000000000..885e8617c495
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/java/org/apache/druid/data/input/opencensus/protobuf/OpenCensusProtobufReaderTest.java
@@ -0,0 +1,445 @@
+/*
+ * 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.data.input.opencensus.protobuf;
+
+import com.google.common.collect.ImmutableList;
+import io.opentelemetry.proto.common.v1.AnyValue;
+import io.opentelemetry.proto.common.v1.KeyValue;
+import io.opentelemetry.proto.metrics.v1.Metric;
+import io.opentelemetry.proto.metrics.v1.MetricsData;
+import org.apache.curator.shaded.com.google.common.base.Predicate;
+import org.apache.druid.data.input.ColumnsFilter;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.indexing.common.task.FilteringCloseableInputRowIterator;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+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.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.record.TimestampType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.mockito.Mockito.mock;
+
+
+public class OpenCensusProtobufReaderTest
+{
+ private static final long TIMESTAMP = TimeUnit.MILLISECONDS.toNanos(Instant.parse("2019-07-12T09:30:01.123Z").toEpochMilli());
+ public static final String RESOURCE_ATTRIBUTE_COUNTRY = "country";
+ public static final String RESOURCE_ATTRIBUTE_VALUE_USA = "usa";
+
+ public static final String RESOURCE_ATTRIBUTE_ENV = "env";
+ public static final String RESOURCE_ATTRIBUTE_VALUE_DEVEL = "devel";
+
+ public static final String INSTRUMENTATION_SCOPE_NAME = "mock-instr-lib";
+ public static final String INSTRUMENTATION_SCOPE_VERSION = "1.0";
+
+ public static final String METRIC_ATTRIBUTE_COLOR = "color";
+ public static final String METRIC_ATTRIBUTE_VALUE_RED = "red";
+
+ public static final String METRIC_ATTRIBUTE_FOO_KEY = "foo_key";
+ public static final String METRIC_ATTRIBUTE_FOO_VAL = "foo_value";
+
+ private final MetricsData.Builder metricsDataBuilder = MetricsData.newBuilder();
+
+ private final Metric.Builder metricBuilder = metricsDataBuilder.addResourceMetricsBuilder()
+ .addScopeMetricsBuilder()
+ .addMetricsBuilder();
+
+ private final DimensionsSpec dimensionsSpec = new DimensionsSpec(ImmutableList.of(
+ new StringDimensionSchema("descriptor." + METRIC_ATTRIBUTE_COLOR),
+ new StringDimensionSchema("descriptor." + METRIC_ATTRIBUTE_FOO_KEY),
+ new StringDimensionSchema("custom." + RESOURCE_ATTRIBUTE_ENV),
+ new StringDimensionSchema("custom." + RESOURCE_ATTRIBUTE_COUNTRY)
+ ));
+
+ public static final String TOPIC = "telemetry.metrics.otel";
+ public static final int PARTITION = 2;
+ public static final long OFFSET = 13095752723L;
+ public static final long TS = 1643974867555L;
+ public static final TimestampType TSTYPE = TimestampType.CREATE_TIME;
+ public static final byte[] V0_HEADER_BYTES = ByteBuffer.allocate(Integer.BYTES)
+ .order(ByteOrder.LITTLE_ENDIAN)
+ .putInt(1)
+ .array();
+ private static final Header HEADERV1 = new RecordHeader("v", V0_HEADER_BYTES);
+ private static final Headers HEADERS = new RecordHeaders(new Header[]{HEADERV1});
+
+ @Before
+ public void setUp()
+ {
+ metricsDataBuilder
+ .getResourceMetricsBuilder(0)
+ .getResourceBuilder()
+ .addAttributes(KeyValue.newBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_COUNTRY)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_USA)));
+
+ metricsDataBuilder
+ .getResourceMetricsBuilder(0)
+ .getScopeMetricsBuilder(0)
+ .getScopeBuilder()
+ .setName(INSTRUMENTATION_SCOPE_NAME)
+ .setVersion(INSTRUMENTATION_SCOPE_VERSION);
+
+ }
+
+ @Test
+ public void testSumWithAttributes() throws IOException
+ {
+ metricBuilder
+ .setName("example_sum")
+ .getSumBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+
+ MetricsData metricsData = metricsDataBuilder.build();
+ ConsumerRecord consumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, metricsData.toByteArray(), HEADERS, Optional.empty());
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat(
+ "metric.name",
+ null,
+ "descriptor.",
+ "custom."
+ );
+
+ SettableByteEntity entity = new SettableByteEntity<>();
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpec,
+ ColumnsFilter.all()
+ ), entity, null);
+
+ entity.setEntity(new KafkaRecordEntity(consumerRecord));
+ try (CloseableIterator rows = reader.read()) {
+ List rowList = new ArrayList<>();
+ rows.forEachRemaining(rowList::add);
+ Assert.assertEquals(1, rowList.size());
+
+ InputRow row = rowList.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_sum");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "value", "6");
+ }
+ }
+
+ @Test
+ public void testGaugeWithAttributes() throws IOException
+ {
+ metricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+ MetricsData metricsData = metricsDataBuilder.build();
+ ConsumerRecord consumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, metricsData.toByteArray(), HEADERS, Optional.empty());
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name",
+ null,
+ "descriptor.",
+ "custom.");
+ SettableByteEntity entity = new SettableByteEntity<>();
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpec,
+ ColumnsFilter.all()
+ ), entity, null);
+
+ entity.setEntity(new KafkaRecordEntity(consumerRecord));
+ try (CloseableIterator rows = reader.read()) {
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "value", "6");
+ }
+ }
+
+ @Test
+ public void testBatchedMetricParse() throws IOException
+ {
+ metricBuilder.setName("example_sum")
+ .getSumBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+ // Create Second Metric
+ Metric.Builder gaugeMetricBuilder = metricsDataBuilder.addResourceMetricsBuilder()
+ .addScopeMetricsBuilder()
+ .addMetricsBuilder();
+
+ metricsDataBuilder.getResourceMetricsBuilder(1)
+ .getResourceBuilder()
+ .addAttributes(KeyValue.newBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_ENV)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_DEVEL))
+ .build());
+
+ metricsDataBuilder.getResourceMetricsBuilder(1)
+ .getScopeMetricsBuilder(0)
+ .getScopeBuilder()
+ .setName(INSTRUMENTATION_SCOPE_NAME)
+ .setVersion(INSTRUMENTATION_SCOPE_VERSION);
+
+ gaugeMetricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(8)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_FOO_KEY)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_FOO_VAL).build());
+
+ MetricsData metricsData = metricsDataBuilder.build();
+ ConsumerRecord consumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, metricsData.toByteArray(), HEADERS, Optional.empty());
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name",
+ null,
+ "descriptor.",
+ "custom.");
+ SettableByteEntity entity = new SettableByteEntity<>();
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpec,
+ ColumnsFilter.all()
+ ), entity, null);
+
+ entity.setEntity(new KafkaRecordEntity(consumerRecord));
+ try (CloseableIterator rows = reader.read()) {
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_sum");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "value", "6");
+
+ Assert.assertTrue(rows.hasNext());
+ row = rows.next();
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "custom.env", "devel");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ assertDimensionEquals(row, "value", "8");
+ }
+ }
+
+ @Test
+ public void testDimensionSpecExclusions() throws IOException
+ {
+ metricsDataBuilder.getResourceMetricsBuilder(0)
+ .getResourceBuilder()
+ .addAttributesBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_ENV)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_DEVEL).build());
+
+ metricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAllAttributes(ImmutableList.of(
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build()).build(),
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_FOO_KEY)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_FOO_VAL).build()).build()));
+
+ DimensionsSpec dimensionsSpecWithExclusions = DimensionsSpec.builder().setDimensionExclusions(
+ ImmutableList.of(
+ "descriptor." + METRIC_ATTRIBUTE_COLOR,
+ "custom." + RESOURCE_ATTRIBUTE_COUNTRY
+ )).build();
+
+ MetricsData metricsData = metricsDataBuilder.build();
+ ConsumerRecord consumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, metricsData.toByteArray(), HEADERS, Optional.empty());
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name",
+ null,
+ "descriptor.",
+ "custom.");
+
+ SettableByteEntity entity = new SettableByteEntity<>();
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpecWithExclusions,
+ ColumnsFilter.all()
+ ), entity, null);
+
+ entity.setEntity(new KafkaRecordEntity(consumerRecord));
+ try (CloseableIterator rows = reader.read()) {
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ long deviatedSeconds = (TS - (TIMESTAMP / 1_000_000)) / 1000;
+ long deviatedMinutes = deviatedSeconds / 60;
+
+ Assert.assertEquals(6, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "value", "6");
+ assertDimensionEquals(row, "custom.env", "devel");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ assertDimensionEquals(row, "deviated_seconds", Long.toString(deviatedSeconds));
+ assertDimensionEquals(row, "deviated_minutes", Long.toString(deviatedMinutes));
+ Assert.assertFalse(row.getDimensions().contains("custom.country"));
+ Assert.assertFalse(row.getDimensions().contains("descriptor.color"));
+ }
+ }
+
+ @Test
+ public void testInvalidProtobuf() throws IOException
+ {
+ byte[] invalidProtobuf = new byte[] {0x00, 0x01};
+ ConsumerRecord consumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, invalidProtobuf, HEADERS, Optional.empty());
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name",
+ null,
+ "descriptor.",
+ "custom.");
+
+ SettableByteEntity entity = new SettableByteEntity<>();
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpec,
+ ColumnsFilter.all()
+ ), entity, null);
+
+ entity.setEntity(new KafkaRecordEntity(consumerRecord));
+ try (CloseableIterator rows = reader.read()) {
+ Assert.assertThrows(ParseException.class, () -> rows.hasNext());
+ Assert.assertThrows(NoSuchElementException.class, () -> rows.next());
+ }
+ }
+
+ @Test
+ public void testMultipleInvalidProtobuf() throws IOException
+ {
+ byte[] invalidProtobuf = new byte[] {0x00, 0x01};
+ byte[] validProtobuf = new byte[] {};
+ ConsumerRecord invalidConsumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET, TS, TSTYPE, -1, -1,
+ null, invalidProtobuf, HEADERS, Optional.empty());
+ ConsumerRecord validConsumerRecord = new ConsumerRecord<>(TOPIC, PARTITION, OFFSET + 1, TS, TSTYPE, -1, -1,
+ null, validProtobuf, HEADERS, Optional.empty());
+ List> records = new ArrayList<>();
+ records.add(new OrderedPartitionableRecord<>(
+ invalidConsumerRecord.topic(),
+ invalidConsumerRecord.partition(),
+ invalidConsumerRecord.offset(),
+ ImmutableList.of(new KafkaRecordEntity(invalidConsumerRecord))
+ ));
+ records.add(new OrderedPartitionableRecord<>(
+ validConsumerRecord.topic(),
+ validConsumerRecord.partition(),
+ validConsumerRecord.offset(),
+ ImmutableList.of(new KafkaRecordEntity(validConsumerRecord))
+ ));
+ int recordsProcessed = 0;
+ OpenCensusProtobufInputFormat inputFormat = new OpenCensusProtobufInputFormat("metric.name",
+ null,
+ "descriptor.",
+ "custom.");
+ for (OrderedPartitionableRecord record : records) {
+
+ SettableByteEntity entity = new SettableByteEntity<>();
+ OpenCensusProtobufReader readR = new OpenCensusProtobufReader(
+ dimensionsSpec,
+ entity,
+ "metric.name",
+ "descriptor.",
+ "custom."
+
+ );
+ InputEntityReader reader = inputFormat.createReader(new InputRowSchema(
+ new TimestampSpec("timestamp", "iso", null),
+ dimensionsSpec,
+ ColumnsFilter.all()
+ ), entity, null);
+ System.out.println("Processing record " + record.getSequenceNumber());
+ final List rows = new ArrayList<>();
+ for (ByteEntity byteEntity : record.getData()) {
+ System.out.println("Processing byte entity " + record.getData().indexOf(byteEntity));
+ entity.setEntity(byteEntity);
+ try (FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator(
+ readR.read(),
+ mock(Predicate.class),
+ mock(RowIngestionMeters.class),
+ mock(ParseExceptionHandler.class)
+ )) {
+ rowIterator.forEachRemaining(rows::add);
+ }
+ }
+ recordsProcessed += 1;
+ }
+ Assert.assertEquals(recordsProcessed, 2);
+ }
+
+ private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ {
+ List values = row.getDimension(dimension);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(expected, values.get(0));
+ }
+
+}
diff --git a/extensions-contrib/opencensus-extensions/src/test/resources/log4j2.xml b/extensions-contrib/opencensus-extensions/src/test/resources/log4j2.xml
new file mode 100644
index 000000000000..05a8e1d69cbe
--- /dev/null
+++ b/extensions-contrib/opencensus-extensions/src/test/resources/log4j2.xml
@@ -0,0 +1,35 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/extensions-contrib/opentelemetry-extensions/pom.xml b/extensions-contrib/opentelemetry-extensions/pom.xml
new file mode 100644
index 000000000000..88882f4ee8cb
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/pom.xml
@@ -0,0 +1,102 @@
+
+
+
+ 4.0.0
+
+ org.apache.druid.extensions.contrib
+ druid-opentelemetry-extensions
+ druid-opentelemetry-extensions
+ druid-opentelemetry-extensions
+
+
+ druid
+ org.apache.druid
+ 30.0.1
+ ../../pom.xml
+
+
+
+ com.google.protobuf
+ protobuf-java
+
+
+ io.opentelemetry.proto
+ opentelemetry-proto
+
+
+ com.google.guava
+ guava
+ provided
+
+
+ com.google.inject
+ guice
+ provided
+
+
+ com.google.code.findbugs
+ jsr305
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-annotations
+ provided
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+ provided
+
+
+ org.apache.druid
+ druid-processing
+ ${project.parent.version}
+ provided
+
+
+ org.apache.druid
+ druid-indexing-service
+ ${project.parent.version}
+ provided
+
+
+
+ junit
+ junit
+ test
+
+
+
+ org.openjdk.jmh
+ jmh-core
+ 1.27
+ test
+
+
+ org.openjdk.jmh
+ jmh-generator-annprocess
+ 1.27
+ test
+
+
+
diff --git a/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufInputFormat.java b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufInputFormat.java
new file mode 100644
index 000000000000..50029e8dfbd9
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufInputFormat.java
@@ -0,0 +1,132 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.data.input.InputEntity;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.StringUtils;
+
+import java.io.File;
+import java.util.Objects;
+
+public class OpenTelemetryMetricsProtobufInputFormat implements InputFormat
+{
+ private static final String DEFAULT_METRIC_DIMENSION = "metric";
+ private static final String DEFAULT_VALUE_DIMENSION = "value";
+ private static final String DEFAULT_RESOURCE_PREFIX = "resource.";
+
+ private final String metricDimension;
+ private final String valueDimension;
+ private final String metricAttributePrefix;
+ private final String resourceAttributePrefix;
+
+ public OpenTelemetryMetricsProtobufInputFormat(
+ @JsonProperty("metricDimension") String metricDimension,
+ @JsonProperty("valueDimension") String valueDimension,
+ @JsonProperty("metricAttributePrefix") String metricAttributePrefix,
+ @JsonProperty("resourceAttributePrefix") String resourceAttributePrefix
+ )
+ {
+ this.metricDimension = metricDimension != null ? metricDimension : DEFAULT_METRIC_DIMENSION;
+ this.valueDimension = valueDimension != null ? valueDimension : DEFAULT_VALUE_DIMENSION;
+ this.metricAttributePrefix = StringUtils.nullToEmptyNonDruidDataString(metricAttributePrefix);
+ this.resourceAttributePrefix = resourceAttributePrefix != null ? resourceAttributePrefix : DEFAULT_RESOURCE_PREFIX;
+ }
+
+ @Override
+ public boolean isSplittable()
+ {
+ return false;
+ }
+
+ @Override
+ public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
+ {
+ // Sampler passes a KafkaRecordEntity directly, while the normal code path wraps the same entity in a
+ // SettableByteEntity
+ SettableByteEntity extends ByteEntity> settableEntity;
+ if (source instanceof SettableByteEntity) {
+ settableEntity = (SettableByteEntity extends ByteEntity>) source;
+ } else {
+ SettableByteEntity wrapper = new SettableByteEntity<>();
+ wrapper.setEntity((ByteEntity) source);
+ settableEntity = wrapper;
+ }
+ return new OpenTelemetryMetricsProtobufReader(
+ inputRowSchema.getDimensionsSpec(),
+ settableEntity,
+ metricDimension,
+ valueDimension,
+ metricAttributePrefix,
+ resourceAttributePrefix
+ );
+ }
+
+ @JsonProperty
+ public String getMetricDimension()
+ {
+ return metricDimension;
+ }
+
+ @JsonProperty
+ public String getValueDimension()
+ {
+ return valueDimension;
+ }
+
+ @JsonProperty
+ public String getMetricAttributePrefix()
+ {
+ return metricAttributePrefix;
+ }
+
+ @JsonProperty
+ public String getResourceAttributePrefix()
+ {
+ return resourceAttributePrefix;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof OpenTelemetryMetricsProtobufInputFormat)) {
+ return false;
+ }
+ OpenTelemetryMetricsProtobufInputFormat that = (OpenTelemetryMetricsProtobufInputFormat) o;
+ return Objects.equals(metricDimension, that.metricDimension)
+ && Objects.equals(valueDimension, that.valueDimension)
+ && Objects.equals(metricAttributePrefix, that.metricAttributePrefix)
+ && Objects.equals(resourceAttributePrefix, that.resourceAttributePrefix);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(metricDimension, valueDimension, metricAttributePrefix, resourceAttributePrefix);
+ }
+}
diff --git a/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReader.java b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReader.java
new file mode 100644
index 000000000000..8fd678f4480c
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReader.java
@@ -0,0 +1,272 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.protobuf.InvalidProtocolBufferException;
+import io.opentelemetry.proto.common.v1.AnyValue;
+import io.opentelemetry.proto.metrics.v1.DataPointFlags;
+import io.opentelemetry.proto.metrics.v1.Metric;
+import io.opentelemetry.proto.metrics.v1.MetricsData;
+import io.opentelemetry.proto.metrics.v1.NumberDataPoint;
+import org.apache.druid.data.input.InputEntityReader;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowListPlusRawValues;
+import org.apache.druid.data.input.KafkaEntity;
+import org.apache.druid.data.input.MapBasedInputRow;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.CloseableIterators;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+public class OpenTelemetryMetricsProtobufReader implements InputEntityReader
+{
+ private static final Logger log = new Logger(OpenTelemetryMetricsProtobufReader.class);
+
+ private final SettableByteEntity extends ByteEntity> source;
+ private final String metricDimension;
+ private final String valueDimension;
+ private final String metricAttributePrefix;
+ private final String resourceAttributePrefix;
+ private final DimensionsSpec dimensionsSpec;
+
+ private static final long NANOS_TO_MILLIS = 1_000_000L;
+ private static final long MILLIS_PER_SECOND = 1_000L;
+
+ public OpenTelemetryMetricsProtobufReader(
+ DimensionsSpec dimensionsSpec,
+ SettableByteEntity extends ByteEntity> source,
+ String metricDimension,
+ String valueDimension,
+ String metricAttributePrefix,
+ String resourceAttributePrefix
+ )
+ {
+ this.dimensionsSpec = dimensionsSpec;
+ this.source = source;
+ this.metricDimension = metricDimension;
+ this.valueDimension = valueDimension;
+ this.metricAttributePrefix = metricAttributePrefix;
+ this.resourceAttributePrefix = resourceAttributePrefix;
+ }
+
+ @Override
+ public CloseableIterator read()
+ {
+ Supplier> supplier = Suppliers.memoize(() -> readAsList().iterator());
+ return CloseableIterators.withEmptyBaggage(new Iterator() {
+ @Override
+ public boolean hasNext()
+ {
+ return supplier.get().hasNext();
+ }
+ @Override
+ public InputRow next()
+ {
+ return supplier.get().next();
+ }
+ });
+ }
+
+ List readAsList()
+ {
+ ByteBuffer buffer = source.getEntity().getBuffer();
+ try {
+ return parseMetricsData(MetricsData.parseFrom(buffer));
+ }
+ catch (InvalidProtocolBufferException e) {
+ throw new ParseException(null, e, "Protobuf message could not be parsed");
+ }
+ finally {
+ // Explicitly move the position assuming that all the remaining bytes have been consumed because the protobuf
+ // parser does not update the position itself
+ // In case of an exception, the buffer is moved to the end to avoid parsing it in a loop.
+ buffer.position(buffer.limit());
+ }
+ }
+
+ private List parseMetricsData(final MetricsData metricsData)
+ {
+ return metricsData.getResourceMetricsList()
+ .stream()
+ .flatMap(resourceMetrics -> {
+ Map resourceAttributes = resourceMetrics.getResource()
+ .getAttributesList()
+ .stream()
+ .collect(HashMap::new,
+ (m, kv) -> {
+ Object value = parseAnyValue(kv.getValue());
+ if (value != null) {
+ m.put(resourceAttributePrefix + kv.getKey(), value);
+ }
+ },
+ HashMap::putAll);
+ return resourceMetrics.getScopeMetricsList()
+ .stream()
+ .flatMap(scopeMetrics -> scopeMetrics.getMetricsList()
+ .stream()
+ .flatMap(metric -> parseMetric(metric, resourceAttributes).stream()));
+ })
+ .collect(Collectors.toList());
+ }
+
+ private List parseMetric(Metric metric, Map resourceAttributes)
+ {
+ final List inputRows;
+ String metricName = metric.getName();
+ switch (metric.getDataCase()) {
+ case SUM: {
+ inputRows = new ArrayList<>(metric.getSum().getDataPointsCount());
+ metric.getSum()
+ .getDataPointsList()
+ .forEach(dataPoint -> {
+ if (hasRecordedValue(dataPoint)) {
+ inputRows.add(parseNumberDataPoint(dataPoint, resourceAttributes, metricName));
+ }
+ });
+ break;
+ }
+ case GAUGE: {
+ inputRows = new ArrayList<>(metric.getGauge().getDataPointsCount());
+ metric.getGauge()
+ .getDataPointsList()
+ .forEach(dataPoint -> {
+ if (hasRecordedValue(dataPoint)) {
+ inputRows.add(parseNumberDataPoint(dataPoint, resourceAttributes, metricName));
+ }
+ });
+ break;
+ }
+ // TODO Support HISTOGRAM and SUMMARY metrics
+ case HISTOGRAM:
+ case SUMMARY:
+ default:
+ log.trace("Metric type %s is not supported", metric.getDataCase());
+ inputRows = Collections.emptyList();
+
+ }
+ return inputRows;
+ }
+
+ private static boolean hasRecordedValue(NumberDataPoint d)
+ {
+ return (d.getFlags() & DataPointFlags.FLAG_NO_RECORDED_VALUE_VALUE) == 0;
+ }
+
+ private InputRow parseNumberDataPoint(NumberDataPoint dataPoint,
+ Map resourceAttributes,
+ String metricName)
+ {
+
+ int capacity = resourceAttributes.size()
+ + dataPoint.getAttributesCount()
+ + 4; // metric name + value columns + deviated_seconds + deviated_minutes
+ Map event = Maps.newHashMapWithExpectedSize(capacity);
+ event.put(metricDimension, metricName);
+
+ if (dataPoint.hasAsInt()) {
+ event.put(valueDimension, dataPoint.getAsInt());
+ } else {
+ event.put(valueDimension, dataPoint.getAsDouble());
+ }
+
+ event.putAll(resourceAttributes);
+ dataPoint.getAttributesList().forEach(att -> {
+ Object value = parseAnyValue(att.getValue());
+ if (value != null) {
+ event.put(metricAttributePrefix + att.getKey(), value);
+ }
+ });
+
+ try {
+ Object entity = source.getEntity();
+ if (entity instanceof KafkaEntity) {
+ long timeUnixNano = dataPoint.getTimeUnixNano();
+ KafkaEntity kafkaEntity = (KafkaEntity) entity;
+ long createdTime = kafkaEntity.getRecordTimestampMillis();
+ long deviated_seconds = (createdTime - (timeUnixNano / NANOS_TO_MILLIS)) / MILLIS_PER_SECOND;
+ long deviated_minutes = deviated_seconds / 60;
+ event.put("deviated_seconds", deviated_seconds);
+ event.put("deviated_minutes", deviated_minutes);
+ } else {
+ log.error("Source entity is not an instance of KafkaEntity.");
+ }
+ }
+ catch (Exception e) {
+ log.error(e, "Could not extract timestamp from record entity");
+ }
+ return createRow(TimeUnit.NANOSECONDS.toMillis(dataPoint.getTimeUnixNano()), event);
+ }
+
+ @Nullable
+ private static Object parseAnyValue(AnyValue value)
+ {
+ switch (value.getValueCase()) {
+ case INT_VALUE:
+ return value.getIntValue();
+ case BOOL_VALUE:
+ return value.getBoolValue();
+ case DOUBLE_VALUE:
+ return value.getDoubleValue();
+ case STRING_VALUE:
+ return value.getStringValue();
+
+ // TODO: Support KVLIST_VALUE, ARRAY_VALUE and BYTES_VALUE
+
+ default:
+ // VALUE_NOT_SET
+ return null;
+ }
+ }
+
+ InputRow createRow(long timeUnixMilli, Map event)
+ {
+ final List dimensions;
+ if (!dimensionsSpec.getDimensionNames().isEmpty()) {
+ dimensions = dimensionsSpec.getDimensionNames();
+ } else {
+ dimensions = new ArrayList<>(Sets.difference(event.keySet(), dimensionsSpec.getDimensionExclusions()));
+ }
+ return new MapBasedInputRow(timeUnixMilli, dimensions, event);
+ }
+
+ @Override
+ public CloseableIterator sample()
+ {
+ return read().map(row -> InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()));
+ }
+}
diff --git a/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryProtobufExtensionsModule.java b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryProtobufExtensionsModule.java
new file mode 100644
index 000000000000..4c027c31248c
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/main/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryProtobufExtensionsModule.java
@@ -0,0 +1,49 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.fasterxml.jackson.databind.Module;
+import com.fasterxml.jackson.databind.jsontype.NamedType;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.google.inject.Binder;
+import org.apache.druid.initialization.DruidModule;
+
+import java.util.Collections;
+import java.util.List;
+
+public class OpenTelemetryProtobufExtensionsModule implements DruidModule
+{
+
+ @Override
+ public List extends Module> getJacksonModules()
+ {
+ return Collections.singletonList(
+ new SimpleModule("OpenTelemetryProtobufInputFormat")
+ .registerSubtypes(
+ new NamedType(OpenTelemetryMetricsProtobufInputFormat.class, "opentelemetry-metrics-protobuf")
+ )
+ );
+ }
+
+ @Override
+ public void configure(Binder binder)
+ {
+ }
+}
diff --git a/extensions-contrib/opentelemetry-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/opentelemetry-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
new file mode 100755
index 000000000000..b2a7d04bb635
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
@@ -0,0 +1,17 @@
+# 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.
+
+org.apache.druid.data.input.opentelemetry.protobuf.OpenTelemetryProtobufExtensionsModule
+
diff --git a/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryBenchmark.java b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryBenchmark.java
new file mode 100644
index 000000000000..0238aeccafa5
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryBenchmark.java
@@ -0,0 +1,135 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.google.common.collect.ImmutableList;
+import io.opentelemetry.proto.common.v1.AnyValue;
+import io.opentelemetry.proto.common.v1.KeyValue;
+import io.opentelemetry.proto.metrics.v1.Metric;
+import io.opentelemetry.proto.metrics.v1.MetricsData;
+import io.opentelemetry.proto.metrics.v1.NumberDataPoint;
+import io.opentelemetry.proto.metrics.v1.ResourceMetrics;
+import io.opentelemetry.proto.metrics.v1.ScopeMetrics;
+import io.opentelemetry.proto.resource.v1.Resource;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.util.concurrent.TimeUnit;
+
+@Fork(1)
+@State(Scope.Benchmark)
+public class OpenTelemetryBenchmark
+{
+
+ private static ByteBuffer BUFFER;
+
+ @Param(value = {"1", "2", "4", "8" })
+ private int resourceMetricCount = 1;
+
+ @Param(value = {"1"})
+ private int instrumentationScopeCount = 1;
+
+ @Param(value = {"1", "2", "4", "8" })
+ private int metricsCount = 1;
+
+ @Param(value = {"1", "2", "4", "8" })
+ private int dataPointCount;
+
+ private static final long TIMESTAMP = TimeUnit.MILLISECONDS.toNanos(Instant.parse("2019-07-12T09:30:01.123Z").toEpochMilli());
+
+ private static final InputRowSchema ROW_SCHEMA = new InputRowSchema(null,
+ new DimensionsSpec(ImmutableList.of(
+ new StringDimensionSchema("name"),
+ new StringDimensionSchema("value"),
+ new StringDimensionSchema("foo_key"))),
+ null);
+
+ private static final OpenTelemetryMetricsProtobufInputFormat INPUT_FORMAT =
+ new OpenTelemetryMetricsProtobufInputFormat("name",
+ "value",
+ "",
+ "resource.");
+
+ private ByteBuffer createMetricBuffer()
+ {
+ MetricsData.Builder metricsData = MetricsData.newBuilder();
+ for (int i = 0; i < resourceMetricCount; i++) {
+ ResourceMetrics.Builder resourceMetricsBuilder = metricsData.addResourceMetricsBuilder();
+ Resource.Builder resourceBuilder = resourceMetricsBuilder.getResourceBuilder();
+
+ for (int resourceAttributeI = 0; resourceAttributeI < 5; resourceAttributeI++) {
+ KeyValue.Builder resourceAttributeBuilder = resourceBuilder.addAttributesBuilder();
+ resourceAttributeBuilder.setKey("resource.label_key_" + resourceAttributeI);
+ resourceAttributeBuilder.setValue(AnyValue.newBuilder().setStringValue("resource.label_value"));
+ }
+
+ for (int j = 0; j < instrumentationScopeCount; j++) {
+ ScopeMetrics.Builder scopeMetricsBuilder =
+ resourceMetricsBuilder.addScopeMetricsBuilder();
+
+ for (int k = 0; k < metricsCount; k++) {
+ Metric.Builder metricBuilder = scopeMetricsBuilder.addMetricsBuilder();
+ metricBuilder.setName("io.confluent.domain/such/good/metric/wow");
+
+ for (int l = 0; l < dataPointCount; l++) {
+ NumberDataPoint.Builder dataPointBuilder = metricBuilder.getSumBuilder().addDataPointsBuilder();
+ dataPointBuilder.setAsDouble(42.0).setTimeUnixNano(TIMESTAMP);
+
+ for (int metricAttributeI = 0; metricAttributeI < 10; metricAttributeI++) {
+ KeyValue.Builder attributeBuilder = dataPointBuilder.addAttributesBuilder();
+ attributeBuilder.setKey("foo_key_" + metricAttributeI);
+ attributeBuilder.setValue(AnyValue.newBuilder().setStringValue("foo-value"));
+ }
+ }
+ }
+ }
+ }
+ return ByteBuffer.wrap(metricsData.build().toByteArray());
+ }
+
+ @Setup
+ public void init()
+ {
+ BUFFER = createMetricBuffer();
+ }
+
+ @Benchmark()
+ public void measureSerde(Blackhole blackhole) throws IOException
+ {
+ for (CloseableIterator it = INPUT_FORMAT.createReader(ROW_SCHEMA, new ByteEntity(BUFFER), null).read(); it.hasNext(); ) {
+ InputRow row = it.next();
+ blackhole.consume(row);
+ }
+ }
+}
diff --git a/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsInputFormatTest.java b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsInputFormatTest.java
new file mode 100644
index 000000000000..536247ab5716
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsInputFormatTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.InputFormat;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class OpenTelemetryMetricsInputFormatTest
+{
+ @Test
+ public void testSerde() throws Exception
+ {
+ OpenTelemetryMetricsProtobufInputFormat inputFormat = new OpenTelemetryMetricsProtobufInputFormat(
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ );
+
+ final ObjectMapper jsonMapper = new ObjectMapper();
+ jsonMapper.registerModules(new OpenTelemetryProtobufExtensionsModule().getJacksonModules());
+
+ final OpenTelemetryMetricsProtobufInputFormat actual = (OpenTelemetryMetricsProtobufInputFormat) jsonMapper.readValue(
+ jsonMapper.writeValueAsString(inputFormat),
+ InputFormat.class
+ );
+ Assert.assertEquals(inputFormat, actual);
+ Assert.assertEquals("metric.name", actual.getMetricDimension());
+ Assert.assertEquals("raw.value", actual.getValueDimension());
+ Assert.assertEquals("descriptor.", actual.getMetricAttributePrefix());
+ Assert.assertEquals("custom.", actual.getResourceAttributePrefix());
+ }
+
+ @Test
+ public void testDefaults()
+ {
+ OpenTelemetryMetricsProtobufInputFormat inputFormat = new OpenTelemetryMetricsProtobufInputFormat(
+ null,
+ null,
+ null,
+ null
+ );
+
+ Assert.assertEquals("metric", inputFormat.getMetricDimension());
+ Assert.assertEquals("value", inputFormat.getValueDimension());
+ Assert.assertEquals("", inputFormat.getMetricAttributePrefix());
+ Assert.assertEquals("resource.", inputFormat.getResourceAttributePrefix());
+ }
+}
diff --git a/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReaderTest.java b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReaderTest.java
new file mode 100644
index 000000000000..b5bca8bd7da4
--- /dev/null
+++ b/extensions-contrib/opentelemetry-extensions/src/test/java/org/apache/druid/data/input/opentelemetry/protobuf/OpenTelemetryMetricsProtobufReaderTest.java
@@ -0,0 +1,442 @@
+/*
+ * 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.data.input.opentelemetry.protobuf;
+
+import com.google.common.collect.ImmutableList;
+import io.opentelemetry.proto.common.v1.AnyValue;
+import io.opentelemetry.proto.common.v1.KeyValue;
+import io.opentelemetry.proto.common.v1.KeyValueList;
+import io.opentelemetry.proto.metrics.v1.DataPointFlags;
+import io.opentelemetry.proto.metrics.v1.Metric;
+import io.opentelemetry.proto.metrics.v1.MetricsData;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.data.input.impl.ByteEntity;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.indexing.seekablestream.SettableByteEntity;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+import org.apache.druid.java.util.common.parsers.ParseException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.concurrent.TimeUnit;
+
+public class OpenTelemetryMetricsProtobufReaderTest
+{
+ private static final long TIMESTAMP = TimeUnit.MILLISECONDS.toNanos(Instant.parse("2019-07-12T09:30:01.123Z").toEpochMilli());
+ public static final String RESOURCE_ATTRIBUTE_COUNTRY = "country";
+ public static final String RESOURCE_ATTRIBUTE_VALUE_USA = "usa";
+
+ public static final String RESOURCE_ATTRIBUTE_ENV = "env";
+ public static final String RESOURCE_ATTRIBUTE_VALUE_DEVEL = "devel";
+
+ public static final String INSTRUMENTATION_SCOPE_NAME = "mock-instr-lib";
+ public static final String INSTRUMENTATION_SCOPE_VERSION = "1.0";
+
+ public static final String METRIC_ATTRIBUTE_COLOR = "color";
+ public static final String METRIC_ATTRIBUTE_VALUE_RED = "red";
+
+ public static final String METRIC_ATTRIBUTE_FOO_KEY = "foo_key";
+ public static final String METRIC_ATTRIBUTE_FOO_VAL = "foo_value";
+
+ private final MetricsData.Builder metricsDataBuilder = MetricsData.newBuilder();
+
+ private final Metric.Builder metricBuilder = metricsDataBuilder.addResourceMetricsBuilder()
+ .addScopeMetricsBuilder()
+ .addMetricsBuilder();
+
+ private final DimensionsSpec dimensionsSpec = new DimensionsSpec(ImmutableList.of(
+ new StringDimensionSchema("descriptor." + METRIC_ATTRIBUTE_COLOR),
+ new StringDimensionSchema("descriptor." + METRIC_ATTRIBUTE_FOO_KEY),
+ new StringDimensionSchema("custom." + RESOURCE_ATTRIBUTE_ENV),
+ new StringDimensionSchema("custom." + RESOURCE_ATTRIBUTE_COUNTRY)
+ ));
+
+ @Rule
+ public ExpectedException expectedException = ExpectedException.none();
+
+ @Before
+ public void setUp()
+ {
+ metricsDataBuilder
+ .getResourceMetricsBuilder(0)
+ .getResourceBuilder()
+ .addAttributes(KeyValue.newBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_COUNTRY)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_USA)));
+
+ metricsDataBuilder
+ .getResourceMetricsBuilder(0)
+ .getScopeMetricsBuilder(0)
+ .getScopeBuilder()
+ .setName(INSTRUMENTATION_SCOPE_NAME)
+ .setVersion(INSTRUMENTATION_SCOPE_VERSION);
+
+ }
+
+ @Test
+ public void testSumWithAttributes()
+ {
+ metricBuilder
+ .setName("example_sum")
+ .getSumBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ List rowList = new ArrayList<>();
+ rows.forEachRemaining(rowList::add);
+ Assert.assertEquals(1, rowList.size());
+
+ InputRow row = rowList.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_sum");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "raw.value", "6");
+ }
+
+ @Test
+ public void testGaugeWithAttributes()
+ {
+ metricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "raw.value", "6");
+ }
+
+ @Test
+ public void testBatchedMetricParse()
+ {
+ metricBuilder.setName("example_sum")
+ .getSumBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build());
+
+ // Create Second Metric
+ Metric.Builder gaugeMetricBuilder = metricsDataBuilder.addResourceMetricsBuilder()
+ .addScopeMetricsBuilder()
+ .addMetricsBuilder();
+
+ metricsDataBuilder.getResourceMetricsBuilder(1)
+ .getResourceBuilder()
+ .addAttributes(KeyValue.newBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_ENV)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_DEVEL))
+ .build());
+
+ metricsDataBuilder.getResourceMetricsBuilder(1)
+ .getScopeMetricsBuilder(0)
+ .getScopeBuilder()
+ .setName(INSTRUMENTATION_SCOPE_NAME)
+ .setVersion(INSTRUMENTATION_SCOPE_VERSION);
+
+ gaugeMetricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(8)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAttributesBuilder() // test sum with attributes
+ .setKey(METRIC_ATTRIBUTE_FOO_KEY)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_FOO_VAL).build());
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_sum");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+ assertDimensionEquals(row, "raw.value", "6");
+
+ Assert.assertTrue(rows.hasNext());
+ row = rows.next();
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "custom.env", "devel");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ assertDimensionEquals(row, "raw.value", "8");
+
+ }
+
+ @Test
+ public void testDimensionSpecExclusions()
+ {
+ metricsDataBuilder.getResourceMetricsBuilder(0)
+ .getResourceBuilder()
+ .addAttributesBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_ENV)
+ .setValue(AnyValue.newBuilder().setStringValue(RESOURCE_ATTRIBUTE_VALUE_DEVEL).build());
+
+ metricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAllAttributes(ImmutableList.of(
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build()).build(),
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_FOO_KEY)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_FOO_VAL).build()).build()));
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ DimensionsSpec dimensionsSpecWithExclusions = DimensionsSpec.builder().setDimensionExclusions(ImmutableList.of(
+ "descriptor." + METRIC_ATTRIBUTE_COLOR,
+ "custom." + RESOURCE_ATTRIBUTE_COUNTRY
+ )).build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpecWithExclusions,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ Assert.assertTrue(rows.hasNext());
+ InputRow row = rows.next();
+
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_gauge");
+ assertDimensionEquals(row, "raw.value", "6");
+ assertDimensionEquals(row, "custom.env", "devel");
+ assertDimensionEquals(row, "descriptor.foo_key", "foo_value");
+ Assert.assertFalse(row.getDimensions().contains("custom.country"));
+ Assert.assertFalse(row.getDimensions().contains("descriptor.color"));
+ }
+
+ @Test
+ public void testUnsupportedValueTypes()
+ {
+ KeyValueList kvList = KeyValueList.newBuilder()
+ .addValues(
+ KeyValue.newBuilder()
+ .setKey("foo")
+ .setValue(AnyValue.newBuilder().setStringValue("bar").build()))
+ .build();
+
+ metricsDataBuilder.getResourceMetricsBuilder(0)
+ .getResourceBuilder()
+ .addAttributesBuilder()
+ .setKey(RESOURCE_ATTRIBUTE_ENV)
+ .setValue(AnyValue.newBuilder().setKvlistValue(kvList).build());
+
+ metricBuilder
+ .setName("example_sum")
+ .getSumBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setTimeUnixNano(TIMESTAMP)
+ .addAllAttributes(ImmutableList.of(
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_COLOR)
+ .setValue(AnyValue.newBuilder().setStringValue(METRIC_ATTRIBUTE_VALUE_RED).build()).build(),
+ KeyValue.newBuilder()
+ .setKey(METRIC_ATTRIBUTE_FOO_KEY)
+ .setValue(AnyValue.newBuilder().setKvlistValue(kvList).build()).build()));
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ List rowList = new ArrayList<>();
+ rows.forEachRemaining(rowList::add);
+ Assert.assertEquals(1, rowList.size());
+
+ InputRow row = rowList.get(0);
+ Assert.assertEquals(4, row.getDimensions().size());
+ assertDimensionEquals(row, "metric.name", "example_sum");
+ assertDimensionEquals(row, "custom.country", "usa");
+ assertDimensionEquals(row, "descriptor.color", "red");
+
+ // Unsupported resource attribute type is omitted
+ Assert.assertEquals(0, row.getDimension("custom.env").size());
+
+ // Unsupported metric attribute type is omitted
+ Assert.assertEquals(0, row.getDimension("descriptor.foo_key").size());
+
+ assertDimensionEquals(row, "raw.value", "6");
+ }
+
+ @Test
+ public void testInvalidProtobuf()
+ {
+ byte[] invalidProtobuf = new byte[] {0x00, 0x01};
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(invalidProtobuf));
+ try (CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read()) {
+ Assert.assertThrows(ParseException.class, () -> rows.hasNext());
+ Assert.assertThrows(NoSuchElementException.class, () -> rows.next());
+ }
+ catch (IOException e) {
+ // Comes from the implicit call to close. Ignore
+ }
+ }
+
+ @Test
+ public void testInvalidMetricType()
+ {
+ metricBuilder
+ .setName("unsupported_histogram_metric")
+ .getExponentialHistogramBuilder()
+ .addDataPointsBuilder()
+ .setTimeUnixNano(TIMESTAMP);
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ List rowList = new ArrayList<>();
+ rows.forEachRemaining(rowList::add);
+ Assert.assertEquals(0, rowList.size());
+ }
+
+ @Test
+ public void testNoRecordedValueMetric()
+ {
+ metricBuilder.setName("example_gauge")
+ .getGaugeBuilder()
+ .addDataPointsBuilder()
+ .setAsInt(6)
+ .setFlags(DataPointFlags.FLAG_NO_RECORDED_VALUE_VALUE)
+ .setTimeUnixNano(TIMESTAMP);
+
+ MetricsData metricsData = metricsDataBuilder.build();
+
+ SettableByteEntity settableByteEntity = new SettableByteEntity<>();
+ settableByteEntity.setEntity(new ByteEntity(metricsData.toByteArray()));
+ CloseableIterator rows = new OpenTelemetryMetricsProtobufReader(
+ dimensionsSpec,
+ settableByteEntity,
+ "metric.name",
+ "raw.value",
+ "descriptor.",
+ "custom."
+ ).read();
+
+ Assert.assertFalse(rows.hasNext());
+ }
+
+ private void assertDimensionEquals(InputRow row, String dimension, Object expected)
+ {
+ List values = row.getDimension(dimension);
+ Assert.assertEquals(1, values.size());
+ Assert.assertEquals(expected, values.get(0));
+ }
+
+}
diff --git a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json
index cb646f056513..74d26cd081a1 100644
--- a/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json
+++ b/extensions-contrib/opentsdb-emitter/src/main/resources/defaultMetrics.json
@@ -65,6 +65,9 @@
"ingest/events/processed": [
"dataSource"
],
+ "ingest/events/filtered": [
+ "dataSource"
+ ],
"ingest/rows/output": [
"dataSource"
],
@@ -224,4 +227,4 @@
"sys/cpu": [],
"coordinator-segment/count": [],
"historical-segment/count": []
-}
\ No newline at end of file
+}
diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json
index 59c9de66adb7..c9e31eb01360 100644
--- a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json
+++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json
@@ -1,5 +1,6 @@
{
"query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete a query."},
+"query/planningTime" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete query planning at Broker before query is fanned out to the Data nodes."},
"query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count", "help": "Number of bytes returned in query response."},
"query/node/time" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual historical/realtime processes."},
"query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Time to first byte. Seconds elapsed until Broker starts receiving the response from individual historical/realtime processes."},
@@ -46,6 +47,7 @@
"ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."},
"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/filtered" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected by header-based filtering before parsing." },
"ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission."},
"ingest/notices/queueSize" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of pending notices to be processed by the coordinator."},
"ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."},
diff --git a/extensions-contrib/rabbit-stream-indexing-service/pom.xml b/extensions-contrib/rabbit-stream-indexing-service/pom.xml
index 179c59f0de37..6b1ebade7f1c 100644
--- a/extensions-contrib/rabbit-stream-indexing-service/pom.xml
+++ b/extensions-contrib/rabbit-stream-indexing-service/pom.xml
@@ -117,6 +117,16 @@
com.rabbitmq
stream-client
0.15.0
+
+
+ org.lz4
+ lz4-java
+
+
+
+
+ at.yawk.lz4
+ lz4-java
jakarta.validation
@@ -175,4 +185,18 @@
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
+
+
diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json
index ad065c63d39a..629cb3a0154e 100644
--- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json
+++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json
@@ -1,5 +1,6 @@
{
"query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer"},
+ "query/planningTime" : { "dimensions" : ["dataSource", "type"], "type" : "timer"},
"query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count"},
"query/node/time" : { "dimensions" : ["server"], "type" : "timer"},
"query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer"},
@@ -43,6 +44,7 @@
"ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count" },
"ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count" },
"ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count" },
+ "ingest/events/filtered" : { "dimensions" : ["dataSource"], "type" : "count" },
"ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge" },
"ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count" },
"ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count" },
@@ -189,5 +191,7 @@
"killTask/availableSlot/count" : { "dimensions" : [], "type" : "count" },
"killTask/maxSlot/count" : { "dimensions" : [], "type" : "count" },
- "killTask/task/count" : { "dimensions" : [], "type" : "count" }
+ "killTask/task/count" : { "dimensions" : [], "type" : "count" },
+
+ "router/http/numRequestsQueued" : { "dimensions" : [], "type" : "gauge" }
}
diff --git a/extensions-core/avro-extensions/pom.xml b/extensions-core/avro-extensions/pom.xml
index 3cc366087ac0..fa44b0841063 100644
--- a/extensions-core/avro-extensions/pom.xml
+++ b/extensions-core/avro-extensions/pom.xml
@@ -189,8 +189,16 @@
jakarta.ws.rs
jakarta.ws.rs-api
+
+ org.lz4
+ lz4-java
+
+
+ at.yawk.lz4
+ lz4-java
+
com.google.code.findbugs
jsr305
@@ -327,6 +335,16 @@
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java
index b3b52838bb3a..2c32ee231a49 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java
@@ -153,6 +153,9 @@ public AuthenticationResult validateCredentials(
char[] password
)
{
+ if (username.isEmpty() || password.length == 0) {
+ return null;
+ }
final SearchResult userResult;
final LdapName userDn;
final Map contextMap = new HashMap<>();
diff --git a/extensions-core/druid-pac4j/pom.xml b/extensions-core/druid-pac4j/pom.xml
index 1f95186c19af..2e0c13ade7f9 100644
--- a/extensions-core/druid-pac4j/pom.xml
+++ b/extensions-core/druid-pac4j/pom.xml
@@ -34,12 +34,12 @@
- 4.5.7
+ 5.7.3
1.7
- 9.37.2
- 8.22
+ 10.0.2
+ 10.8
@@ -73,6 +73,13 @@
+
+
+ org.pac4j
+ pac4j-javaee
+ ${pac4j.version}
+
+
com.nimbusds
lang-tag
diff --git a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jAuthenticator.java b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jAuthenticator.java
index b63fcdf72775..ef30f4c7e69d 100644
--- a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jAuthenticator.java
+++ b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jAuthenticator.java
@@ -82,6 +82,7 @@ public Filter getFilter()
name,
authorizerName,
pac4jConfigSupplier.get(),
+ Pac4jCallbackResource.SELF_URL,
pac4jCommonConfig.getCookiePassphrase().getPassword()
);
}
diff --git a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java
index 0495242835c4..c992fd29dfb3 100644
--- a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java
+++ b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jFilter.java
@@ -23,14 +23,11 @@
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.server.security.AuthenticationResult;
import org.pac4j.core.config.Config;
-import org.pac4j.core.context.JEEContext;
-import org.pac4j.core.context.session.SessionStore;
-import org.pac4j.core.engine.CallbackLogic;
import org.pac4j.core.engine.DefaultCallbackLogic;
import org.pac4j.core.engine.DefaultSecurityLogic;
-import org.pac4j.core.engine.SecurityLogic;
-import org.pac4j.core.http.adapter.JEEHttpActionAdapter;
-import org.pac4j.core.profile.UserProfile;
+import org.pac4j.core.exception.http.HttpAction;
+import org.pac4j.jee.context.JEEContext;
+import org.pac4j.jee.http.adapter.JEEHttpActionAdapter;
import javax.servlet.Filter;
import javax.servlet.FilterChain;
@@ -41,30 +38,30 @@
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
-import java.util.Collection;
public class Pac4jFilter implements Filter
{
private static final Logger LOGGER = new Logger(Pac4jFilter.class);
private final Config pac4jConfig;
- private final SecurityLogic securityLogic;
- private final CallbackLogic callbackLogic;
- private final SessionStore sessionStore;
-
+ private final Pac4jSessionStore sessionStore;
+ private final String callbackPath;
private final String name;
private final String authorizerName;
- public Pac4jFilter(String name, String authorizerName, Config pac4jConfig, String cookiePassphrase)
+ public Pac4jFilter(
+ String name,
+ String authorizerName,
+ Config pac4jConfig,
+ String callbackPath,
+ String cookiePassphrase
+ )
{
this.pac4jConfig = pac4jConfig;
- this.securityLogic = new DefaultSecurityLogic<>();
- this.callbackLogic = new DefaultCallbackLogic<>();
-
+ this.callbackPath = callbackPath;
this.name = name;
this.authorizerName = authorizerName;
-
- this.sessionStore = new Pac4jSessionStore<>(cookiePassphrase);
+ this.sessionStore = new Pac4jSessionStore(cookiePassphrase);
}
@Override
@@ -72,7 +69,6 @@ public void init(FilterConfig filterConfig)
{
}
-
@Override
public void doFilter(ServletRequest servletRequest, ServletResponse servletResponse, FilterChain filterChain)
throws IOException, ServletException
@@ -84,38 +80,59 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo
return;
}
- HttpServletRequest httpServletRequest = (HttpServletRequest) servletRequest;
- HttpServletResponse httpServletResponse = (HttpServletResponse) servletResponse;
- JEEContext context = new JEEContext(httpServletRequest, httpServletResponse, sessionStore);
+ HttpServletRequest request = (HttpServletRequest) servletRequest;
+ HttpServletResponse response = (HttpServletResponse) servletResponse;
+ JEEContext context = new JEEContext(request, response);
+
+ if (request.getRequestURI().equals(callbackPath)) {
+ DefaultCallbackLogic callbackLogic = new DefaultCallbackLogic();
+ String originalUrl = (String) request.getSession().getAttribute("pac4j.originalUrl");
+ String redirectUrl = originalUrl != null ? originalUrl : "/";
- if (Pac4jCallbackResource.SELF_URL.equals(httpServletRequest.getRequestURI())) {
callbackLogic.perform(
context,
+ sessionStore,
pac4jConfig,
JEEHttpActionAdapter.INSTANCE,
- "/",
- true, false, false, null);
+ redirectUrl, // Redirect to original URL or root
+ null,
+ null
+ );
} else {
- Object uid = securityLogic.perform(
- context,
- pac4jConfig,
- (JEEContext ctx, Collection profiles, Object... parameters) -> {
- if (profiles.isEmpty()) {
- LOGGER.warn("No profiles found after OIDC auth.");
+ DefaultSecurityLogic securityLogic = new DefaultSecurityLogic();
+ try {
+ securityLogic.perform(
+ context,
+ sessionStore,
+ pac4jConfig,
+ (ctx, session, profiles, parameters) -> {
+ try {
+ // Extract user ID from pac4j profiles and create AuthenticationResult
+ if (profiles != null && !profiles.isEmpty()) {
+ String uid = profiles.iterator().next().getId();
+ if (uid != null) {
+ AuthenticationResult authenticationResult = new AuthenticationResult(uid, authorizerName, name, null);
+ servletRequest.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult);
+ filterChain.doFilter(servletRequest, servletResponse);
+ }
+ } else {
+ LOGGER.warn("No profiles found after OIDC auth.");
+ // Don't continue the filter chain - let pac4j handle the authentication failure
+ }
+ }
+ catch (IOException | ServletException e) {
+ throw new RuntimeException(e);
+ }
return null;
- } else {
- return profiles.iterator().next().getId();
- }
- },
- JEEHttpActionAdapter.INSTANCE,
- null, "none", null, null);
- // Changed the Authorizer from null to "none".
- // In the older version, if it is null, it simply grant access and returns authorized.
- // But in the newer pac4j version, it uses CsrfAuthorizer as default, And because of this, It was returning 403 in API calls.
- if (uid != null) {
- AuthenticationResult authenticationResult = new AuthenticationResult(uid.toString(), authorizerName, name, null);
- servletRequest.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult);
- filterChain.doFilter(servletRequest, servletResponse);
+ },
+ JEEHttpActionAdapter.INSTANCE,
+ null,
+ "none", // Use "none" instead of authorizerName to avoid CSRF issues
+ null
+ );
+ }
+ catch (HttpAction e) {
+ JEEHttpActionAdapter.INSTANCE.adapt(e, context);
}
}
}
diff --git a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jSessionStore.java b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jSessionStore.java
index b0187d5e7293..19ac98cb486a 100644
--- a/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jSessionStore.java
+++ b/extensions-core/druid-pac4j/src/main/java/org/apache/druid/security/pac4j/Pac4jSessionStore.java
@@ -19,71 +19,77 @@
package org.apache.druid.security.pac4j;
-import org.apache.commons.io.IOUtils;
+import com.google.common.base.Preconditions;
+import com.google.common.io.ByteStreams;
import org.apache.druid.crypto.CryptoService;
+import org.apache.druid.error.InvalidInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
-import org.pac4j.core.context.ContextHelper;
-import org.pac4j.core.context.Cookie;
import org.pac4j.core.context.WebContext;
import org.pac4j.core.context.session.SessionStore;
-import org.pac4j.core.exception.TechnicalException;
import org.pac4j.core.profile.CommonProfile;
-import org.pac4j.core.util.JavaSerializationHelper;
import org.pac4j.core.util.Pac4jConstants;
+import org.pac4j.jee.context.JEEContext;
+import org.pac4j.jee.context.session.JEESessionStore;
import javax.annotation.Nullable;
+import javax.servlet.http.Cookie;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
import java.io.Serializable;
+import java.util.Collection;
import java.util.Map;
import java.util.Optional;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
/**
- * Code here is slight adaptation from KnoxSessionStore
- * for storing oauth session information in cookies.
+ * Code here is slight adaptation from Apache Knox KnoxSessionStore
+ * for storing oauth session information in encrypted cookies.
*/
-public class Pac4jSessionStore implements SessionStore
+public class Pac4jSessionStore implements SessionStore
{
-
private static final Logger LOGGER = new Logger(Pac4jSessionStore.class);
public static final String PAC4J_SESSION_PREFIX = "pac4j.session.";
- private final JavaSerializationHelper javaSerializationHelper;
+ private final JEESessionStore delegate = JEESessionStore.INSTANCE;
private final CryptoService cryptoService;
public Pac4jSessionStore(String cookiePassphrase)
{
- javaSerializationHelper = new JavaSerializationHelper();
- cryptoService = new CryptoService(
- cookiePassphrase,
- "AES",
- "CBC",
- "PKCS5Padding",
- "PBKDF2WithHmacSHA256",
- 128,
- 65536,
- 128
+ this.cryptoService = new CryptoService(
+ cookiePassphrase,
+ "AES",
+ "CBC",
+ "PKCS5Padding",
+ "PBKDF2WithHmacSHA256",
+ 128,
+ 65536,
+ 128
);
}
@Override
- public String getOrCreateSessionId(WebContext context)
+ public Optional getSessionId(WebContext context, boolean createSession)
{
- return null;
+ if (context instanceof JEEContext) {
+ return delegate.getSessionId(context, createSession);
+ }
+ return Optional.empty();
}
- @Nullable
@Override
public Optional get(WebContext context, String key)
{
- final Cookie cookie = ContextHelper.getCookie(context, PAC4J_SESSION_PREFIX + key);
+ final Cookie cookie = getCookie(context, PAC4J_SESSION_PREFIX + key);
Object value = null;
- if (cookie != null) {
+ if (cookie != null && cookie.getValue() != null) {
value = uncompressDecryptBase64(cookie.getValue());
}
LOGGER.debug("Get from session: [%s] = [%s]", key, value);
@@ -96,37 +102,103 @@ public void set(WebContext context, String key, @Nullable Object value)
Object profile = value;
Cookie cookie;
- if (value == null) {
- cookie = new Cookie(PAC4J_SESSION_PREFIX + key, null);
+ // Check if value is null, empty string, or empty collection
+ boolean isEmpty = value == null ||
+ (value instanceof String && ((String) value).isEmpty()) ||
+ (value instanceof Collection && ((Collection>) value).isEmpty()) ||
+ (value instanceof Map && ((Map, ?>) value).isEmpty());
+
+ if (isEmpty) {
+ cookie = new Cookie(PAC4J_SESSION_PREFIX + key, "");
+ cookie.setMaxAge(0);
} else {
- if (key.contentEquals(Pac4jConstants.USER_PROFILES)) {
+ if (Pac4jConstants.USER_PROFILES.equals(key)) {
/* trim the profile object */
profile = clearUserProfile(value);
}
LOGGER.debug("Save in session: [%s] = [%s]", key, profile);
- cookie = new Cookie(
- PAC4J_SESSION_PREFIX + key,
- compressEncryptBase64(profile)
- );
+
+ String encryptedValue = compressEncryptBase64(profile);
+ cookie = new Cookie(PAC4J_SESSION_PREFIX + key, encryptedValue);
+ cookie.setMaxAge(900); // 15 minutes
}
- cookie.setDomain("");
cookie.setHttpOnly(true);
- cookie.setSecure(ContextHelper.isHttpsOrSecure(context));
+ // Always set secure flag for authentication cookies to prevent transmission over HTTP
+ // This ensures the cookie is only sent over HTTPS connections
+ boolean isSecure = isHttpsOrSecure(context);
+ if (!isSecure) {
+ LOGGER.warn("Setting authentication cookie over non-HTTPS connection. This is not recommended for production.");
+ }
+ cookie.setSecure(true); // Always set secure flag for authentication cookies
cookie.setPath("/");
- cookie.setMaxAge(900);
- context.addResponseCookie(cookie);
+ if (context instanceof JEEContext) {
+ JEEContext jeeContext = (JEEContext) context;
+ HttpServletResponse response = jeeContext.getNativeResponse();
+ response.addCookie(cookie);
+ // Only delegate to JEESessionStore if we have a JEEContext
+ delegate.set(context, key, value);
+ } else {
+ // For non-JEE contexts (like test mocks), add cookie to response
+ org.pac4j.core.context.Cookie pac4jCookie = new org.pac4j.core.context.Cookie(
+ cookie.getName(), cookie.getValue()
+ );
+ pac4jCookie.setHttpOnly(cookie.isHttpOnly());
+ pac4jCookie.setSecure(cookie.getSecure());
+ pac4jCookie.setMaxAge(cookie.getMaxAge());
+ pac4jCookie.setPath(cookie.getPath());
+ if (cookie.getDomain() != null) {
+ pac4jCookie.setDomain(cookie.getDomain());
+ }
+ context.addResponseCookie(pac4jCookie);
+ }
+ }
+
+ @Override
+ public boolean destroySession(WebContext context)
+ {
+ if (context instanceof JEEContext) {
+ return delegate.destroySession(context);
+ }
+ return false;
+ }
+
+ @Override
+ public Optional getTrackableSession(WebContext context)
+ {
+ if (context instanceof JEEContext) {
+ return delegate.getTrackableSession(context);
+ }
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional buildFromTrackableSession(WebContext context, Object trackableSession)
+ {
+ if (context instanceof JEEContext) {
+ return delegate.buildFromTrackableSession(context, trackableSession);
+ }
+ return Optional.empty();
+ }
+
+ @Override
+ public boolean renewSession(WebContext context)
+ {
+ if (context instanceof JEEContext) {
+ return delegate.renewSession(context);
+ }
+ return false;
}
@Nullable
private String compressEncryptBase64(final Object o)
{
if (o == null || "".equals(o)
- || (o instanceof Map, ?> && ((Map, ?>) o).isEmpty())) {
+ || (o instanceof Map, ?> && ((Map, ?>) o).isEmpty())) {
return null;
} else {
- byte[] bytes = javaSerializationHelper.serializeToBytes((Serializable) o);
+ byte[] bytes = serializeToBytes((Serializable) o);
bytes = compress(bytes);
if (bytes.length > 3000) {
@@ -141,9 +213,15 @@ private String compressEncryptBase64(final Object o)
private Serializable uncompressDecryptBase64(final String v)
{
if (v != null && !v.isEmpty()) {
- byte[] bytes = StringUtils.decodeBase64String(v);
- if (bytes != null) {
- return javaSerializationHelper.deserializeFromBytes(unCompress(cryptoService.decrypt(bytes)));
+ try {
+ byte[] bytes = StringUtils.decodeBase64String(v);
+ if (bytes != null) {
+ return deserializeFromBytes(uncompress(cryptoService.decrypt(bytes)));
+ }
+ }
+ catch (Exception e) {
+ LOGGER.debug("Failed to decrypt cookie value: %s", e.getMessage());
+ throw InvalidInput.exception(e, "Decryption failed. Check service logs.");
}
}
return null;
@@ -158,55 +236,140 @@ private byte[] compress(final byte[] data)
return byteStream.toByteArray();
}
catch (IOException ex) {
- throw new TechnicalException(ex);
+ throw new RuntimeException("Compression failed", ex);
}
}
- private byte[] unCompress(final byte[] data)
+ private byte[] uncompress(final byte[] data)
{
try (ByteArrayInputStream inputStream = new ByteArrayInputStream(data);
GZIPInputStream gzip = new GZIPInputStream(inputStream)) {
- return IOUtils.toByteArray(gzip);
+ return ByteStreams.toByteArray(gzip);
}
catch (IOException ex) {
- throw new TechnicalException(ex);
+ throw new RuntimeException("Decompression failed", ex);
}
}
- private Object clearUserProfile(final Object value)
+ /**
+ * Serialize object using standard Java serialization
+ */
+ private byte[] serializeToBytes(Serializable obj)
{
- if (value instanceof Map, ?>) {
- final Map profiles = (Map) value;
- profiles.forEach((name, profile) -> profile.removeLoginData());
- return profiles;
- } else {
- final CommonProfile profile = (CommonProfile) value;
- profile.removeLoginData();
- return profile;
+ Preconditions.checkNotNull(obj, "Object to serialize cannot be null");
+
+ try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ ObjectOutputStream oos = new ObjectOutputStream(baos)) {
+ oos.writeObject(obj);
+ oos.flush();
+ return baos.toByteArray();
+ }
+ catch (IOException e) {
+ throw new RuntimeException("Failed to serialize object", e);
}
}
- @Override
- public Optional> buildFromTrackableSession(WebContext arg0, Object arg1)
+ /**
+ * Deserialize object using standard Java serialization
+ */
+ private Serializable deserializeFromBytes(byte[] data)
{
- return Optional.empty();
+ Preconditions.checkNotNull(data, "Data to deserialize cannot be null");
+
+ try (ByteArrayInputStream bais = new ByteArrayInputStream(data);
+ ObjectInputStream ois = new ObjectInputStream(bais)) {
+ return (Serializable) ois.readObject();
+ }
+ catch (IOException | ClassNotFoundException e) {
+ throw new RuntimeException("Failed to deserialize object", e);
+ }
}
- @Override
- public boolean destroySession(WebContext arg0)
+ /**
+ * Clear sensitive data from user profiles before storing in cookies
+ */
+ private Object clearUserProfile(final Object value)
{
- return false;
+ if (value instanceof Map, ?>) {
+ final Map profiles = (Map) value;
+ profiles.forEach((name, profile) -> {
+ // In pac4j 5.x, we need to manually clear sensitive data
+ // since removeLoginData() is no longer available
+ if (profile != null) {
+ profile.removeAttribute("access_token");
+ profile.removeAttribute("refresh_token");
+ profile.removeAttribute("id_token");
+ profile.removeAttribute("credentials");
+ }
+ });
+ return profiles;
+ } else if (value instanceof CommonProfile) {
+ final CommonProfile profile = (CommonProfile) value;
+ profile.removeAttribute("access_token");
+ profile.removeAttribute("refresh_token");
+ profile.removeAttribute("id_token");
+ profile.removeAttribute("credentials");
+ return profile;
+ }
+ return value;
}
- @Override
- public Optional getTrackableSession(WebContext arg0)
+ /**
+ * Get cookie from request - replacement for ContextHelper.getCookie
+ */
+ private Cookie getCookie(WebContext context, String name)
{
- return Optional.empty();
+ if (context instanceof JEEContext) {
+ JEEContext jeeContext = (JEEContext) context;
+ HttpServletRequest request = jeeContext.getNativeRequest();
+ Cookie[] cookies = request.getCookies();
+ if (cookies != null) {
+ for (Cookie cookie : cookies) {
+ if (name.equals(cookie.getName())) {
+ return cookie;
+ }
+ }
+ }
+ } else {
+ // For non-JEE contexts (like test mocks), check if context supports cookies
+ if (context != null) {
+ Collection requestCookies = context.getRequestCookies();
+ if (requestCookies != null) {
+ for (org.pac4j.core.context.Cookie cookie : requestCookies) {
+ if (name.equals(cookie.getName())) {
+ // Convert pac4j Cookie to javax.servlet.http.Cookie
+ Cookie servletCookie = new Cookie(cookie.getName(), cookie.getValue());
+ servletCookie.setHttpOnly(cookie.isHttpOnly());
+ servletCookie.setSecure(cookie.isSecure());
+ servletCookie.setMaxAge(cookie.getMaxAge());
+ if (cookie.getPath() != null) {
+ servletCookie.setPath(cookie.getPath());
+ }
+ if (cookie.getDomain() != null) {
+ servletCookie.setDomain(cookie.getDomain());
+ }
+ return servletCookie;
+ }
+ }
+ }
+ }
+ }
+ return null;
}
- @Override
- public boolean renewSession(final WebContext context)
+ /**
+ * Check if connection is secure - replacement for ContextHelper.isHttpsOrSecure
+ */
+ private boolean isHttpsOrSecure(WebContext context)
{
- return false;
+ if (context instanceof JEEContext) {
+ JEEContext jeeContext = (JEEContext) context;
+ HttpServletRequest request = jeeContext.getNativeRequest();
+ return request.isSecure() ||
+ "https".equalsIgnoreCase(request.getScheme()) ||
+ "https".equalsIgnoreCase(request.getHeader("X-Forwarded-Proto"));
+ }
+ // For non-JEE contexts (like test mocks), check the scheme
+ return "https".equalsIgnoreCase(context.getScheme());
}
}
diff --git a/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jFilterTest.java b/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jFilterTest.java
index 0523c970178d..d1534175fcbf 100644
--- a/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jFilterTest.java
+++ b/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jFilterTest.java
@@ -26,32 +26,146 @@
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
-import org.pac4j.core.context.JEEContext;
+import org.pac4j.core.config.Config;
import org.pac4j.core.exception.http.ForbiddenAction;
import org.pac4j.core.exception.http.FoundAction;
import org.pac4j.core.exception.http.HttpAction;
import org.pac4j.core.exception.http.WithLocationAction;
-import org.pac4j.core.http.adapter.JEEHttpActionAdapter;
+import org.pac4j.jee.context.JEEContext;
+import org.pac4j.jee.http.adapter.JEEHttpActionAdapter;
+import javax.servlet.FilterChain;
+import javax.servlet.ServletException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpSession;
+import java.io.IOException;
+import java.io.PrintWriter;
import static org.mockito.ArgumentMatchers.any;
@RunWith(MockitoJUnitRunner.class)
public class Pac4jFilterTest
{
+ private static final String DRUID_AUTHENTICATION_RESULT = "Druid-Authentication-Result";
@Mock
private HttpServletRequest request;
@Mock
private HttpServletResponse response;
+ @Mock
+ private PrintWriter printWriter;
+ @Mock
+ private FilterChain filterChain;
+ @Mock
+ private HttpSession session;
+ @Mock
+ private Config pac4jConfig;
+
private JEEContext context;
+ private Pac4jFilter pac4jFilter;
@Before
- public void setUp()
+ public void setUp() throws IOException
{
+ // Mock the PrintWriter for the response
+ Mockito.when(response.getWriter()).thenReturn(printWriter);
context = new JEEContext(request, response);
+ pac4jFilter = new Pac4jFilter("test", "testAuthorizer", pac4jConfig, "/callback", "testPassphrase");
+ }
+
+ @Test
+ public void testInit()
+ {
+ // Test that init method doesn't throw exceptions
+ pac4jFilter.init(null);
+ // No exception should be thrown
+ }
+
+ @Test
+ public void testDestroy()
+ {
+ // Test that destroy method doesn't throw exceptions
+ pac4jFilter.destroy();
+ // No exception should be thrown
+ }
+
+ @Test
+ public void testDoFilterWithAlreadyAuthenticatedRequest() throws IOException, ServletException
+ {
+ // Mock request with existing authentication result
+ Mockito.when(request.getAttribute(DRUID_AUTHENTICATION_RESULT)).thenReturn("already-authenticated");
+
+ // Call doFilter
+ pac4jFilter.doFilter(request, response, filterChain);
+
+ // Verify it continues the filter chain without doing authentication
+ Mockito.verify(filterChain).doFilter(request, response);
+ }
+
+ @Test
+ public void testDoFilterWithNullAuthenticationResult() throws IOException, ServletException
+ {
+ // Mock request without authentication result
+ Mockito.when(request.getAttribute(DRUID_AUTHENTICATION_RESULT)).thenReturn(null);
+ Mockito.when(request.getRequestURI()).thenReturn("/some/path");
+
+ // This will attempt to do authentication, which may throw due to missing pac4j config
+ // but we're mainly testing the basic flow
+ try {
+ pac4jFilter.doFilter(request, response, filterChain);
+ }
+ catch (Exception e) {
+ // Expected due to mock limitations, but we verified the basic flow
+ }
+
+ // Verify that the authentication attribute was checked
+ Mockito.verify(request).getAttribute(DRUID_AUTHENTICATION_RESULT);
+ Mockito.verify(request).getRequestURI();
+ }
+
+ @Test
+ public void testDoFilterWithCallbackPath() throws IOException, ServletException
+ {
+ // Mock request for callback path
+ Mockito.when(request.getAttribute(DRUID_AUTHENTICATION_RESULT)).thenReturn(null);
+ Mockito.when(request.getRequestURI()).thenReturn("/callback");
+ Mockito.when(request.getSession()).thenReturn(session);
+ Mockito.when(session.getAttribute("pac4j.originalUrl")).thenReturn("/original");
+
+ // This will attempt to do callback logic
+ try {
+ pac4jFilter.doFilter(request, response, filterChain);
+ }
+ catch (Exception e) {
+ // Expected due to mock limitations
+ }
+
+ // Verify that the callback path was detected
+ Mockito.verify(request).getRequestURI();
+ Mockito.verify(request).getSession();
+ }
+
+ @Test
+ public void testDoFilterWithCallbackPathAndNoOriginalUrl() throws IOException, ServletException
+ {
+ // Mock request for callback path without original URL
+ Mockito.when(request.getAttribute(DRUID_AUTHENTICATION_RESULT)).thenReturn(null);
+ Mockito.when(request.getRequestURI()).thenReturn("/callback");
+ Mockito.when(request.getSession()).thenReturn(session);
+ Mockito.when(session.getAttribute("pac4j.originalUrl")).thenReturn(null);
+
+ // This will attempt to do callback logic
+ try {
+ pac4jFilter.doFilter(request, response, filterChain);
+ }
+ catch (Exception e) {
+ // Expected due to mock limitations
+ }
+
+ // Verify that the callback path was detected and session was accessed
+ Mockito.verify(request).getRequestURI();
+ Mockito.verify(session).getAttribute("pac4j.originalUrl");
}
@Test
@@ -74,4 +188,11 @@ public void testActionAdapterForForbidden()
Assert.assertEquals(response.getStatus(), HttpServletResponse.SC_FORBIDDEN);
}
+ @Test
+ public void testFilterCreation()
+ {
+ // Test that filter can be created without exceptions
+ Pac4jFilter filter = new Pac4jFilter("testName", "testAuthorizer", pac4jConfig, "/test-callback", "testPassphrase");
+ Assert.assertNotNull(filter);
+ }
}
diff --git a/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jSessionStoreTest.java b/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jSessionStoreTest.java
index e2f46a4062ea..e342206b5a9c 100644
--- a/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jSessionStoreTest.java
+++ b/extensions-core/druid-pac4j/src/test/java/org/apache/druid/security/pac4j/Pac4jSessionStoreTest.java
@@ -19,7 +19,6 @@
package org.apache.druid.security.pac4j;
-import org.apache.druid.error.DruidException;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.junit.Assert;
@@ -27,7 +26,7 @@
import org.pac4j.core.context.Cookie;
import org.pac4j.core.context.WebContext;
import org.pac4j.core.profile.CommonProfile;
-import org.pac4j.core.profile.definition.CommonProfileDefinition;
+import org.pac4j.core.util.Pac4jConstants;
import java.util.Collections;
import java.util.HashMap;
@@ -42,7 +41,7 @@ public class Pac4jSessionStoreTest
@Test
public void testSetAndGet()
{
- Pac4jSessionStore sessionStore = new Pac4jSessionStore<>(COOKIE_PASSPHRASE);
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
WebContext webContext1 = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext1.getScheme()).andReturn("https");
@@ -56,20 +55,145 @@ public void testSetAndGet()
Cookie cookie = cookieCapture.getValue();
Assert.assertTrue(cookie.isSecure());
Assert.assertTrue(cookie.isHttpOnly());
- Assert.assertTrue(cookie.isSecure());
Assert.assertEquals(900, cookie.getMaxAge());
-
+ // For the get test, we need to mock the context to return the cookie
WebContext webContext2 = EasyMock.mock(WebContext.class);
+ // The get method will call getRequestCookies() once for each getCookie() call
EasyMock.expect(webContext2.getRequestCookies()).andReturn(Collections.singletonList(cookie));
EasyMock.replay(webContext2);
+
Assert.assertEquals("value", Objects.requireNonNull(sessionStore.get(webContext2, "key")).orElse(null));
+ EasyMock.verify(webContext2);
}
+ @Test
+ public void testSetAndGetWithHttpScheme()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext1 = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext1.getScheme()).andReturn("http");
+ Capture cookieCapture = EasyMock.newCapture();
+
+ webContext1.addResponseCookie(EasyMock.capture(cookieCapture));
+ EasyMock.replay(webContext1);
+
+ sessionStore.set(webContext1, "key", "value");
+
+ Cookie cookie = cookieCapture.getValue();
+ Assert.assertTrue(cookie.isSecure()); // Should still be secure due to our fix
+ Assert.assertTrue(cookie.isHttpOnly());
+ Assert.assertEquals(900, cookie.getMaxAge());
+
+ EasyMock.verify(webContext1);
+ }
+
+ @Test
+ public void testSetNullValue()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getScheme()).andReturn("https");
+ Capture cookieCapture = EasyMock.newCapture();
+
+ webContext.addResponseCookie(EasyMock.capture(cookieCapture));
+ EasyMock.replay(webContext);
+
+ sessionStore.set(webContext, "key", null);
+
+ Cookie cookie = cookieCapture.getValue();
+ Assert.assertTrue(cookie.isSecure());
+ Assert.assertTrue(cookie.isHttpOnly());
+ Assert.assertEquals(0, cookie.getMaxAge()); // Should be 0 for null values
+ Assert.assertEquals("", cookie.getValue());
+
+ EasyMock.verify(webContext);
+ }
+
+ @Test
+ public void testSetEmptyString()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getScheme()).andReturn("https");
+ Capture cookieCapture = EasyMock.newCapture();
+
+ webContext.addResponseCookie(EasyMock.capture(cookieCapture));
+ EasyMock.replay(webContext);
+
+ sessionStore.set(webContext, "key", "");
+
+ Cookie cookie = cookieCapture.getValue();
+ Assert.assertTrue(cookie.isSecure());
+ Assert.assertTrue(cookie.isHttpOnly());
+ Assert.assertEquals(0, cookie.getMaxAge()); // Should be 0 for empty string
+ Assert.assertEquals("", cookie.getValue());
+
+ EasyMock.verify(webContext);
+ }
+
+ @Test
+ public void testSetEmptyMap()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getScheme()).andReturn("https");
+ Capture cookieCapture = EasyMock.newCapture();
+
+ webContext.addResponseCookie(EasyMock.capture(cookieCapture));
+ EasyMock.replay(webContext);
+
+ sessionStore.set(webContext, "key", Collections.emptyMap());
+
+ Cookie cookie = cookieCapture.getValue();
+ Assert.assertTrue(cookie.isSecure());
+ Assert.assertTrue(cookie.isHttpOnly());
+ Assert.assertEquals(0, cookie.getMaxAge()); // Should be 0 for empty map
+ Assert.assertEquals("", cookie.getValue());
+
+ EasyMock.verify(webContext);
+ }
+
+ @Test
+ public void testGetWithNoCookies()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getRequestCookies()).andReturn(Collections.emptyList());
+ EasyMock.replay(webContext);
+
+ Optional result = sessionStore.get(webContext, "key");
+ Assert.assertFalse(result.isPresent());
+
+ EasyMock.verify(webContext);
+ }
+
+ @Test
+ public void testGetWithNullCookies()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getRequestCookies()).andReturn(null);
+ EasyMock.replay(webContext);
+
+ Optional result = sessionStore.get(webContext, "key");
+ Assert.assertFalse(result.isPresent());
+
+ EasyMock.verify(webContext);
+ }
+
+
+
@Test
public void testSetAndGetClearUserProfile()
{
- Pac4jSessionStore sessionStore = new Pac4jSessionStore<>(COOKIE_PASSPHRASE);
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
WebContext webContext1 = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext1.getScheme()).andReturn("https");
@@ -80,28 +204,41 @@ public void testSetAndGetClearUserProfile()
CommonProfile profile = new CommonProfile();
profile.setId("profile1");
- profile.addAttribute(CommonProfileDefinition.DISPLAY_NAME, "name");
- sessionStore.set(webContext1, "pac4jUserProfiles", profile);
+ profile.addAttribute("display_name", "name");
+ profile.addAttribute("access_token", "token");
+ profile.addAttribute("refresh_token", "refresh");
+ profile.addAttribute("id_token", "id");
+ profile.addAttribute("credentials", "creds");
+
+ sessionStore.set(webContext1, Pac4jConstants.USER_PROFILES, profile);
Cookie cookie = cookieCapture.getValue();
Assert.assertTrue(cookie.isSecure());
Assert.assertTrue(cookie.isHttpOnly());
- Assert.assertTrue(cookie.isSecure());
Assert.assertEquals(900, cookie.getMaxAge());
-
WebContext webContext2 = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext2.getRequestCookies()).andReturn(Collections.singletonList(cookie));
EasyMock.replay(webContext2);
- Optional value = sessionStore.get(webContext2, "pac4jUserProfiles");
+
+ Optional value = sessionStore.get(webContext2, Pac4jConstants.USER_PROFILES);
Assert.assertTrue(Objects.requireNonNull(value).isPresent());
- Assert.assertEquals("name", ((CommonProfile) value.get()).getAttribute(CommonProfileDefinition.DISPLAY_NAME));
+ CommonProfile retrievedProfile = (CommonProfile) value.get();
+ Assert.assertEquals("name", retrievedProfile.getAttribute("display_name"));
+
+ // Verify sensitive data was removed
+ Assert.assertNull(retrievedProfile.getAttribute("access_token"));
+ Assert.assertNull(retrievedProfile.getAttribute("refresh_token"));
+ Assert.assertNull(retrievedProfile.getAttribute("id_token"));
+ Assert.assertNull(retrievedProfile.getAttribute("credentials"));
+
+ EasyMock.verify(webContext2);
}
@Test
public void testSetAndGetClearUserMultipleProfile()
{
- Pac4jSessionStore sessionStore = new Pac4jSessionStore<>(COOKIE_PASSPHRASE);
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
WebContext webContext1 = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext1.getScheme()).andReturn("https");
@@ -112,58 +249,120 @@ public void testSetAndGetClearUserMultipleProfile()
CommonProfile profile1 = new CommonProfile();
profile1.setId("profile1");
- profile1.addAttribute(CommonProfileDefinition.DISPLAY_NAME, "name1");
+ profile1.addAttribute("display_name", "name1");
+ profile1.addAttribute("access_token", "token1");
+
CommonProfile profile2 = new CommonProfile();
profile2.setId("profile2");
- profile2.addAttribute(CommonProfileDefinition.DISPLAY_NAME, "name2");
+ profile2.addAttribute("display_name", "name2");
+ profile2.addAttribute("refresh_token", "refresh2");
+
Map profiles = new HashMap<>();
profiles.put("profile1", profile1);
profiles.put("profile2", profile2);
- sessionStore.set(webContext1, "pac4jUserProfiles", profiles);
+
+ sessionStore.set(webContext1, Pac4jConstants.USER_PROFILES, profiles);
Cookie cookie = cookieCapture.getValue();
Assert.assertTrue(cookie.isSecure());
Assert.assertTrue(cookie.isHttpOnly());
- Assert.assertTrue(cookie.isSecure());
Assert.assertEquals(900, cookie.getMaxAge());
-
WebContext webContext2 = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext2.getRequestCookies()).andReturn(Collections.singletonList(cookie));
EasyMock.replay(webContext2);
- Optional value = sessionStore.get(webContext2, "pac4jUserProfiles");
+
+ Optional value = sessionStore.get(webContext2, Pac4jConstants.USER_PROFILES);
Assert.assertTrue(Objects.requireNonNull(value).isPresent());
- Assert.assertEquals(2, ((Map) value.get()).size());
+ @SuppressWarnings("unchecked")
+ Map retrievedProfiles = (Map) value.get();
+ Assert.assertEquals(2, retrievedProfiles.size());
+
+ // Verify sensitive data was removed from both profiles
+ Assert.assertNull(retrievedProfiles.get("profile1").getAttribute("access_token"));
+ Assert.assertNull(retrievedProfiles.get("profile2").getAttribute("refresh_token"));
+
+ EasyMock.verify(webContext2);
}
@Test
- public void testGetWithWrongPassphraseThrowsDruidException()
+ public void testSessionStoreInterfaceMethods()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.replay(webContext);
+
+ // Test methods that return empty/false for non-JEE contexts
+ Assert.assertFalse(sessionStore.getSessionId(webContext, true).isPresent());
+ Assert.assertFalse(sessionStore.destroySession(webContext));
+ Assert.assertFalse(sessionStore.getTrackableSession(webContext).isPresent());
+ Assert.assertFalse(sessionStore.buildFromTrackableSession(webContext, "test").isPresent());
+ Assert.assertFalse(sessionStore.renewSession(webContext));
+
+ EasyMock.verify(webContext);
+ }
+
+ @Test
+ public void testGetWithWrongPassphraseThrowsException()
{
final WebContext webContext = EasyMock.mock(WebContext.class);
EasyMock.expect(webContext.getScheme()).andReturn("https");
final Capture cookieCapture = EasyMock.newCapture();
- EasyMock.expect(webContext.getRequestCookies())
- .andAnswer(() -> Collections.singleton(cookieCapture.getValue()));
-
webContext.addResponseCookie(EasyMock.capture(cookieCapture));
EasyMock.expectLastCall().once();
EasyMock.replay(webContext);
// Create a cookie with an invalid passphrase
- new Pac4jSessionStore<>("invalid-passphrase").set(webContext, "key", "value");
+ new Pac4jSessionStore("invalid-passphrase").set(webContext, "key", "value");
+
+ EasyMock.verify(webContext);
+
+ // Create a new mock for the get operation
+ final WebContext getContext = EasyMock.mock(WebContext.class);
+
+ // The captured cookie should have the correct name "pac4j.session.key"
+ Cookie capturedCookie = cookieCapture.getValue();
+ EasyMock.expect(getContext.getRequestCookies())
+ .andReturn(Collections.singletonList(capturedCookie));
+ EasyMock.replay(getContext);
// Verify that trying to decrypt the invalid cookie throws an exception
- final Pac4jSessionStore sessionStore = new Pac4jSessionStore<>(COOKIE_PASSPHRASE);
- DruidException exception = Assert.assertThrows(
- DruidException.class,
- () -> sessionStore.get(webContext, "key")
- );
- Assert.assertEquals(
- "Decryption failed. Check service logs.",
- exception.getMessage()
+ final Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+ RuntimeException exception = Assert.assertThrows(
+ RuntimeException.class,
+ () -> sessionStore.get(getContext, "key")
);
- Assert.assertNull(exception.getCause());
+ Assert.assertTrue(exception.getMessage().contains("Decryption failed"));
+ Assert.assertNotNull(exception.getCause());
+
+ EasyMock.verify(getContext);
+ }
+
+ @Test
+ public void testLargeCookieWarning()
+ {
+ Pac4jSessionStore sessionStore = new Pac4jSessionStore(COOKIE_PASSPHRASE);
+
+ WebContext webContext = EasyMock.mock(WebContext.class);
+ EasyMock.expect(webContext.getScheme()).andReturn("https");
+ Capture cookieCapture = EasyMock.newCapture();
+
+ webContext.addResponseCookie(EasyMock.capture(cookieCapture));
+ EasyMock.replay(webContext);
+
+ // Create a large object that will result in a big cookie
+ StringBuilder largeData = new StringBuilder();
+ for (int i = 0; i < 1000; i++) {
+ largeData.append("This is a large piece of data that will make the cookie very big. ");
+ }
+
+ sessionStore.set(webContext, "key", largeData.toString());
+
+ Cookie cookie = cookieCapture.getValue();
+ Assert.assertTrue(cookie.isSecure());
+ Assert.assertTrue(cookie.isHttpOnly());
+ Assert.assertEquals(900, cookie.getMaxAge());
EasyMock.verify(webContext);
}
diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml
index 2f6c08558c19..3f0a4891bbc8 100644
--- a/extensions-core/kafka-extraction-namespace/pom.xml
+++ b/extensions-core/kafka-extraction-namespace/pom.xml
@@ -63,8 +63,16 @@
net.jpountz.lz4
lz4
+
+ org.lz4
+ lz4-java
+
+
+ at.yawk.lz4
+ lz4-java
+
com.google.code.findbugs
jsr305
@@ -141,4 +149,19 @@
test
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
+
+
diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml
index 326d26b620ba..e9fbe7d7c388 100644
--- a/extensions-core/kafka-indexing-service/pom.xml
+++ b/extensions-core/kafka-indexing-service/pom.xml
@@ -71,8 +71,16 @@
net.jpountz.lz4
lz4
+
+ org.lz4
+ lz4-java
+
+
+ at.yawk.lz4
+ lz4-java
+
com.google.code.findbugs
jsr305
@@ -128,6 +136,11 @@
jakarta.validation-api
provided
+
+ com.github.ben-manes.caffeine
+ caffeine
+ provided
+
@@ -196,4 +209,18 @@
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
+
+
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java
index 41c2c0a03258..39149ce4e144 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafka/KafkaRecordEntity.java
@@ -20,6 +20,7 @@
package org.apache.druid.data.input.kafka;
import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.KafkaEntity;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.indexing.kafka.KafkaRecordSupplier;
import org.apache.kafka.clients.consumer.ConsumerRecord;
@@ -36,7 +37,7 @@
*
* This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions.
*/
-public class KafkaRecordEntity extends ByteEntity
+public class KafkaRecordEntity extends ByteEntity implements KafkaEntity
{
private final ConsumerRecord record;
@@ -50,4 +51,10 @@ public ConsumerRecord getRecord()
{
return record;
}
+
+ @Override
+ public long getRecordTimestampMillis()
+ {
+ return record.timestamp();
+ }
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
index 31b7cf66be19..f00d94d5d06a 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/data/input/kafkainput/KafkaInputReader.java
@@ -35,6 +35,7 @@
import org.joda.time.DateTime;
import javax.annotation.Nullable;
+
import java.io.IOException;
import java.util.AbstractMap;
import java.util.Collections;
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandler.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandler.java
new file mode 100644
index 000000000000..b67f11d5254d
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandler.java
@@ -0,0 +1,52 @@
+/*
+ * 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.kafka;
+
+/**
+ * Interface for handling different filter types in Kafka header evaluation.
+ *
+ * This provides an extensible way to support various Druid filter types for
+ * header-based filtering without modifying the core evaluation logic.
+ */
+public interface HeaderFilterHandler
+{
+ /**
+ * Gets the header name/key to evaluate from the filter.
+ *
+ * @return the header name to look for in Kafka message headers
+ */
+ String getHeaderName();
+
+ /**
+ * Evaluates whether a record should be included based on the header value.
+ *
+ * @param headerValue the decoded header value (guaranteed to be non-null when called)
+ * @return true if the record should be included, false if it should be filtered out
+ */
+ boolean shouldInclude(String headerValue);
+
+ /**
+ * Gets a human-readable description of this filter for logging and debugging.
+ *
+ * @return a descriptive string representation of the filter
+ */
+ String getDescription();
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerFactory.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerFactory.java
new file mode 100644
index 000000000000..224b2afe3cef
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.kafka;
+
+import org.apache.druid.query.filter.Filter;
+import org.apache.druid.query.filter.InDimFilter;
+
+/**
+ * Factory for creating HeaderFilterHandler instances.
+ *
+ * This factory uses explicit instanceof checks for clarity and performance,
+ * making it easy to add support for new filter types by simply adding
+ * new conditional branches.
+ */
+public final class HeaderFilterHandlerFactory
+{
+ private HeaderFilterHandlerFactory()
+ {
+ // Utility class - prevent instantiation
+ }
+
+ /**
+ * Creates the appropriate handler for the given filter.
+ *
+ * @param filter the Druid filter to create a handler for
+ * @return a HeaderFilterHandler that can evaluate the given filter type
+ * @throws IllegalArgumentException if the filter type is not supported
+ */
+ public static HeaderFilterHandler forFilter(Filter filter)
+ {
+ if (filter instanceof InDimFilter) {
+ return new InDimFilterHandler((InDimFilter) filter);
+ }
+
+ throw new IllegalArgumentException(
+ "Unsupported filter type for header filtering: " + filter.getClass().getSimpleName() +
+ ". Supported types: InDimFilter"
+ );
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/InDimFilterHandler.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/InDimFilterHandler.java
new file mode 100644
index 000000000000..35efe17480c0
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/InDimFilterHandler.java
@@ -0,0 +1,78 @@
+/*
+ * 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.kafka;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.filter.InDimFilter;
+
+import javax.annotation.Nullable;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Handler for InDimFilter in Kafka header evaluation.
+ *
+ * This handler evaluates whether a header value is contained in the filter's value set.
+ * It uses a HashSet for O(1) average-case lookup performance instead of the filter's
+ * internal TreeSet which has O(log n) lookup time.
+ */
+public class InDimFilterHandler implements HeaderFilterHandler
+{
+ private final InDimFilter filter;
+ private final Set filterValues;
+
+ /**
+ * Creates a new handler for the given InDimFilter.
+ *
+ * @param filter the InDimFilter to handle, must not be null
+ * @throws IllegalArgumentException if filter is null
+ */
+ public InDimFilterHandler(InDimFilter filter)
+ {
+ this.filter = Preconditions.checkNotNull(filter, "filter cannot be null");
+
+ // Convert to HashSet for O(1) lookups instead of O(log n) TreeSet lookups
+ // This optimization is particularly beneficial when the filter has many values
+ this.filterValues = new HashSet<>(filter.getValues());
+ }
+
+ @Override
+ public String getHeaderName()
+ {
+ return filter.getDimension();
+ }
+
+ @Override
+ public boolean shouldInclude(@Nullable String headerValue)
+ {
+ return filterValues.contains(headerValue);
+ }
+
+ @Override
+ public String getDescription()
+ {
+ return StringUtils.format(
+ "InDimFilter[header=%s, values=%d]",
+ filter.getDimension(),
+ filterValues.size()
+ );
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterEvaluator.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterEvaluator.java
new file mode 100644
index 000000000000..89be079424e7
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterEvaluator.java
@@ -0,0 +1,146 @@
+/*
+ * 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.kafka;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.filter.Filter;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.Header;
+import org.apache.kafka.common.header.Headers;
+
+import javax.annotation.Nullable;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+
+/**
+ * Evaluates Kafka header filters for pre-ingestion filtering.
+ */
+public class KafkaHeaderBasedFilterEvaluator
+{
+ private static final Logger log = new Logger(KafkaHeaderBasedFilterEvaluator.class);
+
+ private final HeaderFilterHandler filterHandler;
+ private final String headerName;
+ private final Charset encoding;
+ private final Cache stringDecodingCache;
+
+ /**
+ * Creates a new KafkaHeaderBasedFilterEvaluator with the given configuration.
+ *
+ * @param headerBasedFilterConfig the configuration containing filter, encoding, and cache settings
+ * @throws IllegalArgumentException if the filter type is not supported
+ */
+ public KafkaHeaderBasedFilterEvaluator(KafkaHeaderBasedFilterConfig headerBasedFilterConfig)
+ {
+ this.encoding = Charset.forName(headerBasedFilterConfig.getEncoding());
+ this.stringDecodingCache = Caffeine.newBuilder()
+ .maximumSize(headerBasedFilterConfig.getStringDecodingCacheSize())
+ .build();
+
+ Filter filter = headerBasedFilterConfig.getFilter().toFilter();
+ this.filterHandler = HeaderFilterHandlerFactory.forFilter(filter);
+ this.headerName = filterHandler.getHeaderName();
+
+ log.info("Initialized Kafka header filter: %s with encoding [%s] and cache size [%d]",
+ filterHandler.getDescription(),
+ headerBasedFilterConfig.getEncoding(),
+ headerBasedFilterConfig.getStringDecodingCacheSize());
+ }
+
+
+ /**
+ * Evaluates whether a Kafka record should be included based on its headers.
+ *
+ * @param record the Kafka consumer record
+ * @return true if the record should be included, false if it should be filtered out
+ */
+ public boolean shouldIncludeRecord(ConsumerRecord record)
+ {
+ try {
+ return evaluateInclusion(record.headers());
+ }
+ catch (Exception e) {
+ log.warn(
+ e,
+ "Error evaluating header filter for record at topic [%s] partition [%d] offset [%d], including record",
+ record.topic(),
+ record.partition(),
+ record.offset()
+ );
+ return true; // Default to including record on error
+ }
+ }
+
+ /**
+ * Evaluates whether a record should be included based on its headers.
+ *
+ * Uses permissive behavior: records with missing, null, or undecodable headers
+ * are included by default. Only records with successfully decoded header values
+ * that don't match the filter criteria are excluded.
+ *
+ * @param headers the Kafka message headers to evaluate
+ * @return true if the record should be included, false if it should be filtered out
+ */
+ private boolean evaluateInclusion(Headers headers)
+ {
+ // Permissive behavior: missing headers result in inclusion
+ if (headers == null) {
+ return true;
+ }
+
+ Header header = headers.lastHeader(headerName);
+
+ // Permissive behavior: header is null or empty
+ if (header == null || header.value() == null) {
+ return true;
+ }
+
+ String headerValue = getDecodedHeaderValue(header.value());
+ // Permissive behavior: failed to decode header value
+ if (headerValue == null) {
+ return true;
+ }
+
+ return filterHandler.shouldInclude(headerValue);
+ }
+
+
+ /**
+ * Decode header bytes to string with caching.
+ * Returns null if decoding fails.
+ */
+ @Nullable
+ private String getDecodedHeaderValue(byte[] headerBytes)
+ {
+ try {
+ ByteBuffer key = ByteBuffer.wrap(headerBytes);
+ return stringDecodingCache.get(key, k -> new String(headerBytes, encoding));
+ }
+ catch (Exception e) {
+ log.warn(e, "Failed to decode header bytes, treating as null");
+ return null;
+ }
+ }
+
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
index e5ff77467cdb..e4eb99762f9f 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java
@@ -39,6 +39,7 @@
import org.apache.druid.server.security.ResourceType;
import javax.annotation.Nonnull;
+
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@@ -110,8 +111,13 @@ protected KafkaRecordSupplier newTaskRecordSupplier(final TaskToolbox toolbox)
props.put("auto.offset.reset", "none");
final KafkaRecordSupplier recordSupplier =
- new KafkaRecordSupplier(props, configMapper, kafkaIndexTaskIOConfig.getConfigOverrides(),
- kafkaIndexTaskIOConfig.isMultiTopic());
+ new KafkaRecordSupplier(
+ props,
+ configMapper,
+ kafkaIndexTaskIOConfig.getConfigOverrides(),
+ kafkaIndexTaskIOConfig.isMultiTopic(),
+ kafkaIndexTaskIOConfig.getHeaderBasedFilterConfig()
+ );
if (toolbox.getMonitorScheduler() != null) {
toolbox.getMonitorScheduler().addMonitor(recordSupplier.monitor());
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java
index 82c9ad71c973..d62a3f8f350d 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java
@@ -24,6 +24,7 @@
import com.google.common.base.Preconditions;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers;
import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig;
@@ -32,6 +33,7 @@
import org.joda.time.DateTime;
import javax.annotation.Nullable;
+
import java.util.Map;
public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig
@@ -39,6 +41,7 @@ public class KafkaIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig consumerProperties;
private final long pollTimeout;
private final KafkaConfigOverrides configOverrides;
+ private final KafkaHeaderBasedFilterConfig headerBasedFilterConfig;
private final boolean multiTopic;
@@ -63,6 +66,7 @@ public KafkaIndexTaskIOConfig(
@JsonProperty("maximumMessageTime") DateTime maximumMessageTime,
@JsonProperty("inputFormat") @Nullable InputFormat inputFormat,
@JsonProperty("configOverrides") @Nullable KafkaConfigOverrides configOverrides,
+ @JsonProperty("headerBasedFilterConfig") @Nullable KafkaHeaderBasedFilterConfig headerBasedFilterConfig,
@JsonProperty("multiTopic") @Nullable Boolean multiTopic
)
{
@@ -82,6 +86,7 @@ public KafkaIndexTaskIOConfig(
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
this.pollTimeout = pollTimeout != null ? pollTimeout : KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS;
this.configOverrides = configOverrides;
+ this.headerBasedFilterConfig = headerBasedFilterConfig;
this.multiTopic = multiTopic != null ? multiTopic : KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC;
final SeekableStreamEndSequenceNumbers myEndSequenceNumbers = getEndSequenceNumbers();
@@ -107,7 +112,8 @@ public KafkaIndexTaskIOConfig(
DateTime minimumMessageTime,
DateTime maximumMessageTime,
InputFormat inputFormat,
- KafkaConfigOverrides configOverrides
+ KafkaConfigOverrides configOverrides,
+ KafkaHeaderBasedFilterConfig headerBasedFilterConfig
)
{
this(
@@ -124,6 +130,7 @@ public KafkaIndexTaskIOConfig(
maximumMessageTime,
inputFormat,
configOverrides,
+ headerBasedFilterConfig,
KafkaSupervisorIOConfig.DEFAULT_IS_MULTI_TOPIC
);
}
@@ -180,6 +187,14 @@ public boolean isMultiTopic()
return multiTopic;
}
+
+ @JsonProperty
+ @Nullable
+ public KafkaHeaderBasedFilterConfig getHeaderBasedFilterConfig()
+ {
+ return headerBasedFilterConfig;
+ }
+
@Override
public String toString()
{
@@ -194,6 +209,8 @@ public String toString()
", minimumMessageTime=" + getMinimumMessageTime() +
", maximumMessageTime=" + getMaximumMessageTime() +
", configOverrides=" + getConfigOverrides() +
+ ", headerBasedFilterConfig=" + getHeaderBasedFilterConfig() +
+ ", multiTopic=" + multiTopic +
'}';
}
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
index 6a364fc5f031..2d3120317f37 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskModule.java
@@ -26,6 +26,7 @@
import com.google.inject.Binder;
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.data.input.kafkainput.KafkaInputFormat;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
import org.apache.druid.initialization.DruidModule;
@@ -51,7 +52,8 @@ public List extends Module> getJacksonModules()
new NamedType(KafkaSupervisorTuningConfig.class, SCHEME),
new NamedType(KafkaSupervisorSpec.class, SCHEME),
new NamedType(KafkaSamplerSpec.class, SCHEME),
- new NamedType(KafkaInputFormat.class, SCHEME)
+ new NamedType(KafkaInputFormat.class, SCHEME),
+ new NamedType(KafkaHeaderBasedFilterConfig.class, SCHEME)
)
.addKeySerializer(KafkaTopicPartition.class, new KafkaTopicPartition.KafkaTopicPartitionKeySerializer())
);
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
index a88300552e2a..3c4f501555a7 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaRecordSupplier.java
@@ -27,6 +27,7 @@
import org.apache.druid.data.input.kafka.KafkaTopicPartition;
import org.apache.druid.error.DruidException;
import org.apache.druid.error.InvalidInput;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@@ -45,6 +46,8 @@
import org.apache.kafka.common.serialization.Deserializer;
import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Type;
@@ -68,6 +71,9 @@ public class KafkaRecordSupplier implements RecordSupplier consumerProperties,
+ ObjectMapper sortingMapper,
+ KafkaConfigOverrides configOverrides,
+ boolean multiTopic,
+ @Nullable KafkaHeaderBasedFilterConfig headerBasedFilterConfig
+ )
+ {
+ this(getKafkaConsumer(sortingMapper, consumerProperties, configOverrides), multiTopic, headerBasedFilterConfig);
+ }
+
+ @VisibleForTesting
+ public KafkaRecordSupplier(KafkaConsumer consumer, boolean multiTopic)
+ {
+ this(consumer, multiTopic, (KafkaHeaderBasedFilterConfig) null);
}
@VisibleForTesting
public KafkaRecordSupplier(
KafkaConsumer consumer,
- boolean multiTopic
+ boolean multiTopic,
+ @Nullable KafkaHeaderBasedFilterConfig headerBasedFilterConfig
)
{
this.consumer = consumer;
this.multiTopic = multiTopic;
this.monitor = new KafkaConsumerMonitor(consumer);
+ this.headerFilterEvaluator = headerBasedFilterConfig != null ?
+ new KafkaHeaderBasedFilterEvaluator(headerBasedFilterConfig) : null;
}
@Override
@@ -160,15 +186,32 @@ public Set> getAssignment()
public List> poll(long timeout)
{
List> polledRecords = new ArrayList<>();
- for (ConsumerRecord record : consumer.poll(Duration.ofMillis(timeout))) {
- polledRecords.add(new OrderedPartitionableRecord<>(
- record.topic(),
- new KafkaTopicPartition(multiTopic, record.topic(), record.partition()),
- record.offset(),
- record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record))
- ));
+ for (ConsumerRecord record : consumer.poll(Duration.ofMillis(timeout))) {
+ KafkaTopicPartition kafkaPartition = new KafkaTopicPartition(multiTopic, record.topic(), record.partition());
+
+ // Apply header filter if configured
+ if (headerFilterEvaluator != null && !headerFilterEvaluator.shouldIncludeRecord(record)) {
+ // Create filtered record for offset advancement with filtered=true flag
+ polledRecords.add(new OrderedPartitionableRecord<>(
+ record.topic(),
+ kafkaPartition,
+ record.offset(),
+ Collections.emptyList(), // Empty list for filtered records
+ true // Mark as filtered
+ ));
+ } else {
+ // Create record for accepted records
+ polledRecords.add(new OrderedPartitionableRecord<>(
+ record.topic(),
+ kafkaPartition,
+ record.offset(),
+ record.value() == null ? null : ImmutableList.of(new KafkaRecordEntity(record)),
+ false
+ ));
+ }
}
+
return polledRecords;
}
@@ -241,7 +284,7 @@ public Set getPartitionIds(String stream)
}
/**
- * Returns a Monitor that emits Kafka consumer metrics.
+ * Returns the Kafka consumer monitor.
*/
public Monitor monitor()
{
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfig.java
new file mode 100644
index 000000000000..5f83c65bb9bd
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfig.java
@@ -0,0 +1,137 @@
+/*
+ * 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.kafka.supervisor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.error.InvalidInput;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.filter.InDimFilter;
+
+import javax.annotation.Nullable;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Kafka-specific implementation of header-based filtering.
+ * Allows filtering Kafka records based on message headers before deserialization.
+ */
+public class KafkaHeaderBasedFilterConfig
+{
+ private static final ImmutableSet> SUPPORTED_FILTER_TYPES = ImmutableSet.of(
+ InDimFilter.class
+ );
+
+ private final DimFilter filter;
+ private final String encoding;
+ private final int stringDecodingCacheSize;
+
+ @JsonCreator
+ public KafkaHeaderBasedFilterConfig(
+ @JsonProperty("filter") DimFilter filter,
+ @JsonProperty("encoding") @Nullable String encoding,
+ @JsonProperty("stringDecodingCacheSize") @Nullable Integer stringDecodingCacheSize
+ )
+ {
+ this.filter = Preconditions.checkNotNull(filter, "filter cannot be null");
+ this.encoding = encoding != null ? encoding : StandardCharsets.UTF_8.name();
+ this.stringDecodingCacheSize = stringDecodingCacheSize != null ? stringDecodingCacheSize : 10_000;
+
+ // Validate encoding
+ try {
+ Charset.forName(this.encoding);
+ }
+ catch (Exception e) {
+ throw new IllegalArgumentException("Invalid encoding: " + this.encoding, e);
+ }
+
+ // Validate that only supported filter types are used
+ validateSupportedFilter(this.filter);
+ }
+
+ /**
+ * Validates that the filter is one of the supported types.
+ * Only 'in' filters are supported for direct evaluation.
+ */
+ private void validateSupportedFilter(DimFilter dimFilter)
+ {
+ if (!SUPPORTED_FILTER_TYPES.contains(dimFilter.getClass())) {
+ throw InvalidInput.exception(
+ "Unsupported filter type [%s]. Only 'in' filters are supported for Kafka header filtering.",
+ dimFilter.getClass().getSimpleName()
+ );
+ }
+ }
+
+ @JsonProperty
+ public DimFilter getFilter()
+ {
+ return filter;
+ }
+
+ @JsonProperty
+ public String getEncoding()
+ {
+ return encoding;
+ }
+
+ @JsonProperty
+ public int getStringDecodingCacheSize()
+ {
+ return stringDecodingCacheSize;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ KafkaHeaderBasedFilterConfig that = (KafkaHeaderBasedFilterConfig) o;
+ return stringDecodingCacheSize == that.stringDecodingCacheSize &&
+ filter.equals(that.filter) &&
+ encoding.equals(that.encoding);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ int result = filter.hashCode();
+ result = 31 * result + encoding.hashCode();
+ result = 31 * result + stringDecodingCacheSize;
+ return result;
+ }
+
+ @Override
+ public String toString()
+ {
+ return "KafkaHeaderBasedFilterConfig{" +
+ "filter=" + filter +
+ ", encoding='" + encoding + '\'' +
+ ", stringDecodingCacheSize=" + stringDecodingCacheSize +
+ '}';
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
index aebacecff662..bb01a12ee1d3 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java
@@ -61,6 +61,7 @@
import org.joda.time.DateTime;
import javax.annotation.Nullable;
+
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@@ -137,7 +138,8 @@ protected RecordSupplier setupReco
spec.getIoConfig().getConsumerProperties(),
sortingMapper,
spec.getIoConfig().getConfigOverrides(),
- spec.getIoConfig().isMultiTopic()
+ spec.getIoConfig().isMultiTopic(),
+ spec.getIoConfig().getHeaderBasedFilterConfig()
);
}
@@ -218,6 +220,7 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig(
maximumMessageTime,
ioConfig.getInputFormat(),
kafkaIoConfig.getConfigOverrides(),
+ kafkaIoConfig.getHeaderBasedFilterConfig(),
kafkaIoConfig.isMultiTopic()
);
}
diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
index fbf55f4ab5ed..50d09439c11a 100644
--- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
+++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java
@@ -33,6 +33,7 @@
import org.joda.time.Period;
import javax.annotation.Nullable;
+
import java.util.Map;
public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
@@ -51,6 +52,7 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
private final KafkaConfigOverrides configOverrides;
private final String topic;
private final String topicPattern;
+ private final KafkaHeaderBasedFilterConfig headerBasedFilterConfig;
@JsonCreator
public KafkaSupervisorIOConfig(
@@ -71,6 +73,7 @@ public KafkaSupervisorIOConfig(
@JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod,
@JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime,
@JsonProperty("configOverrides") KafkaConfigOverrides configOverrides,
+ @JsonProperty("headerBasedFilterConfig") KafkaHeaderBasedFilterConfig headerBasedFilterConfig,
@JsonProperty("idleConfig") IdleConfig idleConfig,
@JsonProperty("stopTaskCount") Integer stopTaskCount
)
@@ -93,6 +96,7 @@ public KafkaSupervisorIOConfig(
stopTaskCount
);
+ this.headerBasedFilterConfig = headerBasedFilterConfig;
this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties");
Preconditions.checkNotNull(
consumerProperties.get(BOOTSTRAP_SERVERS_KEY),
@@ -151,6 +155,14 @@ public boolean isMultiTopic()
return topicPattern != null;
}
+ @JsonProperty
+ @Nullable
+ public KafkaHeaderBasedFilterConfig getHeaderBasedFilterConfig()
+ {
+ return headerBasedFilterConfig;
+ }
+
+
@Override
public String toString()
{
@@ -171,6 +183,7 @@ public String toString()
", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() +
", lateMessageRejectionStartDateTime=" + getLateMessageRejectionStartDateTime() +
", configOverrides=" + getConfigOverrides() +
+ ", headerBasedFilterConfig=" + headerBasedFilterConfig +
", idleConfig=" + getIdleConfig() +
", stopTaskCount=" + getStopTaskCount() +
'}';
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
index 858cd79fbd78..2b4c827695de 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/data/input/kafkainput/KafkaInputFormatTest.java
@@ -28,6 +28,7 @@
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
@@ -405,6 +406,7 @@ public byte[] value()
while (iterator.hasNext()) {
final InputRow row = iterator.next();
+ final MapBasedInputRow mrow = (MapBasedInputRow) row;
// Payload verifications
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
// but test reading them anyway since it isn't technically illegal
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerTest.java
new file mode 100644
index 000000000000..2a62c0642450
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/HeaderFilterHandlerTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.kafka;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.query.filter.Filter;
+import org.apache.druid.query.filter.InDimFilter;
+import org.apache.druid.query.filter.TrueDimFilter;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class HeaderFilterHandlerTest
+{
+ @Test
+ public void testInDimFilterHandler()
+ {
+ // Create an InDimFilter for testing
+ InDimFilter filter = new InDimFilter("environment", ImmutableSet.of("production", "staging"));
+
+ // Create handler using our extensible factory
+ HeaderFilterHandler handler = HeaderFilterHandlerFactory.forFilter(filter);
+
+ // Verify it's the correct type
+ Assert.assertTrue("Handler should be InDimFilterHandler", handler instanceof InDimFilterHandler);
+
+ // Test header name extraction
+ Assert.assertEquals("environment", handler.getHeaderName());
+
+ // Test matching values
+ Assert.assertTrue("Production should be included", handler.shouldInclude("production"));
+ Assert.assertTrue("Staging should be included", handler.shouldInclude("staging"));
+
+ // Test non-matching values
+ Assert.assertFalse("Development should be excluded", handler.shouldInclude("development"));
+ Assert.assertFalse("Test should be excluded", handler.shouldInclude("test"));
+
+ // Test description
+ String description = handler.getDescription();
+ Assert.assertTrue("Description should contain filter type", description.contains("InDimFilter"));
+ Assert.assertTrue("Description should contain header name", description.contains("environment"));
+ Assert.assertTrue("Description should contain value count", description.contains("2"));
+ }
+
+ @Test
+ public void testUnsupportedFilterType()
+ {
+ // Create a mock filter that's not supported
+ Filter unsupportedFilter = TrueDimFilter.instance().toFilter();
+
+ // Should throw IllegalArgumentException
+ try {
+ HeaderFilterHandlerFactory.forFilter(unsupportedFilter);
+ Assert.fail("Should have thrown IllegalArgumentException for unsupported filter type");
+ }
+ catch (IllegalArgumentException e) {
+ Assert.assertTrue("Error message should mention unsupported type",
+ e.getMessage().contains("Unsupported filter type"));
+ Assert.assertTrue("Error message should mention True",
+ e.getMessage().contains("True"));
+ }
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigEvaluatorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigEvaluatorTest.java
new file mode 100644
index 000000000000..e8c9703b3b7b
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigEvaluatorTest.java
@@ -0,0 +1,294 @@
+/*
+ * 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.kafka;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
+import org.apache.druid.query.filter.InDimFilter;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+
+public class KafkaHeaderBasedFilterConfigEvaluatorTest
+{
+ private KafkaHeaderBasedFilterEvaluator evaluator;
+ private ConsumerRecord record;
+
+ @BeforeClass
+ public static void setUpStatic()
+ {
+ NullHandling.initializeForTests();
+ }
+
+ @Before
+ public void setUp()
+ {
+ // Create a test record with headers
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("environment", "production".getBytes(StandardCharsets.UTF_8)));
+ headers.add(new RecordHeader("service", "user-service".getBytes(StandardCharsets.UTF_8)));
+ headers.add(new RecordHeader("version", "1.0".getBytes(StandardCharsets.UTF_8)));
+
+ record = new ConsumerRecord<>(
+ "test-topic",
+ 0,
+ 100L,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ // Use reflection to set headers since ConsumerRecord doesn't have a public setter
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(record, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+ }
+
+ @Test
+ public void testInFilterSingleValueMatch()
+ {
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertTrue(evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testInFilterSingleValueNoMatch()
+ {
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("staging"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertFalse(evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testInFilterMultipleValuesMatch()
+ {
+ InDimFilter filter = new InDimFilter("environment", Arrays.asList("staging", "production", "development"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertTrue(evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testInFilterMultipleValuesNoMatch()
+ {
+ InDimFilter filter = new InDimFilter("environment", Arrays.asList("staging", "development"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertFalse(evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testInFilterMissingHeader()
+ {
+ InDimFilter filter = new InDimFilter("missing-header", Collections.singletonList("value"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ // With permissive filtering, missing headers should result in inclusion
+ Assert.assertTrue("InDimFilter with missing header should include record (permissive)", evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testInFilterNullValue()
+ {
+ // Create record with null header value
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("null-header", null));
+
+ ConsumerRecord nullRecord = new ConsumerRecord<>(
+ "test-topic",
+ 0,
+ 100L,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(nullRecord, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ InDimFilter filter = new InDimFilter("null-header", Collections.singletonList("value"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertTrue(evaluator.shouldIncludeRecord(nullRecord));
+ }
+
+ @Test
+ public void testInFilterWithDifferentServices()
+ {
+ InDimFilter filter = new InDimFilter("service", Arrays.asList("user-service", "payment-service"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertTrue(evaluator.shouldIncludeRecord(record)); // matches "user-service"
+ }
+
+ @Test
+ public void testInFilterWithDifferentServicesNoMatch()
+ {
+ InDimFilter filter = new InDimFilter("service", Arrays.asList("payment-service", "notification-service"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ Assert.assertFalse(evaluator.shouldIncludeRecord(record)); // doesn't match "user-service"
+ }
+
+ @Test
+ public void testRepeatedEvaluations()
+ {
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ // Test multiple evaluations to verify consistent behavior
+ boolean result1 = evaluator.shouldIncludeRecord(record); // should match
+ boolean result2 = evaluator.shouldIncludeRecord(record); // should match
+
+ Assert.assertTrue("First evaluation should match", result1);
+ Assert.assertTrue("Second evaluation should match", result2);
+ Assert.assertEquals("Results should be consistent", result1, result2);
+ }
+
+ @Test
+ public void testDifferentEncodings()
+ {
+ // Test with ISO-8859-1 encoding
+ String testValue = "café"; // Contains non-ASCII characters
+
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("text", testValue.getBytes(StandardCharsets.ISO_8859_1)));
+
+ ConsumerRecord encodedRecord = new ConsumerRecord<>(
+ "test-topic",
+ 0,
+ 100L,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(encodedRecord, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ InDimFilter filter = new InDimFilter("text", Collections.singletonList(testValue), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, "ISO-8859-1", null));
+
+ Assert.assertTrue(evaluator.shouldIncludeRecord(encodedRecord));
+ }
+
+ @Test
+ public void testNullHeaderValue()
+ {
+ // Create record without the header we're filtering on
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("other-header", "other-value".getBytes(StandardCharsets.UTF_8)));
+
+ ConsumerRecord noHeaderRecord = new ConsumerRecord<>(
+ "test-topic",
+ 0,
+ 100L,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(noHeaderRecord, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, null));
+
+ // Missing header should result in inclusion (permissive behavior)
+ Assert.assertTrue(evaluator.shouldIncludeRecord(noHeaderRecord));
+ }
+
+ @Test
+ public void testMultipleHeadersWithSameKey()
+ {
+ // Create record with multiple headers with the same key (Kafka allows this)
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("environment", "staging".getBytes(StandardCharsets.UTF_8)));
+ headers.add(new RecordHeader("environment", "production".getBytes(StandardCharsets.UTF_8))); // Last one wins
+
+ ConsumerRecord multiHeaderRecord = new ConsumerRecord<>(
+ "test-topic",
+ 0,
+ 100L,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(multiHeaderRecord, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ // Filter should match "production" (the last value), not "staging" (the first value)
+ InDimFilter prodFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(prodFilter, null, null));
+ Assert.assertTrue("Should match last header value 'production'", evaluator.shouldIncludeRecord(multiHeaderRecord));
+
+ // Filter should NOT match "staging" (the first value)
+ InDimFilter stagingFilter = new InDimFilter("environment", Collections.singletonList("staging"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(stagingFilter, null, null));
+ Assert.assertFalse("Should not match first header value 'staging'", evaluator.shouldIncludeRecord(multiHeaderRecord));
+ }
+
+ @Test
+ public void testStringDecodingCacheSize()
+ {
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(new KafkaHeaderBasedFilterConfig(filter, null, 50_000));
+
+ // Test that the evaluator works with custom cache size
+ Assert.assertTrue(evaluator.shouldIncludeRecord(record));
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigIntegrationTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigIntegrationTest.java
new file mode 100644
index 000000000000..f032eb7c262b
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaHeaderBasedFilterConfigIntegrationTest.java
@@ -0,0 +1,269 @@
+/*
+ * 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.kafka;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.query.filter.InDimFilter;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+
+public class KafkaHeaderBasedFilterConfigIntegrationTest
+{
+ private KafkaHeaderBasedFilterEvaluator evaluator;
+ private final ObjectMapper objectMapper = new DefaultObjectMapper();
+
+ @BeforeClass
+ public static void setUpStatic()
+ {
+ NullHandling.initializeForTests();
+ }
+
+ @Before
+ public void setUp()
+ {
+ // Will be initialized in each test
+ evaluator = null;
+ }
+
+ private ConsumerRecord createRecord(String topic, int partition, long offset, RecordHeaders headers)
+ {
+ ConsumerRecord record = new ConsumerRecord<>(
+ topic,
+ partition,
+ offset,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(record, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ return record;
+ }
+
+ private RecordHeaders headers(String... keyValuePairs)
+ {
+ RecordHeaders headers = new RecordHeaders();
+ for (int i = 0; i < keyValuePairs.length; i += 2) {
+ headers.add(new RecordHeader(keyValuePairs[i], keyValuePairs[i + 1].getBytes(StandardCharsets.UTF_8)));
+ }
+ return headers;
+ }
+
+ @Test
+ public void testProductionEnvironmentFiltering()
+ {
+ // Test Case: Only include records from production environment
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ // Production record - should be included
+ ConsumerRecord prodRecord = createRecord(
+ "events",
+ 0,
+ 100L,
+ headers("environment", "production", "service", "user-service")
+ );
+ Assert.assertTrue("Production record should be included", evaluator.shouldIncludeRecord(prodRecord));
+
+ // Staging record - should be excluded
+ ConsumerRecord stagingRecord = createRecord(
+ "events",
+ 0,
+ 101L,
+ headers("environment", "staging", "service", "user-service")
+ );
+ Assert.assertFalse("Staging record should be excluded", evaluator.shouldIncludeRecord(stagingRecord));
+
+ // Record without environment header - should be included (permissive)
+ ConsumerRecord noEnvRecord = createRecord(
+ "events",
+ 0,
+ 102L,
+ headers("service", "user-service")
+ );
+ Assert.assertTrue("Record without environment header should be included", evaluator.shouldIncludeRecord(noEnvRecord));
+ }
+
+ @Test
+ public void testMultiServiceFiltering()
+ {
+ // Test Case: Include records from multiple services
+ InDimFilter filter = new InDimFilter("service", Arrays.asList("user-service", "payment-service"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ // User service record - should be included
+ ConsumerRecord userRecord = createRecord(
+ "events",
+ 0,
+ 100L,
+ headers("environment", "production", "service", "user-service")
+ );
+ Assert.assertTrue("User service record should be included", evaluator.shouldIncludeRecord(userRecord));
+
+ // Payment service record - should be included
+ ConsumerRecord paymentRecord = createRecord(
+ "events",
+ 0,
+ 101L,
+ headers("environment", "production", "service", "payment-service")
+ );
+ Assert.assertTrue("Payment service record should be included", evaluator.shouldIncludeRecord(paymentRecord));
+
+ // Notification service record - should be excluded
+ ConsumerRecord notificationRecord = createRecord(
+ "events",
+ 0,
+ 102L,
+ headers("environment", "production", "service", "notification-service")
+ );
+ Assert.assertFalse("Notification service record should be excluded", evaluator.shouldIncludeRecord(notificationRecord));
+ }
+
+ @Test
+ public void testHighThroughputFiltering()
+ {
+ // Test Case: Filter for high-throughput scenarios with many values
+ InDimFilter filter = new InDimFilter(
+ "metric.name",
+ Arrays.asList(
+ "io.kafka.server/delayed_share_fetch/expires/total/delta",
+ "io.kafka.server/request_bytes/total/delta",
+ "io.kafka.server/response_bytes/total/delta"
+ ),
+ null
+ );
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ // Matching metric - should be included
+ ConsumerRecord matchingRecord = createRecord(
+ "telemetry.metrics.cloud.stag",
+ 0,
+ 100L,
+ headers("metric.name", "io.kafka.server/delayed_share_fetch/expires/total/delta")
+ );
+ Assert.assertTrue("Matching metric should be included", evaluator.shouldIncludeRecord(matchingRecord));
+
+ // Non-matching metric - should be excluded
+ ConsumerRecord nonMatchingRecord = createRecord(
+ "telemetry.metrics.cloud.stag",
+ 0,
+ 101L,
+ headers("metric.name", "some.other.metric")
+ );
+ Assert.assertFalse("Non-matching metric should be excluded", evaluator.shouldIncludeRecord(nonMatchingRecord));
+ }
+
+ @Test
+ public void testFilteringBehavior()
+ {
+ // Test Case: Verify basic filtering behavior
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ // Process multiple records to verify filtering logic
+ ConsumerRecord record1 = createRecord("events", 0, 100L, headers("environment", "production"));
+ ConsumerRecord record2 = createRecord("events", 0, 101L, headers("environment", "staging"));
+ ConsumerRecord record3 = createRecord("events", 0, 102L, headers("environment", "production"));
+
+ // Verify filtering results
+ Assert.assertTrue("Production record should be included", evaluator.shouldIncludeRecord(record1));
+ Assert.assertFalse("Staging record should be excluded", evaluator.shouldIncludeRecord(record2));
+ Assert.assertTrue("Production record should be included", evaluator.shouldIncludeRecord(record3));
+ }
+
+ @Test
+ public void testConfigurationSerialization() throws Exception
+ {
+ // Test that header filter configurations can be serialized/deserialized correctly
+ InDimFilter filter = new InDimFilter("environment", Arrays.asList("production", "staging"), null);
+ KafkaHeaderBasedFilterConfig originalFilter = new KafkaHeaderBasedFilterConfig(filter, "UTF-16", null);
+
+ // Serialize to JSON
+ String json = objectMapper.writeValueAsString(originalFilter);
+
+ // Deserialize back
+ KafkaHeaderBasedFilterConfig deserializedFilter = objectMapper.readValue(json, KafkaHeaderBasedFilterConfig.class);
+
+ // Verify they're equivalent
+ Assert.assertEquals(originalFilter.getFilter(), deserializedFilter.getFilter());
+ Assert.assertEquals(originalFilter.getEncoding(), deserializedFilter.getEncoding());
+ Assert.assertEquals(originalFilter.getStringDecodingCacheSize(), deserializedFilter.getStringDecodingCacheSize());
+
+ // Test that the deserialized filter works
+ evaluator = new KafkaHeaderBasedFilterEvaluator(deserializedFilter);
+ ConsumerRecord record = createRecord("events", 0, 100L, headers("environment", "production"));
+ Assert.assertFalse("Deserialized filter should work", evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testEncodingHandling()
+ {
+ // Test different character encodings
+ String testValue = "café"; // Contains non-ASCII characters
+
+ InDimFilter filter = new InDimFilter("text", Collections.singletonList(testValue), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, "ISO-8859-1", null);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ // Create record with ISO-8859-1 encoded header
+ RecordHeaders headers = new RecordHeaders();
+ headers.add(new RecordHeader("text", testValue.getBytes(StandardCharsets.ISO_8859_1)));
+
+ ConsumerRecord record = createRecord("events", 0, 100L, headers);
+
+ Assert.assertTrue("Should handle different encodings correctly", evaluator.shouldIncludeRecord(record));
+ }
+
+ @Test
+ public void testCustomCacheSize()
+ {
+ // Test with custom cache size
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, 100_000);
+ evaluator = new KafkaHeaderBasedFilterEvaluator(headerFilter);
+
+ ConsumerRecord record = createRecord("events", 0, 100L, headers("environment", "production"));
+ Assert.assertTrue("Should work with custom cache size", evaluator.shouldIncludeRecord(record));
+ }
+}
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 5ee288d2622d..e52794979f9b 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
@@ -357,6 +357,7 @@ public void testRunAfterDataInserted() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -413,6 +414,7 @@ public void testIngestNullColumnAfterDataInserted() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -461,6 +463,7 @@ public void testIngestNullColumnAfterDataInserted_storeEmptyColumnsOff_shouldNot
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -496,6 +499,7 @@ public void testRunAfterDataInsertedWithLegacyParser() throws Exception
null,
null,
null,
+ null,
null
)
);
@@ -537,6 +541,7 @@ public void testRunBeforeDataInserted() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -588,6 +593,7 @@ public void testRunAfterDataInsertedLiveReport() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -665,6 +671,7 @@ public void testIncrementalHandOff() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -768,6 +775,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -894,6 +902,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -972,6 +981,7 @@ public void testCheckpointResetWithSameEndOffsets() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1035,6 +1045,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1051,6 +1062,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1117,6 +1129,7 @@ public void testRunWithMinimumMessageTime() throws Exception
DateTimes.of("2010"),
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1165,6 +1178,7 @@ public void testRunWithMaximumMessageTime() throws Exception
null,
DateTimes.of("2010"),
INPUT_FORMAT,
+ null,
null
)
);
@@ -1222,6 +1236,7 @@ public void testRunWithTransformSpec() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1294,6 +1309,7 @@ public void testKafkaRecordEntityInputFormat() throws Exception
null,
null,
new TestKafkaInputFormat(INPUT_FORMAT),
+ null,
null
)
);
@@ -1367,6 +1383,7 @@ public void testKafkaInputFormat() throws Exception
null,
null,
KAFKA_INPUT_FORMAT,
+ null,
null
)
);
@@ -1418,6 +1435,7 @@ public void testRunOnNothing() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1453,6 +1471,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1499,6 +1518,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1550,6 +1570,7 @@ public void testReportParseExceptions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1588,6 +1609,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1624,7 +1646,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.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1676,6 +1699,7 @@ public void testMultipleParseExceptionsFailure() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1704,7 +1728,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.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1738,6 +1763,7 @@ public void testRunReplicas() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1754,6 +1780,7 @@ public void testRunReplicas() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1802,6 +1829,7 @@ public void testRunConflicting() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1818,6 +1846,7 @@ public void testRunConflicting() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1868,6 +1897,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1884,6 +1914,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1932,6 +1963,7 @@ public void testRunOneTaskTwoPartitions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1978,6 +2010,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -1994,6 +2027,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2044,6 +2078,7 @@ public void testRestore() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2078,6 +2113,7 @@ public void testRestore() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2128,6 +2164,7 @@ public void testRestoreAfterPersistingSequences() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2171,6 +2208,7 @@ public void testRestoreAfterPersistingSequences() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2222,6 +2260,7 @@ public void testRunWithPauseAndResume() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2293,6 +2332,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2330,6 +2370,7 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2377,6 +2418,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
),
context
@@ -2421,6 +2463,7 @@ public void testRunWithDuplicateRequest() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2460,6 +2503,7 @@ public void testRunTransactionModeRollback() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2541,6 +2585,7 @@ public void testRunUnTransactionMode() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2601,6 +2646,7 @@ public void testCanStartFromLaterThanEarliestOffset() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2624,6 +2670,7 @@ public void testRunWithoutDataInserted() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2670,6 +2717,7 @@ public void testSerde() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2702,6 +2750,7 @@ public void testCorrectInputSources() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -2975,6 +3024,7 @@ public void testMultipleLinesJSONText() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -3036,6 +3086,7 @@ public void testParseExceptionsInIteratorConstructionSuccess() throws Exception
null,
null,
new TestKafkaFormatWithMalformedDataDetection(INPUT_FORMAT),
+ null,
null
)
);
@@ -3104,6 +3155,7 @@ public void testNoParseExceptionsTaskSucceeds() throws Exception
null,
null,
new TestKafkaFormatWithMalformedDataDetection(INPUT_FORMAT),
+ null,
null
)
);
@@ -3178,6 +3230,7 @@ public void testParseExceptionsBeyondThresholdTaskFails() throws Exception
null,
null,
new TestKafkaFormatWithMalformedDataDetection(INPUT_FORMAT),
+ null,
null
)
);
@@ -3225,6 +3278,7 @@ public void testCompletionReportPartitionStats() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
@@ -3273,6 +3327,7 @@ public void testCompletionReportMultiplePartitionStats() throws Exception
null,
null,
INPUT_FORMAT,
+ null,
null
)
);
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierHeaderFilterTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierHeaderFilterTest.java
new file mode 100644
index 000000000000..84ee674a66f5
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaRecordSupplierHeaderFilterTest.java
@@ -0,0 +1,485 @@
+/*
+ * 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.kafka;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.data.input.kafka.KafkaRecordEntity;
+import org.apache.druid.data.input.kafka.KafkaTopicPartition;
+import org.apache.druid.indexing.kafka.supervisor.KafkaHeaderBasedFilterConfig;
+import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.query.filter.InDimFilter;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.lang.reflect.Field;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test KafkaRecordSupplier with header-based filtering integrated into the main poll() method.
+ */
+public class KafkaRecordSupplierHeaderFilterTest
+{
+ private KafkaConsumer mockConsumer;
+
+ private KafkaRecordSupplier recordSupplier;
+
+ @BeforeClass
+ public static void setUpClass()
+ {
+ NullHandling.initializeForTests();
+ }
+
+ @Before
+ public void setUp()
+ {
+ mockConsumer = EasyMock.createMock(KafkaConsumer.class);
+ }
+
+ @Test
+ public void testNoHeaderFilter()
+ {
+ // Test that records are not filtered when no header filter is configured
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, null);
+
+ ConsumerRecord record1 = createRecord("topic", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord record2 = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(record1, record2)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should include all records when no filter", 2, results.size());
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testInHeaderFilterSingleValue()
+ {
+ // Test filtering with in filter (single value)
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ ConsumerRecord prodRecord = createRecord("topic", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord stagingRecord = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+ ConsumerRecord noHeaderRecord = createRecord("topic", 0, 102L,
+ new RecordHeaders()); // No headers
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(prodRecord, stagingRecord, noHeaderRecord)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should return all records (accepted + filtered markers)", 3, results.size());
+
+ // First record: production (accepted - has data)
+ Assert.assertNotNull("Production record should have data", results.get(0).getData());
+ Assert.assertFalse("Production record should have data", results.get(0).getData().isEmpty());
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+
+ // Second record: staging (filtered - empty data for offset advancement)
+ Assert.assertTrue("Staging record should have empty data", results.get(1).getData().isEmpty());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+
+ // Third record: no-header (accepted - has data, permissive behavior)
+ Assert.assertNotNull("No-header record should have data", results.get(2).getData());
+ Assert.assertFalse("No-header record should have data", results.get(2).getData().isEmpty());
+ Assert.assertEquals(102L, (long) results.get(2).getSequenceNumber());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testFilteredFlagTracking()
+ {
+ // Test that filtered records are properly marked with filtered flag
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ ConsumerRecord prodRecord = createRecord("topic", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord stagingRecord = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(prodRecord, stagingRecord)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ // Verify records returned
+ Assert.assertEquals("Should return 2 records (accepted + filtered)", 2, results.size());
+
+ // Verify filtered flags
+ Assert.assertFalse("Production record should not be filtered", results.get(0).isFiltered());
+ Assert.assertTrue("Staging record should be filtered", results.get(1).isFiltered());
+
+ // Verify data presence
+ Assert.assertFalse("Production record should have data", results.get(0).getData().isEmpty());
+ Assert.assertTrue("Filtered record should have empty data", results.get(1).getData().isEmpty());
+
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testInHeaderFilterMultipleValues()
+ {
+ // Test filtering with in filter (multiple values)
+ InDimFilter filter = new InDimFilter("service", Arrays.asList("user-service", "payment-service"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ ConsumerRecord userServiceRecord = createRecord("topic", 0, 100L,
+ headers("service", "user-service"));
+ ConsumerRecord paymentServiceRecord = createRecord("topic", 0, 101L,
+ headers("service", "payment-service"));
+ ConsumerRecord orderServiceRecord = createRecord("topic", 0, 102L,
+ headers("service", "order-service"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(userServiceRecord, paymentServiceRecord, orderServiceRecord)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should return all records (accepted + filtered markers)", 3, results.size());
+
+ // First record: user-service (accepted - has data)
+ Assert.assertNotNull("User-service record should have data", results.get(0).getData());
+ Assert.assertFalse("User-service record should have data", results.get(0).getData().isEmpty());
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+
+ // Second record: payment-service (accepted - has data)
+ Assert.assertNotNull("Payment-service record should have data", results.get(1).getData());
+ Assert.assertFalse("Payment-service record should have data", results.get(1).getData().isEmpty());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+
+ // Third record: order-service (filtered - empty data for offset advancement marker)
+ Assert.assertTrue("Order-service record should have empty data", results.get(2).getData().isEmpty());
+ Assert.assertEquals(102L, (long) results.get(2).getSequenceNumber());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testInFilterWithMultipleHeaders()
+ {
+ // Test InDimFilter with multiple possible values
+ InDimFilter serviceFilter = new InDimFilter("service", Arrays.asList("user-service", "payment-service"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(serviceFilter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ ConsumerRecord userServiceRecord = createRecord("topic", 0, 100L,
+ headers("service", "user-service"));
+ ConsumerRecord paymentServiceRecord = createRecord("topic", 0, 101L,
+ headers("service", "payment-service"));
+ ConsumerRecord orderServiceRecord = createRecord("topic", 0, 102L,
+ headers("service", "order-service"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(userServiceRecord, paymentServiceRecord, orderServiceRecord)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should return all records (accepted + filtered markers)", 3, results.size());
+
+ // First record: user-service (accepted - has data)
+ Assert.assertNotNull("User-service record should have data", results.get(0).getData());
+ Assert.assertFalse("User-service record should have data", results.get(0).getData().isEmpty());
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+
+ // Second record: payment-service (accepted - has data)
+ Assert.assertNotNull("Payment-service record should have data", results.get(1).getData());
+ Assert.assertFalse("Payment-service record should have data", results.get(1).getData().isEmpty());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+
+ // Third record: order-service (filtered - empty data for offset advancement marker)
+ Assert.assertTrue("Order-service record should have empty data", results.get(2).getData().isEmpty());
+ Assert.assertEquals(102L, (long) results.get(2).getSequenceNumber());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testMultiplePolls()
+ {
+ // Test that statistics accumulate across multiple polls
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ // First poll
+ ConsumerRecord prodRecord1 = createRecord("topic", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord stagingRecord1 = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+
+ // Second poll
+ ConsumerRecord prodRecord2 = createRecord("topic", 0, 102L,
+ headers("environment", "production"));
+ ConsumerRecord stagingRecord2 = createRecord("topic", 0, 103L,
+ headers("environment", "staging"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(prodRecord1, stagingRecord1)));
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(prodRecord2, stagingRecord2)));
+ EasyMock.replay(mockConsumer);
+
+ List> results1 =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("First poll should return 2 records (accepted + filtered marker)", 2, results1.size());
+ Assert.assertNotNull("Production record should have data", results1.get(0).getData());
+ Assert.assertFalse("Production record should have data", results1.get(0).getData().isEmpty());
+ Assert.assertTrue("Staging record should have empty data", results1.get(1).getData().isEmpty());
+
+ List> results2 =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Second poll should return 2 records (accepted + filtered marker)", 2, results2.size());
+ Assert.assertNotNull("Production record should have data", results2.get(0).getData());
+ Assert.assertFalse("Production record should have data", results2.get(0).getData().isEmpty());
+ Assert.assertTrue("Staging record should have empty data", results2.get(1).getData().isEmpty());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testEmptyPoll()
+ {
+ // Test that empty polls don't affect statistics
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Collections.emptyList()));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Empty poll should return empty list", 0, results.size());
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testAllRecordsFilteredStillAdvanceOffsets()
+ {
+ // CRITICAL TEST: Verify that when ALL records are filtered out, we still return
+ // filtered record markers to prevent infinite loop
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ // All records have "staging" environment - none should pass the "production" filter
+ ConsumerRecord stagingRecord1 = createRecord("topic", 0, 100L,
+ headers("environment", "staging"));
+ ConsumerRecord stagingRecord2 = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+ ConsumerRecord stagingRecord3 = createRecord("topic", 0, 102L,
+ headers("environment", "staging"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(stagingRecord1, stagingRecord2, stagingRecord3)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ // CRITICAL: Even though all records were filtered, we should still get record markers
+ // to advance offsets and prevent infinite loop
+ Assert.assertEquals("Should return filtered record markers for offset advancement", 3, results.size());
+
+ // Verify that all returned records have filtered record markers
+ for (OrderedPartitionableRecord result : results) {
+ Assert.assertTrue("Filtered record should have empty data", result.getData().isEmpty());
+ }
+
+ // Verify offsets are correct
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+ Assert.assertEquals(102L, (long) results.get(2).getSequenceNumber());
+
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testMixedFilteredAndAcceptedRecords()
+ {
+ // Test that mix of filtered and accepted records works correctly
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, false, headerFilter);
+
+ ConsumerRecord prodRecord = createRecord("topic", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord stagingRecord = createRecord("topic", 0, 101L,
+ headers("environment", "staging"));
+ ConsumerRecord prodRecord2 = createRecord("topic", 0, 102L,
+ headers("environment", "production"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(prodRecord, stagingRecord, prodRecord2)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should return all records (accepted + filtered markers)", 3, results.size());
+
+ // First record: accepted (has data)
+ Assert.assertNotNull("Accepted record should have data", results.get(0).getData());
+ Assert.assertEquals(100L, (long) results.get(0).getSequenceNumber());
+
+ // Second record: filtered (empty data for offset advancement marker)
+ Assert.assertTrue("Filtered record should have empty data", results.get(1).getData().isEmpty());
+ Assert.assertEquals(101L, (long) results.get(1).getSequenceNumber());
+
+ // Third record: accepted (has data)
+ Assert.assertNotNull("Accepted record should have data", results.get(2).getData());
+ Assert.assertEquals(102L, (long) results.get(2).getSequenceNumber());
+
+ EasyMock.verify(mockConsumer);
+ }
+
+ @Test
+ public void testMultiTopic()
+ {
+ // Test header filtering with multi-topic configuration
+ InDimFilter filter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig headerFilter = new KafkaHeaderBasedFilterConfig(filter, null, null);
+
+ recordSupplier = new KafkaRecordSupplier(mockConsumer, true, headerFilter); // multiTopic = true
+
+ ConsumerRecord topic1Record = createRecord("topic1", 0, 100L,
+ headers("environment", "production"));
+ ConsumerRecord topic2Record = createRecord("topic2", 0, 101L,
+ headers("environment", "staging"));
+
+ EasyMock.expect(mockConsumer.poll(EasyMock.anyObject(Duration.class)))
+ .andReturn(createConsumerRecords(Arrays.asList(topic1Record, topic2Record)));
+ EasyMock.replay(mockConsumer);
+
+ List> results =
+ recordSupplier.poll(1000);
+
+ Assert.assertEquals("Should return both records (accepted + filtered marker)", 2, results.size());
+
+ // First record: accepted
+ Assert.assertNotNull("Production record should have data", results.get(0).getData());
+ Assert.assertEquals("topic1", results.get(0).getStream());
+ Assert.assertTrue("Should be multi-topic partition",
+ results.get(0).getPartitionId().isMultiTopicPartition());
+
+ // Second record: filtered marker
+ Assert.assertTrue("Staging record should have empty data", results.get(1).getData().isEmpty());
+ Assert.assertEquals("topic2", results.get(1).getStream());
+
+ EasyMock.verify(mockConsumer);
+ }
+
+ // Helper methods
+
+ private ConsumerRecord createRecord(String topic, int partition, long offset, RecordHeaders headers)
+ {
+ ConsumerRecord record = new ConsumerRecord<>(
+ topic,
+ partition,
+ offset,
+ "test-key".getBytes(StandardCharsets.UTF_8),
+ "test-value".getBytes(StandardCharsets.UTF_8)
+ );
+
+ // Set headers using reflection since ConsumerRecord headers are final
+ try {
+ Field headersField = ConsumerRecord.class.getDeclaredField("headers");
+ headersField.setAccessible(true);
+ headersField.set(record, headers);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Failed to set headers on test record", e);
+ }
+
+ return record;
+ }
+
+ private RecordHeaders headers(String... keyValuePairs)
+ {
+ if (keyValuePairs.length % 2 != 0) {
+ throw new IllegalArgumentException("Key-value pairs must be even number of arguments");
+ }
+
+ RecordHeaders headers = new RecordHeaders();
+ for (int i = 0; i < keyValuePairs.length; i += 2) {
+ String key = keyValuePairs[i];
+ String value = keyValuePairs[i + 1];
+ headers.add(new RecordHeader(key, value.getBytes(StandardCharsets.UTF_8)));
+ }
+ return headers;
+ }
+
+ private ConsumerRecords createConsumerRecords(List> records)
+ {
+ Map>> recordsMap = new HashMap<>();
+ for (ConsumerRecord record : records) {
+ TopicPartition tp = new TopicPartition(record.topic(), record.partition());
+ recordsMap.computeIfAbsent(tp, k -> new ArrayList<>()).add(record);
+ }
+ return new ConsumerRecords<>(recordsMap);
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java
index 0a0b64396a66..dfde57ca8656 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java
@@ -181,6 +181,7 @@ public void testSample()
null,
null,
null,
+ null,
null
),
null,
@@ -234,6 +235,7 @@ public void testSampleWithTopicPattern()
null,
null,
null,
+ null,
null
),
null,
@@ -296,6 +298,7 @@ public void testSampleKafkaInputFormat()
null,
null,
null,
+ null,
null
),
null,
@@ -399,6 +402,7 @@ public void testWithInputRowParser() throws IOException
null,
null,
null,
+ null,
null
),
null,
@@ -583,6 +587,7 @@ public void testInvalidKafkaConfig()
null,
null,
null,
+ null,
null
),
null,
@@ -639,6 +644,7 @@ public void testGetInputSourceResources()
null,
null,
null,
+ null,
null
),
null,
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfigTest.java
new file mode 100644
index 000000000000..61406af45eba
--- /dev/null
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaHeaderBasedFilterConfigTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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.kafka.supervisor;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.query.filter.AndDimFilter;
+import org.apache.druid.query.filter.InDimFilter;
+import org.apache.druid.query.filter.NotDimFilter;
+import org.apache.druid.query.filter.SelectorDimFilter;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+public class KafkaHeaderBasedFilterConfigTest
+{
+ private final ObjectMapper objectMapper = new DefaultObjectMapper();
+
+ @BeforeClass
+ public static void setUpStatic()
+ {
+ NullHandling.initializeForTests();
+ }
+
+ @Test
+ public void testInFilterSingleValue()
+ {
+ InDimFilter dimFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig filter = new KafkaHeaderBasedFilterConfig(dimFilter, null, null);
+
+ Assert.assertEquals(dimFilter, filter.getFilter());
+ Assert.assertEquals("UTF-8", filter.getEncoding());
+ Assert.assertEquals(10_000, filter.getStringDecodingCacheSize());
+ }
+
+ @Test
+ public void testInFilterMultipleValues()
+ {
+ InDimFilter dimFilter = new InDimFilter("service", Arrays.asList("user-service", "payment-service"), null);
+ KafkaHeaderBasedFilterConfig filter = new KafkaHeaderBasedFilterConfig(dimFilter, "ISO-8859-1", null);
+
+ Assert.assertEquals(dimFilter, filter.getFilter());
+ Assert.assertEquals("ISO-8859-1", filter.getEncoding());
+ Assert.assertEquals(10_000, filter.getStringDecodingCacheSize());
+ }
+
+ @Test
+ public void testInFilterWithCustomCacheSize()
+ {
+ InDimFilter dimFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig filter = new KafkaHeaderBasedFilterConfig(dimFilter, null, 50_000);
+
+ Assert.assertEquals(dimFilter, filter.getFilter());
+ Assert.assertEquals("UTF-8", filter.getEncoding());
+ Assert.assertEquals(50_000, filter.getStringDecodingCacheSize());
+ }
+
+ @Test
+ public void testSelectorFilterRejected()
+ {
+ SelectorDimFilter dimFilter = new SelectorDimFilter("environment", "production", null);
+ try {
+ new KafkaHeaderBasedFilterConfig(dimFilter, null, null);
+ Assert.fail("Expected DruidException for SelectorDimFilter");
+ }
+ catch (DruidException e) {
+ Assert.assertTrue("Should mention unsupported filter type", e.getMessage().contains("Unsupported filter type"));
+ Assert.assertTrue("Should mention SelectorDimFilter", e.getMessage().contains("SelectorDimFilter"));
+ }
+ }
+
+ @Test
+ public void testAndFilterRejected()
+ {
+ SelectorDimFilter envFilter = new SelectorDimFilter("environment", "production", null);
+ SelectorDimFilter serviceFilter = new SelectorDimFilter("service", "user-service", null);
+ AndDimFilter andFilter = new AndDimFilter(Arrays.asList(envFilter, serviceFilter));
+ try {
+ new KafkaHeaderBasedFilterConfig(andFilter, null, null);
+ Assert.fail("Expected DruidException for AndDimFilter");
+ }
+ catch (DruidException e) {
+ Assert.assertTrue("Should mention unsupported filter type", e.getMessage().contains("Unsupported filter type"));
+ Assert.assertTrue("Should mention AndDimFilter", e.getMessage().contains("AndDimFilter"));
+ }
+ }
+
+ @Test
+ public void testNotFilterRejected()
+ {
+ SelectorDimFilter debugFilter = new SelectorDimFilter("debug-mode", "true", null);
+ NotDimFilter notFilter = new NotDimFilter(debugFilter);
+ try {
+ new KafkaHeaderBasedFilterConfig(notFilter, null, null);
+ Assert.fail("Expected DruidException for NotDimFilter");
+ }
+ catch (DruidException e) {
+ Assert.assertTrue("Should mention unsupported filter type", e.getMessage().contains("Unsupported filter type"));
+ Assert.assertTrue("Should mention NotDimFilter", e.getMessage().contains("NotDimFilter"));
+ }
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testNullFilter()
+ {
+ new KafkaHeaderBasedFilterConfig(null, null, null);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testInvalidEncoding()
+ {
+ InDimFilter dimFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ new KafkaHeaderBasedFilterConfig(dimFilter, "INVALID-ENCODING", null);
+ }
+
+ @Test
+ public void testSerialization() throws Exception
+ {
+ InDimFilter dimFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig originalFilter = new KafkaHeaderBasedFilterConfig(dimFilter, "UTF-16", null);
+
+ // Serialize to JSON
+ String json = objectMapper.writeValueAsString(originalFilter);
+
+ // Deserialize back
+ KafkaHeaderBasedFilterConfig deserializedFilter = objectMapper.readValue(json, KafkaHeaderBasedFilterConfig.class);
+
+ Assert.assertEquals(originalFilter.getFilter(), deserializedFilter.getFilter());
+ Assert.assertEquals(originalFilter.getEncoding(), deserializedFilter.getEncoding());
+ Assert.assertEquals(originalFilter.getStringDecodingCacheSize(), deserializedFilter.getStringDecodingCacheSize());
+ }
+
+ @Test
+ public void testEquals()
+ {
+ InDimFilter dimFilter1 = new InDimFilter("environment", Collections.singletonList("production"), null);
+ InDimFilter dimFilter2 = new InDimFilter("environment", Collections.singletonList("production"), null);
+ InDimFilter dimFilter3 = new InDimFilter("environment", Collections.singletonList("staging"), null);
+
+ KafkaHeaderBasedFilterConfig filter1 = new KafkaHeaderBasedFilterConfig(dimFilter1, "UTF-8", null);
+ KafkaHeaderBasedFilterConfig filter2 = new KafkaHeaderBasedFilterConfig(dimFilter2, "UTF-8", null);
+ KafkaHeaderBasedFilterConfig filter3 = new KafkaHeaderBasedFilterConfig(dimFilter3, "UTF-8", null);
+ KafkaHeaderBasedFilterConfig filter4 = new KafkaHeaderBasedFilterConfig(dimFilter1, "UTF-16", null);
+ KafkaHeaderBasedFilterConfig filter5 = new KafkaHeaderBasedFilterConfig(dimFilter1, "UTF-8", 5000);
+
+ Assert.assertEquals(filter1, filter2);
+ Assert.assertNotEquals(filter1, filter3);
+ Assert.assertNotEquals(filter1, filter4);
+ Assert.assertNotEquals(filter1, filter5); // Different cache size
+ Assert.assertNotEquals(filter1, null);
+ Assert.assertNotEquals(filter1, "string");
+ }
+
+ @Test
+ public void testToString()
+ {
+ InDimFilter dimFilter = new InDimFilter("environment", Collections.singletonList("production"), null);
+ KafkaHeaderBasedFilterConfig filter = new KafkaHeaderBasedFilterConfig(dimFilter, "UTF-8", null);
+
+ String toString = filter.toString();
+ Assert.assertTrue(toString.contains("KafkaHeaderBasedFilterConfig"));
+ Assert.assertTrue(toString.contains("filter="));
+ Assert.assertTrue(toString.contains("encoding='UTF-8'"));
+ Assert.assertTrue(toString.contains("stringDecodingCacheSize=10000"));
+ }
+}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
index c2aee78b3cbd..ee4dbd8fc9f0 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfigTest.java
@@ -338,7 +338,8 @@ public void testAutoScalingConfigSerde() throws JsonProcessingException
null,
null,
null,
- null
+ null,
+ null
);
String ioConfig = mapper.writeValueAsString(kafkaSupervisorIOConfig);
KafkaSupervisorIOConfig kafkaSupervisorIOConfig1 = mapper.readValue(ioConfig, KafkaSupervisorIOConfig.class);
@@ -380,6 +381,7 @@ public void testIdleConfigSerde() throws JsonProcessingException
null,
null,
null,
+ null,
mapper.convertValue(idleConfig, IdleConfig.class),
null
);
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
index a7d477883259..600e952b4ce8 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
@@ -320,6 +320,7 @@ public SeekableStreamIndexTaskClient build(
null,
null,
null,
+ null,
new IdleConfig(true, 1000L),
1
);
@@ -480,6 +481,7 @@ public void testCreateBaseTaskContexts() throws JsonProcessingException
null,
null,
INPUT_FORMAT,
+ null,
null
),
new KafkaIndexTaskTuningConfig(
@@ -4847,6 +4849,7 @@ private TestableKafkaSupervisor getTestableSupervisor(
earlyMessageRejectionPeriod,
null,
null,
+ null,
idleConfig,
null
);
@@ -4962,6 +4965,7 @@ private TestableKafkaSupervisor getTestableSupervisorCustomIsTaskCurrent(
null,
null,
null,
+ null,
null
);
@@ -5079,6 +5083,7 @@ private KafkaSupervisor createSupervisor(
null,
null,
null,
+ null,
null
);
@@ -5222,13 +5227,14 @@ private KafkaIndexTask createKafkaIndexTask(
minimumMessageTime,
maximumMessageTime,
INPUT_FORMAT,
+ null,
null
),
Collections.emptyMap(),
OBJECT_MAPPER
);
}
-
+
private static ImmutableMap singlePartitionMap(String topic, int partition, long offset)
{
return ImmutableMap.of(new KafkaTopicPartition(false, topic, partition), offset);
diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml
index 00995fd06aeb..d2499110b53a 100644
--- a/extensions-core/kinesis-indexing-service/pom.xml
+++ b/extensions-core/kinesis-indexing-service/pom.xml
@@ -195,4 +195,18 @@
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ com.amazonaws:amazon-kinesis-client:jar:1.14.4
+
+
+
+
+
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 c1adf018b216..a1ced1c2eedc 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
@@ -1191,7 +1191,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.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1277,7 +1278,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.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/DefaultK8sApiClient.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/DefaultK8sApiClient.java
index 00ad6b76abb2..dec7e09d5c39 100644
--- a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/DefaultK8sApiClient.java
+++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/DefaultK8sApiClient.java
@@ -36,6 +36,9 @@
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.logger.Logger;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
import java.io.IOException;
import java.net.SocketTimeoutException;
@@ -76,7 +79,8 @@ public void patchPod(String podName, String podNamespace, String jsonPatchStr)
public DiscoveryDruidNodeList listPods(
String podNamespace,
String labelSelector,
- NodeRole nodeRole
+ NodeRole nodeRole,
+ @Nullable Duration terminatingStateCheckDuration
)
{
try {
@@ -85,6 +89,26 @@ public DiscoveryDruidNodeList listPods(
Map allNodes = new HashMap();
for (V1Pod podDef : podList.getItems()) {
+ // irrespective of the grace period, we skip the pod if it has been in termination for more than terminatingStateCheckDuration
+ if (podDef.getMetadata() != null && podDef.getMetadata().getDeletionTimestamp() != null) {
+ long deletionTimestamp = podDef.getMetadata().getDeletionTimestamp().toInstant().toEpochMilli();
+ long currentTimestamp = System.currentTimeMillis();
+ long terminationGracePeriod = podDef.getSpec().getTerminationGracePeriodSeconds() != null ?
+ podDef.getSpec().getTerminationGracePeriodSeconds() * 1000L : 30 * 1000L; // Default to 30s if graceperiod is not set
+ long checkDuration = terminatingStateCheckDuration != null ?
+ terminatingStateCheckDuration.getMillis() : 30 * 1000L; // Default to 30s if not specified
+
+ if (currentTimestamp - deletionTimestamp + terminationGracePeriod > checkDuration) {
+ LOGGER.info(
+ "Skipping pod %s/%s from discovery as it has been in termination for more than grace period + %d seconds",
+ podDef.getMetadata().getNamespace(),
+ podDef.getMetadata().getName(),
+ checkDuration / 1000
+ );
+ continue;
+ }
+ }
+
DiscoveryDruidNode node = getDiscoveryDruidNodeFromPodDef(nodeRole, podDef);
allNodes.put(node.getDruidNode().getHostAndPortToUse(), node);
}
diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sApiClient.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sApiClient.java
index 1e61677420c8..4f4efebdbbb8 100644
--- a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sApiClient.java
+++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sApiClient.java
@@ -20,6 +20,9 @@
package org.apache.druid.k8s.discovery;
import org.apache.druid.discovery.NodeRole;
+import org.joda.time.Duration;
+
+import javax.annotation.Nullable;
/**
* Interface to abstract pod read/update with K8S API Server to allow unit tests with mock impl.
@@ -28,7 +31,7 @@ public interface K8sApiClient
{
void patchPod(String podName, String namespace, String jsonPatchStr);
- DiscoveryDruidNodeList listPods(String namespace, String labelSelector, NodeRole nodeRole);
+ DiscoveryDruidNodeList listPods(String namespace, String labelSelector, NodeRole nodeRole, @Nullable Duration terminatingStateCheckDuration);
/**
* @return NULL if history not available or else return the {@link WatchResult} object
diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfig.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfig.java
index 998b8641c83a..ab9302a5c420 100644
--- a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfig.java
+++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfig.java
@@ -60,6 +60,12 @@ public class K8sDiscoveryConfig
@JsonProperty
private final Duration retryPeriod;
+ @JsonProperty
+ private final Duration terminatingStateCheckDuration;
+
+ @JsonProperty
+ private final Duration periodicListInterval;
+
@JsonCreator
public K8sDiscoveryConfig(
@JsonProperty("clusterIdentifier") String clusterIdentifier,
@@ -69,7 +75,9 @@ public K8sDiscoveryConfig(
@JsonProperty("overlordLeaderElectionConfigMapNamespace") String overlordLeaderElectionConfigMapNamespace,
@JsonProperty("leaseDuration") Duration leaseDuration,
@JsonProperty("renewDeadline") Duration renewDeadline,
- @JsonProperty("retryPeriod") Duration retryPeriod
+ @JsonProperty("retryPeriod") Duration retryPeriod,
+ @JsonProperty("terminatingStateCheckDuration") Duration terminatingStateCheckDuration,
+ @JsonProperty("periodicListInterval") Duration periodicListInterval
)
{
Preconditions.checkArgument(clusterIdentifier != null && !clusterIdentifier.isEmpty(), "null/empty clusterIdentifier");
@@ -97,6 +105,8 @@ public K8sDiscoveryConfig(
this.leaseDuration = leaseDuration == null ? Duration.millis(60000) : leaseDuration;
this.renewDeadline = renewDeadline == null ? Duration.millis(17000) : renewDeadline;
this.retryPeriod = retryPeriod == null ? Duration.millis(5000) : retryPeriod;
+ this.terminatingStateCheckDuration = terminatingStateCheckDuration == null ? Duration.standardSeconds(30) : terminatingStateCheckDuration;
+ this.periodicListInterval = periodicListInterval == null ? Duration.standardMinutes(1) : periodicListInterval;
}
@JsonProperty
@@ -147,6 +157,18 @@ public Duration getRetryPeriod()
return retryPeriod;
}
+ @JsonProperty
+ public Duration getTerminatingStateCheckDuration()
+ {
+ return terminatingStateCheckDuration;
+ }
+
+ @JsonProperty
+ public Duration getPeriodicListInterval()
+ {
+ return periodicListInterval;
+ }
+
@Override
public String toString()
{
@@ -159,6 +181,8 @@ public String toString()
", leaseDuration=" + leaseDuration +
", renewDeadline=" + renewDeadline +
", retryPeriod=" + retryPeriod +
+ ", terminatingStateCheckDuration=" + terminatingStateCheckDuration +
+ ", periodicListInterval=" + periodicListInterval +
'}';
}
@@ -185,7 +209,9 @@ public boolean equals(Object o)
) &&
Objects.equals(leaseDuration, that.leaseDuration) &&
Objects.equals(renewDeadline, that.renewDeadline) &&
- Objects.equals(retryPeriod, that.retryPeriod);
+ Objects.equals(retryPeriod, that.retryPeriod) &&
+ Objects.equals(terminatingStateCheckDuration, that.terminatingStateCheckDuration) &&
+ Objects.equals(periodicListInterval, that.periodicListInterval);
}
@Override
@@ -199,7 +225,9 @@ public int hashCode()
overlordLeaderElectionConfigMapNamespace,
leaseDuration,
renewDeadline,
- retryPeriod
+ retryPeriod,
+ terminatingStateCheckDuration,
+ periodicListInterval
);
}
}
diff --git a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProvider.java b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProvider.java
index 559b53b60809..a38b04802438 100644
--- a/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProvider.java
+++ b/extensions-core/kubernetes-extensions/src/main/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProvider.java
@@ -98,7 +98,7 @@ public BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole)
return () -> k8sApiClient.listPods(
podInfo.getPodNamespace(),
K8sDruidNodeAnnouncer.getLabelSelectorForNode(discoveryConfig, nodeRole, node),
- nodeRole
+ nodeRole, discoveryConfig.getTerminatingStateCheckDuration()
).getDruidNodes().containsKey(node.getHostAndPortToUse());
}
@@ -225,9 +225,32 @@ private void watch()
return;
}
+ // Create a scheduled executor for periodic listing
+ ScheduledExecutorService periodicListExecutor = Execs.scheduledSingleThreaded(
+ "K8sDruidNodeDiscoveryProvider-PeriodicList-" + nodeRole.getJsonName()
+ );
+
+ // Schedule periodic listing every minute
+ periodicListExecutor.scheduleAtFixedRate(() -> {
+ try {
+ if (lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
+ LOGGER.info("Performing periodic pod listing for NodeRole [%s]", nodeRole);
+ DiscoveryDruidNodeList list = k8sApiClient.listPods(
+ podInfo.getPodNamespace(),
+ labelSelector,
+ nodeRole, discoveryConfig.getTerminatingStateCheckDuration()
+ );
+ baseNodeRoleWatcher.resetNodes(list.getDruidNodes());
+ }
+ }
+ catch (Throwable ex) {
+ LOGGER.error(ex, "Error during periodic pod listing for NodeRole [%s]", nodeRole);
+ }
+ }, 120000, discoveryConfig.getPeriodicListInterval().getMillis(), TimeUnit.MILLISECONDS);
+
while (lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
try {
- DiscoveryDruidNodeList list = k8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, nodeRole);
+ DiscoveryDruidNodeList list = k8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, nodeRole, discoveryConfig.getTerminatingStateCheckDuration());
baseNodeRoleWatcher.resetNodes(list.getDruidNodes());
if (!cacheInitialized) {
@@ -242,13 +265,15 @@ private void watch()
);
}
catch (Throwable ex) {
- LOGGER.error(ex, "Expection while watching for NodeRole [%s].", nodeRole);
+ LOGGER.error(ex, "Exception while watching for NodeRole [%s].", nodeRole);
// Wait a little before trying again.
sleep(watcherErrorRetryWaitMS);
}
}
+ // Shutdown the periodic executor when the watch is stopped
+ periodicListExecutor.shutdownNow();
LOGGER.info("Exited Watch for NodeRole [%s].", nodeRole);
}
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sAnnouncerAndDiscoveryIntTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sAnnouncerAndDiscoveryIntTest.java
index e7752d757b73..9b8c8f96c890 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sAnnouncerAndDiscoveryIntTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sAnnouncerAndDiscoveryIntTest.java
@@ -52,7 +52,7 @@ public class K8sAnnouncerAndDiscoveryIntTest
private final PodInfo podInfo = new PodInfo("busybox", "default");
- private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null);
+ private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null, null, null);
@Test(timeout = 30000L)
public void testAnnouncementAndDiscoveryWorkflow() throws Exception
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfigTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfigTest.java
index b76ae4b62462..41ef2bc37408 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfigTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDiscoveryConfigTest.java
@@ -34,7 +34,7 @@ public void testDefaultValuesSerde() throws Exception
{
testSerde(
"{\"clusterIdentifier\": \"test-cluster\"}\n",
- new K8sDiscoveryConfig("test-cluster", null, null, null, null, null, null, null)
+ new K8sDiscoveryConfig("test-cluster", null, null, null, null, null, null, null, null, null)
);
}
@@ -50,8 +50,10 @@ public void testCustomizedValuesSerde() throws Exception
+ " \"overlordLeaderElectionConfigMapNamespace\": \"overlordns\",\n"
+ " \"leaseDuration\": \"PT3S\",\n"
+ " \"renewDeadline\": \"PT2S\",\n"
- + " \"retryPeriod\": \"PT1S\"\n"
- + "}\n",
+ + " \"retryPeriod\": \"PT1S\",\n"
+ + " \"terminatingStateCheckDuration\": \"PT30S\",\n"
+ + " \"periodicListInterval\": \"PT20S\"\n"
+ + "\n}",
new K8sDiscoveryConfig(
"test-cluster",
"PODNAMETEST",
@@ -60,7 +62,9 @@ public void testCustomizedValuesSerde() throws Exception
"overlordns",
Duration.millis(3000),
Duration.millis(2000),
- Duration.millis(1000)
+ Duration.millis(1000),
+ Duration.millis(30000),
+ Duration.millis(20000)
)
);
}
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderElectionIntTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderElectionIntTest.java
index 168c0625cda3..7e19c5efd968 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderElectionIntTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderElectionIntTest.java
@@ -54,7 +54,7 @@ public class K8sDruidLeaderElectionIntTest
);
private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, "default", "default",
- Duration.millis(10_000), Duration.millis(7_000), Duration.millis(3_000));
+ Duration.millis(10_000), Duration.millis(7_000), Duration.millis(3_000), null, null);
private final ApiClient k8sApiClient;
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorTest.java
index a500502524fb..96f66b7ffbd3 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidLeaderSelectorTest.java
@@ -38,7 +38,7 @@ public class K8sDruidLeaderSelectorTest
);
private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null,
- "default", "default", Duration.millis(10_000), Duration.millis(7_000), Duration.millis(3_000));
+ "default", "default", Duration.millis(10_000), Duration.millis(7_000), Duration.millis(3_000), null, null);
private final String lockResourceName = "druid-leader-election";
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeAnnouncerTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeAnnouncerTest.java
index ffcaefb56177..0f8df3c23147 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeAnnouncerTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeAnnouncerTest.java
@@ -47,7 +47,7 @@ public class K8sDruidNodeAnnouncerTest
private final PodInfo podInfo = new PodInfo("testpod", "testns");
- private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null);
+ private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null, null, null);
@Test
public void testAnnounce() throws Exception
diff --git a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProviderTest.java b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProviderTest.java
index 4a9c5d041642..9733bfcd5d9a 100644
--- a/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProviderTest.java
+++ b/extensions-core/kubernetes-extensions/src/test/java/org/apache/druid/k8s/discovery/K8sDruidNodeDiscoveryProviderTest.java
@@ -30,6 +30,7 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.DruidNode;
import org.easymock.EasyMock;
+import org.joda.time.Duration;
import org.junit.Assert;
import org.junit.Test;
@@ -76,14 +77,14 @@ public class K8sDruidNodeDiscoveryProviderTest
private final PodInfo podInfo = new PodInfo("testpod", "testns");
- private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null);
+ private final K8sDiscoveryConfig discoveryConfig = new K8sDiscoveryConfig("druid-cluster", null, null, null, null, null, null, null, null, null);
@Test(timeout = 60_000)
public void testGetForNodeRole() throws Exception
{
String labelSelector = "druidDiscoveryAnnouncement-cluster-identifier=druid-cluster,druidDiscoveryAnnouncement-router=true";
K8sApiClient mockK8sApiClient = EasyMock.createMock(K8sApiClient.class);
- EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER)).andReturn(
+ EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER, Duration.millis(30000))).andReturn(
new DiscoveryDruidNodeList(
"v1",
ImmutableMap.of(
@@ -94,7 +95,7 @@ public void testGetForNodeRole() throws Exception
);
EasyMock.expect(mockK8sApiClient.watchPods(
podInfo.getPodNamespace(), labelSelector, "v1", NodeRole.ROUTER)).andReturn(null);
- EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER)).andReturn(
+ EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER, Duration.millis(30000))).andReturn(
new DiscoveryDruidNodeList(
"v2",
ImmutableMap.of(
@@ -168,7 +169,7 @@ public void testNodeRoleWatcherHandlesNullFromAPIByRestarting() throws Exception
{
String labelSelector = "druidDiscoveryAnnouncement-cluster-identifier=druid-cluster,druidDiscoveryAnnouncement-router=true";
K8sApiClient mockK8sApiClient = EasyMock.createMock(K8sApiClient.class);
- EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER)).andReturn(
+ EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER, Duration.millis(30000))).andReturn(
new DiscoveryDruidNodeList(
"v1",
ImmutableMap.of(
@@ -187,7 +188,7 @@ public void testNodeRoleWatcherHandlesNullFromAPIByRestarting() throws Exception
false
)
);
- EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER)).andReturn(
+ EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER, Duration.millis(30000))).andReturn(
new DiscoveryDruidNodeList(
"v2",
ImmutableMap.of(
@@ -231,7 +232,7 @@ public void testNodeRoleWatcherLoopOnNullItems() throws Exception
{
String labelSelector = "druidDiscoveryAnnouncement-cluster-identifier=druid-cluster,druidDiscoveryAnnouncement-router=true";
K8sApiClient mockK8sApiClient = EasyMock.createMock(K8sApiClient.class);
- EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER)).andReturn(
+ EasyMock.expect(mockK8sApiClient.listPods(podInfo.getPodNamespace(), labelSelector, NodeRole.ROUTER, Duration.millis(30000))).andReturn(
new DiscoveryDruidNodeList(
"v1",
ImmutableMap.of(
diff --git a/extensions-core/multi-stage-query/pom.xml b/extensions-core/multi-stage-query/pom.xml
index ec75e1f99034..2959fa5b378f 100644
--- a/extensions-core/multi-stage-query/pom.xml
+++ b/extensions-core/multi-stage-query/pom.xml
@@ -171,7 +171,7 @@
provided
- org.lz4
+ at.yawk.lz4
lz4-java
provided
diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
index ac568322400e..d93512a92974 100644
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
+++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/groupby/GroupByPreShuffleFrameProcessor.java
@@ -152,6 +152,7 @@ protected ReturnOrAwait runWithSegment(final SegmentWithDescriptor segment
groupingEngine.process(
query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())),
mapSegment(segmentHolder.get()).asStorageAdapter(),
+ null,
null
);
@@ -184,6 +185,7 @@ protected ReturnOrAwait runWithInputChannel(
groupingEngine.process(
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)),
mapSegment(frameSegment).asStorageAdapter(),
+ null,
null
);
diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml
index 47e178d19e53..b2e7208aa981 100644
--- a/extensions-core/orc-extensions/pom.xml
+++ b/extensions-core/orc-extensions/pom.xml
@@ -96,8 +96,8 @@
jsr311-api
- javax.xml.bind
- jaxb-api
+ jakarta.xml.bind
+ jakarta.xml.bind-api
org.apache.hadoop
@@ -131,8 +131,8 @@
commons-lang
- javax.xml.bind
- jaxb-api
+ jakarta.xml.bind
+ jakarta.xml.bind-api
org.apache.hadoop
diff --git a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java
index b44d57115cf7..f0b9ba0b4a0b 100644
--- a/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java
+++ b/extensions-core/postgresql-metadata-storage/src/main/java/org/apache/druid/metadata/storage/postgresql/PostgreSQLConnector.java
@@ -42,6 +42,7 @@
import java.sql.SQLException;
import java.util.List;
import java.util.Locale;
+import java.util.Set;
public class PostgreSQLConnector extends SQLMetadataConnector
{
@@ -287,4 +288,24 @@ protected boolean connectorIsTransientException(Throwable e)
}
return false;
}
+
+ /**
+ * This method has been overridden to pass lowercase tableName.
+ * This is done because PostgreSQL creates tables with lowercased names unless explicitly enclosed in double quotes.
+ */
+ @Override
+ protected boolean tableHasColumn(String tableName, String columnName)
+ {
+ return super.tableHasColumn(StringUtils.toLowerCase(tableName), columnName);
+ }
+
+ /**
+ * This method has been overridden to pass lowercase tableName.
+ * This is done because PostgreSQL creates tables with lowercased names unless explicitly enclosed in double quotes.
+ */
+ @Override
+ public Set getIndexOnTable(String tableName)
+ {
+ return super.getIndexOnTable(StringUtils.toLowerCase(tableName));
+ }
}
diff --git a/extensions-core/protobuf-extensions/pom.xml b/extensions-core/protobuf-extensions/pom.xml
index 23695f19d75f..690cc0b10943 100644
--- a/extensions-core/protobuf-extensions/pom.xml
+++ b/extensions-core/protobuf-extensions/pom.xml
@@ -34,7 +34,7 @@
- 2.11.0
+ 2.14.0
3.6.0
@@ -93,8 +93,16 @@
jakarta.ws.rs-api
jakarta.ws.rs
+
+ org.lz4
+ lz4-java
+
+
+ at.yawk.lz4
+ lz4-java
+
io.confluent
kafka-protobuf-provider
@@ -223,6 +231,16 @@
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java
index cfae0eb084b7..b52d13cd518e 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3StorageConfig.java
@@ -36,12 +36,22 @@ public class S3StorageConfig
@JsonProperty("sse")
private final ServerSideEncryption serverSideEncryption;
+ /**
+ * S3 transfer config.
+ *
+ * @see S3StorageDruidModule#configure
+ */
+ @JsonProperty("transfer")
+ private final S3TransferConfig s3TransferConfig;
+
@JsonCreator
public S3StorageConfig(
- @JsonProperty("sse") ServerSideEncryption serverSideEncryption
+ @JsonProperty("sse") ServerSideEncryption serverSideEncryption,
+ @JsonProperty("transfer") S3TransferConfig s3TransferConfig
)
{
this.serverSideEncryption = serverSideEncryption == null ? new NoopServerSideEncryption() : serverSideEncryption;
+ this.s3TransferConfig = s3TransferConfig == null ? new S3TransferConfig() : s3TransferConfig;
}
@JsonProperty("sse")
@@ -49,4 +59,10 @@ public ServerSideEncryption getServerSideEncryption()
{
return serverSideEncryption;
}
+
+ @JsonProperty("transfer")
+ public S3TransferConfig getS3TransferConfig()
+ {
+ return s3TransferConfig;
+ }
}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TransferConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TransferConfig.java
new file mode 100644
index 000000000000..fc8bd8903fad
--- /dev/null
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3TransferConfig.java
@@ -0,0 +1,71 @@
+/*
+ * 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.storage.s3;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import javax.validation.constraints.Min;
+
+/**
+ */
+public class S3TransferConfig
+{
+ @JsonProperty
+ private boolean useTransferManager = false;
+
+ @JsonProperty
+ @Min(1)
+ private long minimumUploadPartSize = 5 * 1024 * 1024L;
+
+ @JsonProperty
+ @Min(1)
+ private long multipartUploadThreshold = 5 * 1024 * 1024L;
+
+ public void setUseTransferManager(boolean useTransferManager)
+ {
+ this.useTransferManager = useTransferManager;
+ }
+
+ public void setMinimumUploadPartSize(long minimumUploadPartSize)
+ {
+ this.minimumUploadPartSize = minimumUploadPartSize;
+ }
+
+ public void setMultipartUploadThreshold(long multipartUploadThreshold)
+ {
+ this.multipartUploadThreshold = multipartUploadThreshold;
+ }
+
+ public boolean isUseTransferManager()
+ {
+ return useTransferManager;
+ }
+
+ public long getMinimumUploadPartSize()
+ {
+ return minimumUploadPartSize;
+ }
+
+ public long getMultipartUploadThreshold()
+ {
+ return multipartUploadThreshold;
+ }
+
+}
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java
index 65cf8e04249e..452f8209d592 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java
@@ -92,6 +92,9 @@ public boolean apply(Throwable e)
} else if (e instanceof SdkClientException && e.getMessage().contains("Unable to execute HTTP request")) {
// This is likely due to a temporary DNS issue and can be retried.
return true;
+ } else if (e instanceof InterruptedException) {
+ Thread.interrupted(); // Clear interrupted state and not retry
+ return false;
} else if (e instanceof AmazonClientException) {
return AWSClientUtil.isClientExceptionRecoverable((AmazonClientException) e);
} else {
@@ -343,7 +346,7 @@ static void uploadFileIfPossible(
String bucket,
String key,
File file
- )
+ ) throws InterruptedException
{
final PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, file);
@@ -351,7 +354,7 @@ static void uploadFileIfPossible(
putObjectRequest.setAccessControlList(S3Utils.grantFullControlToBucketOwner(service, bucket));
}
log.info("Pushing [%s] to bucket[%s] and key[%s].", file, bucket, key);
- service.putObject(putObjectRequest);
+ service.upload(putObjectRequest);
}
@Nullable
diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java
index 320a0b9a6f99..d97d8df6c8a8 100644
--- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java
+++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3.java
@@ -43,6 +43,9 @@
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.UploadPartRequest;
import com.amazonaws.services.s3.model.UploadPartResult;
+import com.amazonaws.services.s3.transfer.TransferManager;
+import com.amazonaws.services.s3.transfer.TransferManagerBuilder;
+import com.amazonaws.services.s3.transfer.Upload;
import org.apache.druid.java.util.common.ISE;
import java.io.File;
@@ -65,11 +68,21 @@ public static Builder builder()
private final AmazonS3 amazonS3;
private final ServerSideEncryption serverSideEncryption;
+ private final TransferManager transferManager;
- public ServerSideEncryptingAmazonS3(AmazonS3 amazonS3, ServerSideEncryption serverSideEncryption)
+ public ServerSideEncryptingAmazonS3(AmazonS3 amazonS3, ServerSideEncryption serverSideEncryption, S3TransferConfig transferConfig)
{
this.amazonS3 = amazonS3;
this.serverSideEncryption = serverSideEncryption;
+ if (transferConfig.isUseTransferManager()) {
+ this.transferManager = TransferManagerBuilder.standard()
+ .withS3Client(amazonS3)
+ .withMinimumUploadPartSize(transferConfig.getMinimumUploadPartSize())
+ .withMultipartUploadThreshold(transferConfig.getMultipartUploadThreshold())
+ .build();
+ } else {
+ this.transferManager = null;
+ }
}
public boolean doesObjectExist(String bucket, String objectName)
@@ -168,10 +181,20 @@ public CompleteMultipartUploadResult completeMultipartUpload(CompleteMultipartUp
return amazonS3.completeMultipartUpload(request);
}
+ public void upload(PutObjectRequest request) throws InterruptedException
+ {
+ if (transferManager == null) {
+ putObject(request);
+ } else {
+ Upload transfer = transferManager.upload(serverSideEncryption.decorate(request));
+ transfer.waitForCompletion();
+ }
+ }
+
public static class Builder
{
private AmazonS3ClientBuilder amazonS3ClientBuilder = AmazonS3Client.builder();
- private S3StorageConfig s3StorageConfig = new S3StorageConfig(new NoopServerSideEncryption());
+ private S3StorageConfig s3StorageConfig = new S3StorageConfig(new NoopServerSideEncryption(), null);
public Builder setAmazonS3ClientBuilder(AmazonS3ClientBuilder amazonS3ClientBuilder)
{
@@ -204,7 +227,7 @@ public ServerSideEncryptingAmazonS3 build()
throw new ISE("S3StorageConfig cannot be null!");
}
- return new ServerSideEncryptingAmazonS3(amazonS3ClientBuilder.build(), s3StorageConfig.getServerSideEncryption());
+ return new ServerSideEncryptingAmazonS3(amazonS3ClientBuilder.build(), s3StorageConfig.getServerSideEncryption(), s3StorageConfig.getS3TransferConfig());
}
}
}
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java
index 986627f80bf0..989bae527669 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/data/input/s3/S3InputSourceTest.java
@@ -74,6 +74,7 @@
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.storage.s3.NoopServerSideEncryption;
import org.apache.druid.storage.s3.S3InputDataConfig;
+import org.apache.druid.storage.s3.S3TransferConfig;
import org.apache.druid.storage.s3.S3Utils;
import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
import org.apache.druid.testing.InitializedNullHandlingTest;
@@ -113,7 +114,8 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
public static final AmazonS3ClientBuilder AMAZON_S3_CLIENT_BUILDER = AmazonS3Client.builder();
public static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
S3_CLIENT,
- new NoopServerSideEncryption()
+ new NoopServerSideEncryption(),
+ new S3TransferConfig()
);
public static final S3InputDataConfig INPUT_DATA_CONFIG;
private static final int MAX_LISTING_LENGTH = 10;
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ObjectSummaryIteratorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ObjectSummaryIteratorTest.java
index ea2ca4af26c1..8ee6c826718d 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ObjectSummaryIteratorTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ObjectSummaryIteratorTest.java
@@ -195,7 +195,7 @@ private static ServerSideEncryptingAmazonS3 makeMockClient(
final List objects
)
{
- return new ServerSideEncryptingAmazonS3(null, null)
+ return new ServerSideEncryptingAmazonS3(null, null, new S3TransferConfig())
{
@Override
public ListObjectsV2Result listObjectsV2(final ListObjectsV2Request request)
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java
index f5005c706e01..4acf553fae50 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java
@@ -76,7 +76,8 @@ public String getArchiveBaseKey()
private static final Supplier S3_SERVICE = Suppliers.ofInstance(
new ServerSideEncryptingAmazonS3(
EasyMock.createStrictMock(AmazonS3Client.class),
- new NoopServerSideEncryption()
+ new NoopServerSideEncryption(),
+ new S3TransferConfig()
)
);
private static final S3DataSegmentPuller PULLER = new S3DataSegmentPuller(S3_SERVICE.get());
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
index 550a72cef43c..8f653e956a83 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
@@ -201,7 +201,7 @@ private static class MockAmazonS3Client extends ServerSideEncryptingAmazonS3
private MockAmazonS3Client()
{
- super(new AmazonS3Client(), new NoopServerSideEncryption());
+ super(new AmazonS3Client(), new NoopServerSideEncryption(), new S3TransferConfig());
}
public boolean didMove()
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentPusherTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentPusherTest.java
index ba1aba1305ee..ad526f0b7e7d 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentPusherTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentPusherTest.java
@@ -25,7 +25,7 @@
import com.amazonaws.services.s3.model.Grant;
import com.amazonaws.services.s3.model.Owner;
import com.amazonaws.services.s3.model.Permission;
-import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.model.PutObjectRequest;
import com.google.common.io.Files;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.Intervals;
@@ -41,9 +41,9 @@
import org.junit.rules.TemporaryFolder;
import java.io.File;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.function.Consumer;
import java.util.regex.Pattern;
/**
@@ -59,11 +59,7 @@ public void testPush() throws Exception
{
testPushInternal(
false,
- "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip",
- client ->
- EasyMock.expect(client.putObject(EasyMock.anyObject()))
- .andReturn(new PutObjectResult())
- .once()
+ "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip"
);
}
@@ -72,11 +68,7 @@ public void testPushUseUniquePath() throws Exception
{
testPushInternal(
true,
- "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/[A-Za-z0-9-]{36}/index\\.zip",
- client ->
- EasyMock.expect(client.putObject(EasyMock.anyObject()))
- .andReturn(new PutObjectResult())
- .once()
+ "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/[A-Za-z0-9-]{36}/index\\.zip"
);
}
@@ -86,16 +78,9 @@ public void testEntityTooLarge()
final DruidException exception = Assert.assertThrows(
DruidException.class,
() ->
- testPushInternal(
+ testPushInternalForEntityTooLarge(
false,
- "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip",
- client -> {
- final AmazonS3Exception e = new AmazonS3Exception("whoa too many bytes");
- e.setErrorCode(S3Utils.ERROR_ENTITY_TOO_LARGE);
- EasyMock.expect(client.putObject(EasyMock.anyObject()))
- .andThrow(e)
- .once();
- }
+ "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip"
)
);
@@ -105,11 +90,7 @@ public void testEntityTooLarge()
);
}
- private void testPushInternal(
- boolean useUniquePath,
- String matcher,
- Consumer clientDecorator
- ) throws Exception
+ private void testPushInternal(boolean useUniquePath, String matcher) throws Exception
{
ServerSideEncryptingAmazonS3 s3Client = EasyMock.createStrictMock(ServerSideEncryptingAmazonS3.class);
@@ -118,10 +99,36 @@ private void testPushInternal(
acl.grantAllPermissions(new Grant(new CanonicalGrantee(acl.getOwner().getId()), Permission.FullControl));
EasyMock.expect(s3Client.getBucketAcl(EasyMock.eq("bucket"))).andReturn(acl).once();
- clientDecorator.accept(s3Client);
+ s3Client.upload(EasyMock.anyObject(PutObjectRequest.class));
+ EasyMock.expectLastCall().once();
EasyMock.replay(s3Client);
+ validate(useUniquePath, matcher, s3Client);
+ }
+
+ private void testPushInternalForEntityTooLarge(boolean useUniquePath, String matcher) throws Exception
+ {
+ ServerSideEncryptingAmazonS3 s3Client = EasyMock.createStrictMock(ServerSideEncryptingAmazonS3.class);
+ final AmazonS3Exception e = new AmazonS3Exception("whoa too many bytes");
+ e.setErrorCode(S3Utils.ERROR_ENTITY_TOO_LARGE);
+
+
+ final AccessControlList acl = new AccessControlList();
+ acl.setOwner(new Owner("ownerId", "owner"));
+ acl.grantAllPermissions(new Grant(new CanonicalGrantee(acl.getOwner().getId()), Permission.FullControl));
+ EasyMock.expect(s3Client.getBucketAcl(EasyMock.eq("bucket"))).andReturn(acl).once();
+
+ s3Client.upload(EasyMock.anyObject(PutObjectRequest.class));
+ EasyMock.expectLastCall().andThrow(e).once();
+
+ EasyMock.replay(s3Client);
+
+ validate(useUniquePath, matcher, s3Client);
+ }
+
+ private void validate(boolean useUniquePath, String matcher, ServerSideEncryptingAmazonS3 s3Client) throws IOException
+ {
S3DataSegmentPusherConfig config = new S3DataSegmentPusherConfig();
config.setBucket("bucket");
config.setBaseKey("key");
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java
index 9f9d632f6181..790a4f1a2643 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3StorageConnectorProviderTest.java
@@ -145,7 +145,7 @@ public void configure(Binder binder)
new InjectableValues.Std()
.addValue(
ServerSideEncryptingAmazonS3.class,
- new ServerSideEncryptingAmazonS3(null, new NoopServerSideEncryption())
+ new ServerSideEncryptingAmazonS3(null, new NoopServerSideEncryption(), new S3TransferConfig())
));
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java
index 011dc4888456..f4a95eabb328 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TaskLogsTest.java
@@ -29,7 +29,6 @@
import com.amazonaws.services.s3.model.Owner;
import com.amazonaws.services.s3.model.Permission;
import com.amazonaws.services.s3.model.PutObjectRequest;
-import com.amazonaws.services.s3.model.PutObjectResult;
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.google.common.base.Optional;
@@ -123,11 +122,10 @@ public void testTaskLogsPushWithAclEnabled() throws Exception
}
@Test
- public void test_pushTaskStatus() throws IOException
+ public void test_pushTaskStatus() throws IOException, InterruptedException
{
- EasyMock.expect(s3Client.putObject(EasyMock.anyObject(PutObjectRequest.class)))
- .andReturn(new PutObjectResult())
- .once();
+ s3Client.upload(EasyMock.anyObject(PutObjectRequest.class));
+ EasyMock.expectLastCall().once();
EasyMock.replay(s3Client);
@@ -148,12 +146,11 @@ public void test_pushTaskStatus() throws IOException
}
@Test
- public void test_pushTaskPayload() throws IOException
+ public void test_pushTaskPayload() throws IOException, InterruptedException
{
Capture putObjectRequestCapture = Capture.newInstance(CaptureType.FIRST);
- EasyMock.expect(s3Client.putObject(EasyMock.capture(putObjectRequestCapture)))
- .andReturn(new PutObjectResult())
- .once();
+ s3Client.upload(EasyMock.capture(putObjectRequestCapture));
+ EasyMock.expectLastCall().once();
EasyMock.replay(s3Client);
@@ -617,9 +614,8 @@ private S3TaskLogs getS3TaskLogs()
private List testPushInternal(boolean disableAcl, String ownerId, String ownerDisplayName) throws Exception
{
- EasyMock.expect(s3Client.putObject(EasyMock.anyObject()))
- .andReturn(new PutObjectResult())
- .once();
+ s3Client.upload(EasyMock.anyObject(PutObjectRequest.class));
+ EasyMock.expectLastCall().once();
AccessControlList aclExpected = new AccessControlList();
aclExpected.setOwner(new Owner(ownerId, ownerDisplayName));
@@ -628,9 +624,8 @@ private List testPushInternal(boolean disableAcl, String ownerId, String
.andReturn(aclExpected)
.once();
- EasyMock.expect(s3Client.putObject(EasyMock.anyObject(PutObjectRequest.class)))
- .andReturn(new PutObjectResult())
- .once();
+ s3Client.upload(EasyMock.anyObject(PutObjectRequest.class));
+ EasyMock.expectLastCall().once();
EasyMock.replay(s3Client);
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TransferConfigTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TransferConfigTest.java
new file mode 100644
index 000000000000..7af20431ab23
--- /dev/null
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3TransferConfigTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.storage.s3;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class S3TransferConfigTest
+{
+ @Test
+ public void testDefaultValues()
+ {
+ S3TransferConfig config = new S3TransferConfig();
+ Assert.assertFalse(config.isUseTransferManager());
+ Assert.assertEquals(5 * 1024 * 1024L, config.getMinimumUploadPartSize());
+ Assert.assertEquals(5 * 1024 * 1024L, config.getMultipartUploadThreshold());
+ }
+
+ @Test
+ public void testSetUseTransferManager()
+ {
+ S3TransferConfig config = new S3TransferConfig();
+ config.setUseTransferManager(true);
+ Assert.assertTrue(config.isUseTransferManager());
+ }
+
+ @Test
+ public void testSetMinimumUploadPartSize()
+ {
+ S3TransferConfig config = new S3TransferConfig();
+ config.setMinimumUploadPartSize(10 * 1024 * 1024L);
+ Assert.assertEquals(10 * 1024 * 1024L, config.getMinimumUploadPartSize());
+ }
+
+ @Test
+ public void testSetMultipartUploadThreshold()
+ {
+ S3TransferConfig config = new S3TransferConfig();
+ config.setMultipartUploadThreshold(10 * 1024 * 1024L);
+ Assert.assertEquals(10 * 1024 * 1024L, config.getMultipartUploadThreshold());
+ }
+}
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3Test.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3Test.java
new file mode 100644
index 000000000000..75e1a72da0d2
--- /dev/null
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/ServerSideEncryptingAmazonS3Test.java
@@ -0,0 +1,148 @@
+/*
+ * 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.storage.s3;
+
+import com.amazonaws.services.s3.AmazonS3;
+import com.amazonaws.services.s3.model.PutObjectRequest;
+import com.amazonaws.services.s3.model.PutObjectResult;
+import com.amazonaws.services.s3.transfer.TransferManager;
+import com.amazonaws.services.s3.transfer.Upload;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+import java.lang.reflect.Field;
+
+
+
+public class ServerSideEncryptingAmazonS3Test
+{
+ private AmazonS3 mockAmazonS3;
+ private ServerSideEncryption mockServerSideEncryption;
+ private S3TransferConfig mockTransferConfig;
+ private TransferManager mockTransferManager;
+
+ @Before
+ public void setup()
+ {
+ mockAmazonS3 = EasyMock.createMock(AmazonS3.class);
+ mockServerSideEncryption = EasyMock.createMock(ServerSideEncryption.class);
+ mockTransferConfig = EasyMock.createMock(S3TransferConfig.class);
+ mockTransferManager = EasyMock.createMock(TransferManager.class);
+ }
+
+ @Test
+ public void testConstructor_WithTransferManager() throws NoSuchFieldException, IllegalAccessException
+ {
+ EasyMock.expect(mockTransferConfig.isUseTransferManager()).andReturn(true);
+ EasyMock.expect(mockTransferConfig.getMinimumUploadPartSize()).andReturn(5L);
+ EasyMock.expect(mockTransferConfig.getMultipartUploadThreshold()).andReturn(10L);
+ EasyMock.replay(mockTransferConfig);
+
+ ServerSideEncryptingAmazonS3 s3 = new ServerSideEncryptingAmazonS3(mockAmazonS3, mockServerSideEncryption, mockTransferConfig);
+
+ Field transferManagerField = ServerSideEncryptingAmazonS3.class.getDeclaredField("transferManager");
+ transferManagerField.setAccessible(true);
+ Object transferManager = transferManagerField.get(s3);
+
+ Assert.assertNotNull("TransferManager should be initialized", transferManager);
+ Assert.assertNotNull(s3);
+ EasyMock.verify(mockTransferConfig);
+ }
+
+ @Test
+ public void testConstructor_WithoutTransferManager() throws NoSuchFieldException, IllegalAccessException
+ {
+
+ EasyMock.expect(mockTransferConfig.isUseTransferManager()).andReturn(false);
+ EasyMock.replay(mockTransferConfig);
+
+ ServerSideEncryptingAmazonS3 s3 = new ServerSideEncryptingAmazonS3(mockAmazonS3, mockServerSideEncryption, mockTransferConfig);
+
+ Field transferManagerField = ServerSideEncryptingAmazonS3.class.getDeclaredField("transferManager");
+ transferManagerField.setAccessible(true);
+ Object transferManager = transferManagerField.get(s3);
+
+ Assert.assertNull("TransferManager should not be initialized", transferManager);
+ Assert.assertNotNull(s3);
+ EasyMock.verify(mockTransferConfig);
+ }
+
+ @Test
+ public void testUpload_WithoutTransferManager() throws InterruptedException
+ {
+ PutObjectRequest originalRequest = new PutObjectRequest("bucket", "key", "file");
+ PutObjectRequest decoratedRequest = new PutObjectRequest("bucket", "key", "file-encrypted");
+ PutObjectResult mockResult = new PutObjectResult();
+
+ EasyMock.expect(mockTransferConfig.isUseTransferManager()).andReturn(false);
+ EasyMock.replay(mockTransferConfig);
+
+ EasyMock.expect(mockServerSideEncryption.decorate(originalRequest)).andReturn(decoratedRequest);
+ EasyMock.replay(mockServerSideEncryption);
+
+ EasyMock.expect(mockAmazonS3.putObject(decoratedRequest)).andReturn(mockResult).once();
+ EasyMock.replay(mockAmazonS3);
+
+ ServerSideEncryptingAmazonS3 s3 = new ServerSideEncryptingAmazonS3(mockAmazonS3, mockServerSideEncryption, mockTransferConfig);
+ s3.upload(originalRequest);
+
+ EasyMock.verify(mockServerSideEncryption);
+ EasyMock.verify(mockAmazonS3);
+ EasyMock.verify(mockTransferConfig);
+ }
+
+ @Test
+ public void testUpload_WithTransferManager() throws InterruptedException, NoSuchFieldException, IllegalAccessException
+ {
+ PutObjectRequest originalRequest = new PutObjectRequest("bucket", "key", "file");
+ PutObjectRequest decoratedRequest = new PutObjectRequest("bucket", "key", "file-encrypted");
+ Upload mockUpload = EasyMock.createMock(Upload.class);
+
+ EasyMock.expect(mockTransferConfig.isUseTransferManager()).andReturn(true).once();
+ EasyMock.expect(mockTransferConfig.getMinimumUploadPartSize()).andReturn(5242880L).once(); // 5 MB
+ EasyMock.expect(mockTransferConfig.getMultipartUploadThreshold()).andReturn(10485760L).once(); // 10 MB
+ EasyMock.replay(mockTransferConfig);
+
+ EasyMock.expect(mockServerSideEncryption.decorate(originalRequest)).andReturn(decoratedRequest);
+ EasyMock.replay(mockServerSideEncryption);
+
+ EasyMock.expect(mockTransferManager.upload(decoratedRequest)).andReturn(mockUpload);
+ EasyMock.replay(mockTransferManager);
+
+ mockUpload.waitForCompletion();
+ EasyMock.expectLastCall();
+ EasyMock.replay(mockUpload);
+
+ ServerSideEncryptingAmazonS3 s3 = new ServerSideEncryptingAmazonS3(mockAmazonS3, mockServerSideEncryption, mockTransferConfig);
+
+ Field transferManagerField = ServerSideEncryptingAmazonS3.class.getDeclaredField("transferManager");
+ transferManagerField.setAccessible(true);
+ transferManagerField.set(s3, mockTransferManager);
+
+ s3.upload(originalRequest);
+
+ EasyMock.verify(mockServerSideEncryption);
+ EasyMock.verify(mockTransferManager);
+ EasyMock.verify(mockUpload);
+ }
+}
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java
index 3685fc6fa19b..fefcb8c3c38b 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java
@@ -67,7 +67,7 @@ public void testWithFixedAWSKeys()
new AWSProxyConfig(),
new AWSEndpointConfig(),
new AWSClientConfig(),
- new S3StorageConfig(new NoopServerSideEncryption())
+ new S3StorageConfig(new NoopServerSideEncryption(), null)
);
s3Module.getAmazonS3Client(
@@ -102,7 +102,7 @@ public void testWithFileSessionCredentials() throws IOException
new AWSProxyConfig(),
new AWSEndpointConfig(),
new AWSClientConfig(),
- new S3StorageConfig(new NoopServerSideEncryption())
+ new S3StorageConfig(new NoopServerSideEncryption(), null)
);
s3Module.getAmazonS3Client(
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java
index 1f8eac3bbae0..f407c2a41d7e 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/RetryableS3OutputStreamTest.java
@@ -34,6 +34,7 @@
import org.apache.druid.java.util.common.IOE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.storage.s3.NoopServerSideEncryption;
+import org.apache.druid.storage.s3.S3TransferConfig;
import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
import org.easymock.EasyMock;
import org.hamcrest.CoreMatchers;
@@ -228,7 +229,7 @@ private static class TestAmazonS3 extends ServerSideEncryptingAmazonS3
private TestAmazonS3(int totalUploadFailures)
{
- super(EasyMock.createMock(AmazonS3.class), new NoopServerSideEncryption());
+ super(EasyMock.createMock(AmazonS3.class), new NoopServerSideEncryption(), new S3TransferConfig());
this.uploadFailuresLeft = totalUploadFailures;
}
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java
index 380c5cb1e508..a2f4d7e1a459 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java
@@ -33,6 +33,7 @@
import com.google.common.collect.Lists;
import org.apache.druid.storage.StorageConnector;
import org.apache.druid.storage.s3.NoopServerSideEncryption;
+import org.apache.druid.storage.s3.S3TransferConfig;
import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
import org.easymock.Capture;
import org.easymock.EasyMock;
@@ -64,7 +65,8 @@ public class S3StorageConnectorTest
private final AmazonS3Client s3Client = EasyMock.createMock(AmazonS3Client.class);
private final ServerSideEncryptingAmazonS3 service = new ServerSideEncryptingAmazonS3(
s3Client,
- new NoopServerSideEncryption()
+ new NoopServerSideEncryption(),
+ new S3TransferConfig()
);
private final ListObjectsV2Result testResult = EasyMock.createMock(ListObjectsV2Result.class);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
index c2f542b096e1..daccaf9a7519 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertAction.java
@@ -39,6 +39,7 @@
import org.joda.time.Interval;
import javax.annotation.Nullable;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
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 4f15f0f15986..651f1675f626 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
@@ -38,6 +38,7 @@ public class DropwizardRowIngestionMeters implements RowIngestionMeters
private final Meter processedWithError;
private final Meter unparseable;
private final Meter thrownAway;
+ private final Meter filtered;
public DropwizardRowIngestionMeters()
{
@@ -47,6 +48,7 @@ public DropwizardRowIngestionMeters()
this.processedWithError = metricRegistry.meter(PROCESSED_WITH_ERROR);
this.unparseable = metricRegistry.meter(UNPARSEABLE);
this.thrownAway = metricRegistry.meter(THROWN_AWAY);
+ this.filtered = metricRegistry.meter(FILTERED);
}
@Override
@@ -109,6 +111,18 @@ public void incrementThrownAway()
thrownAway.mark();
}
+ @Override
+ public long getFiltered()
+ {
+ return filtered.getCount();
+ }
+
+ @Override
+ public void incrementFiltered()
+ {
+ filtered.mark();
+ }
+
@Override
public RowIngestionMetersTotals getTotals()
{
@@ -117,7 +131,8 @@ public RowIngestionMetersTotals getTotals()
processedBytes.getCount(),
processedWithError.getCount(),
thrownAway.getCount(),
- unparseable.getCount()
+ unparseable.getCount(),
+ filtered.getCount()
);
}
@@ -132,6 +147,7 @@ public Map getMovingAverages()
oneMinute.put(PROCESSED_WITH_ERROR, processedWithError.getOneMinuteRate());
oneMinute.put(UNPARSEABLE, unparseable.getOneMinuteRate());
oneMinute.put(THROWN_AWAY, thrownAway.getOneMinuteRate());
+ oneMinute.put(FILTERED, filtered.getOneMinuteRate());
Map fiveMinute = new HashMap<>();
fiveMinute.put(PROCESSED, processed.getFiveMinuteRate());
@@ -139,6 +155,7 @@ public Map getMovingAverages()
fiveMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFiveMinuteRate());
fiveMinute.put(UNPARSEABLE, unparseable.getFiveMinuteRate());
fiveMinute.put(THROWN_AWAY, thrownAway.getFiveMinuteRate());
+ fiveMinute.put(FILTERED, filtered.getFiveMinuteRate());
Map fifteenMinute = new HashMap<>();
fifteenMinute.put(PROCESSED, processed.getFifteenMinuteRate());
@@ -146,6 +163,7 @@ public Map getMovingAverages()
fifteenMinute.put(PROCESSED_WITH_ERROR, processedWithError.getFifteenMinuteRate());
fifteenMinute.put(UNPARSEABLE, unparseable.getFifteenMinuteRate());
fifteenMinute.put(THROWN_AWAY, thrownAway.getFifteenMinuteRate());
+ fifteenMinute.put(FILTERED, filtered.getFifteenMinuteRate());
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 393035b03efd..bd6c131ea34f 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
@@ -65,7 +65,7 @@ public TaskRealtimeMetricsMonitor(
this.dimensions = ImmutableMap.copyOf(dimensions);
this.metricTags = metricTags;
previousFireDepartmentMetrics = new FireDepartmentMetrics();
- previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0);
+ previousRowIngestionMetersTotals = new RowIngestionMetersTotals(0, 0, 0, 0, 0, 0);
}
@Override
@@ -103,6 +103,9 @@ public boolean doMonitor(ServiceEmitter emitter)
emitter.emit(builder.setMetric("ingest/events/processed", rowIngestionMetersTotals.getProcessed() - previousRowIngestionMetersTotals.getProcessed()));
+ final long filtered = rowIngestionMetersTotals.getFiltered() - previousRowIngestionMetersTotals.getFiltered();
+ emitter.emit(builder.setMetric("ingest/events/filtered", filtered));
+
final long dedup = metrics.dedup() - previousFireDepartmentMetrics.dedup();
if (dedup > 0) {
log.warn("[%,d] duplicate events!", dedup);
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 83e361a4ecd0..2af7c777fcd6 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
@@ -713,7 +713,7 @@ protected boolean waitForSegmentAvailability(
try (
SegmentHandoffNotifier notifier = toolbox.getSegmentHandoffNotifierFactory()
- .createSegmentHandoffNotifier(segmentsToWaitFor.get(0).getDataSource())
+ .createSegmentHandoffNotifier(segmentsToWaitFor.get(0).getDataSource(), getId())
) {
final ExecutorService exec = Execs.directExecutor();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java
index 79a3e8993a8c..0ad143c59f56 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskUtils.java
@@ -37,6 +37,8 @@
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.utils.CircularBuffer;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
@@ -145,10 +147,28 @@ public static void emitSegmentPublishMetrics(
for (DataSegment segment : publishResult.getSegments()) {
IndexTaskUtils.setSegmentDimensions(metricBuilder, segment);
toolbox.getEmitter().emit(metricBuilder.setMetric("segment/added/bytes", segment.getSize()));
+ // Emit the segment related metadata using the configured emitters.
+ // There is a possibility that some segments' metadata event might get missed if the
+ // server crashes after commiting segment but before emitting the event.
+ emitSegmentMetadata(segment, toolbox);
toolbox.getEmitter().emit(SegmentMetadataEvent.create(segment, DateTimes.nowUtc()));
}
} else {
toolbox.getEmitter().emit(metricBuilder.setMetric("segment/txn/failure", 1));
}
}
+
+ private static void emitSegmentMetadata(DataSegment segment, TaskActionToolbox toolbox)
+ {
+ SegmentMetadataEvent event = new SegmentMetadataEvent(
+ segment.getDataSource(),
+ DateTime.now(DateTimeZone.UTC),
+ segment.getInterval().getStart(),
+ segment.getInterval().getEnd(),
+ segment.getVersion(),
+ segment.getLastCompactionState() != null
+ );
+
+ toolbox.getEmitter().emit(event);
+ }
}
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 4ca0f1ff80d7..b806d9f64f49 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
@@ -1596,7 +1596,9 @@ private RowIngestionMetersTotals getTotalsFromBuildSegmentsRowStats(Object build
((Number) buildSegmentsRowStatsMap.get("processedBytes")).longValue(),
((Number) buildSegmentsRowStatsMap.get("processedWithError")).longValue(),
((Number) buildSegmentsRowStatsMap.get("thrownAway")).longValue(),
- ((Number) buildSegmentsRowStatsMap.get("unparseable")).longValue()
+ ((Number) buildSegmentsRowStatsMap.get("unparseable")).longValue(),
+ buildSegmentsRowStatsMap.containsKey("filtered") ?
+ ((Number) buildSegmentsRowStatsMap.get("filtered")).longValue() : 0
);
} else {
// should never happen
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 94ce367fc847..5f3715e54f76 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
@@ -634,6 +634,7 @@ public void run()
stillReading = !assignment.isEmpty();
SequenceMetadata sequenceToCheckpoint = null;
+ boolean flagForLogLine = true;
for (OrderedPartitionableRecord record : records) {
final boolean shouldProcess = verifyRecordInRange(record.getPartitionId(), record.getSequenceNumber());
@@ -646,7 +647,7 @@ public void run()
);
if (shouldProcess) {
- final List rows = parser.parse(record.getData(), isEndOfShard(record.getSequenceNumber()));
+ final List rows = parser.parse(record.getData(), isEndOfShard(record.getSequenceNumber()), record.isFiltered());
boolean isPersistRequired = false;
final SequenceMetadata sequenceToUse = sequences
@@ -684,6 +685,10 @@ public void run()
.getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS)
);
if (isPushRequired && !sequenceToUse.isCheckpointed()) {
+ if (flagForLogLine) {
+ log.info("Hit the row limit updating sequenceToCheckpoint, SequenceToUse: [%s], rowInSegment: [%s], maxTotalRows: [%s]", sequenceToUse, addResult.getNumRowsInSegment(), addResult.getTotalNumRowsInAppenderator());
+ flagForLogLine = false;
+ }
sequenceToCheckpoint = sequenceToUse;
}
isPersistRequired |= addResult.isPersistRequired();
@@ -746,6 +751,7 @@ public void onFailure(Throwable t)
if (System.currentTimeMillis() > nextCheckpointTime) {
sequenceToCheckpoint = getLastSequenceMetadata();
+ log.info("Next checkpoint time, updating sequenceToCheckpoint, SequenceToCheckpoint: [%s]", sequenceToCheckpoint);
}
if (sequenceToCheckpoint != null && stillReading) {
@@ -1860,10 +1866,13 @@ public Response pauseHTTP(
@VisibleForTesting
public Response pause() throws InterruptedException
{
- if (!(status == Status.PAUSED || status == Status.READING)) {
+ // Read the volatile status into a variable so that its value does not change while the condition is evaluated
+ final Status currentStatus = status;
+ if (!(currentStatus == Status.PAUSED || currentStatus == Status.READING)) {
+ log.error("Cannot pause task[%s] as it is currently in state[%s]", task.getId(), currentStatus);
return Response.status(Response.Status.CONFLICT)
.type(MediaType.TEXT_PLAIN)
- .entity(StringUtils.format("Can't pause, task is not in a pausable state (state: [%s])", status))
+ .entity(StringUtils.format("Cannot pause task as it is currently in state[%s]. Pausable states are [READING, PAUSED].", currentStatus))
.build();
}
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..2ee94f34f7f6 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
@@ -111,10 +111,13 @@ class StreamChunkParser
this.parseExceptionHandler = parseExceptionHandler;
}
- List parse(@Nullable List streamChunk, boolean isEndOfShard) throws IOException
+ List parse(@Nullable List streamChunk, boolean isEndOfShard, boolean isFiltered) throws IOException
{
if (streamChunk == null || streamChunk.isEmpty()) {
- if (!isEndOfShard) {
+ if (isFiltered) {
+ // This is a filtered record - track as filtered, not thrownAway
+ rowIngestionMeters.incrementFiltered();
+ } else 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();
@@ -129,6 +132,11 @@ List parse(@Nullable List streamChunk, boolean isEndOfShar
}
}
+ List parse(@Nullable List streamChunk, boolean isEndOfShard) throws IOException
+ {
+ return parse(streamChunk, isEndOfShard, false);
+ }
+
private List parseWithParser(InputRowParser parser, List extends ByteEntity> valueBytes)
{
final FluentIterable iterable = FluentIterable
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java
index 5d052585ba98..71d6885650c4 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/common/OrderedPartitionableRecord.java
@@ -41,6 +41,7 @@ public class OrderedPartitionableRecord data;
+ private final boolean filtered;
public OrderedPartitionableRecord(
String stream,
@@ -48,6 +49,17 @@ public OrderedPartitionableRecord(
SequenceOffsetType sequenceNumber,
List data
)
+ {
+ this(stream, partitionId, sequenceNumber, data, false);
+ }
+
+ public OrderedPartitionableRecord(
+ String stream,
+ PartitionIdType partitionId,
+ SequenceOffsetType sequenceNumber,
+ List data,
+ boolean filtered
+ )
{
Preconditions.checkNotNull(stream, "stream");
Preconditions.checkNotNull(partitionId, "partitionId");
@@ -56,6 +68,7 @@ public OrderedPartitionableRecord(
this.partitionId = partitionId;
this.sequenceNumber = sequenceNumber;
this.data = data == null ? ImmutableList.of() : data;
+ this.filtered = filtered;
}
public String getStream()
@@ -88,6 +101,18 @@ public List extends ByteEntity> getData()
return data;
}
+ /**
+ * Returns whether this record was filtered by pre-ingestion filtering (e.g., header-based filtering).
+ * Filtered records are used for offset advancement but should be excluded from ingestion metrics
+ * like "thrownAway" and instead tracked separately as "filtered".
+ *
+ * @return true if this record was filtered, false otherwise
+ */
+ public boolean isFiltered()
+ {
+ return filtered;
+ }
+
public StreamPartition getStreamPartition()
{
return StreamPartition.of(stream, partitionId);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index 58a433325a3f..9c176faaf20c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -2152,6 +2152,7 @@ public Boolean apply(Pair(taskId, futureResult.valueOrThrow()));
}
}
@@ -2433,43 +2435,67 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup)
);
}
- private void addDiscoveredTaskToPendingCompletionTaskGroups(
+ @VisibleForTesting
+ protected void addDiscoveredTaskToPendingCompletionTaskGroups(
int groupId,
String taskId,
Map startingPartitions
)
{
- final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.computeIfAbsent(
+ final CopyOnWriteArrayList taskGroupList = pendingCompletionTaskGroups.compute(
groupId,
- k -> new CopyOnWriteArrayList<>()
+ (k, val) -> {
+ // Creating new pending completion task groups while compute so that read and writes are locked.
+ // To ensure synchronisatoin across threads, we need to do updates in compute so that we get only one task group for all replica tasks
+ if (val == null) {
+ val = new CopyOnWriteArrayList<>();
+ }
+
+ boolean isTaskGroupPresent = false;
+ for (TaskGroup taskGroup : val) {
+ if (taskGroup.startingSequences.equals(startingPartitions)) {
+ isTaskGroupPresent = true;
+ break;
+ }
+ }
+ if (!isTaskGroupPresent) {
+ log.info("Creating new pending completion task group [%s] for discovered task [%s].", groupId, taskId);
+
+ // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot
+ // change to a state where it will read any more events.
+ // This is a discovered task, so it would not have been assigned closed partitions initially.
+ TaskGroup newTaskGroup = new TaskGroup(
+ groupId,
+ ImmutableMap.copyOf(startingPartitions),
+ null,
+ Optional.absent(),
+ Optional.absent(),
+ null
+ );
+
+ newTaskGroup.tasks.put(taskId, new TaskData());
+ newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
+
+ val.add(newTaskGroup);
+ }
+ return val;
+ }
);
+
for (TaskGroup taskGroup : taskGroupList) {
if (taskGroup.startingSequences.equals(startingPartitions)) {
if (taskGroup.tasks.putIfAbsent(taskId, new TaskData()) == null) {
- log.info("Added discovered task [%s] to existing pending task group [%s]", taskId, groupId);
+ log.info("Added discovered task [%s] to existing pending completion task group [%s]. PendingCompletionTaskGroup: %s", taskId, groupId, taskGroup.taskIds());
}
return;
}
}
+ }
- log.info("Creating new pending completion task group [%s] for discovered task [%s]", groupId, taskId);
-
- // reading the minimumMessageTime & maximumMessageTime from the publishing task and setting it here is not necessary as this task cannot
- // change to a state where it will read any more events.
- // This is a discovered task, so it would not have been assigned closed partitions initially.
- TaskGroup newTaskGroup = new TaskGroup(
- groupId,
- ImmutableMap.copyOf(startingPartitions),
- null,
- Optional.absent(),
- Optional.absent(),
- null
- );
-
- newTaskGroup.tasks.put(taskId, new TaskData());
- newTaskGroup.completionTimeout = DateTimes.nowUtc().plus(ioConfig.getCompletionTimeout());
-
- taskGroupList.add(newTaskGroup);
+ @VisibleForTesting
+ protected CopyOnWriteArrayList getPendingCompletionTaskGroups(int groupId)
+ {
+ return pendingCompletionTaskGroups.get(groupId);
}
// Sanity check to ensure that tasks have the same sequence name as their task group
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index a12a353e4c33..9a913efb7ec2 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -797,7 +797,8 @@ public void testNoReportParseExceptions() throws Exception
RowIngestionMeters.PROCESSED_BYTES, 0,
RowIngestionMeters.PROCESSED_WITH_ERROR, 1,
RowIngestionMeters.UNPARSEABLE, 2,
- RowIngestionMeters.THROWN_AWAY, 0
+ RowIngestionMeters.THROWN_AWAY, 0,
+ RowIngestionMeters.FILTERED, 0
)
);
@@ -901,7 +902,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception
RowIngestionMeters.PROCESSED_BYTES, 0,
RowIngestionMeters.PROCESSED_WITH_ERROR, 2,
RowIngestionMeters.UNPARSEABLE, 2,
- RowIngestionMeters.THROWN_AWAY, 0
+ RowIngestionMeters.THROWN_AWAY, 0,
+ RowIngestionMeters.FILTERED, 0
)
);
@@ -998,7 +1000,8 @@ public void testMultipleParseExceptionsFailure() throws Exception
RowIngestionMeters.PROCESSED_BYTES, 0,
RowIngestionMeters.PROCESSED_WITH_ERROR, 2,
RowIngestionMeters.UNPARSEABLE, 2,
- RowIngestionMeters.THROWN_AWAY, 0
+ RowIngestionMeters.THROWN_AWAY, 0,
+ RowIngestionMeters.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1261,7 +1264,8 @@ public void testRestoreCorruptData() throws Exception
RowIngestionMeters.PROCESSED, 0,
RowIngestionMeters.PROCESSED_BYTES, 0,
RowIngestionMeters.UNPARSEABLE, 0,
- RowIngestionMeters.THROWN_AWAY, 0
+ RowIngestionMeters.THROWN_AWAY, 0,
+ RowIngestionMeters.FILTERED, 0
)
);
@@ -1593,7 +1597,7 @@ public SegmentPublishResult commitSegmentsAndMetadata(
);
handOffCallbacks = new ConcurrentHashMap<>();
- final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier()
+ final SegmentHandoffNotifierFactory handoffNotifierFactory = (dataSource, taskId) -> new SegmentHandoffNotifier()
{
@Override
public boolean registerSegmentHandoffCallback(
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 a0b1c101d40c..effb11aeb1ec 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
@@ -26,6 +26,7 @@
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.segment.incremental.ParseExceptionHandler;
@@ -36,6 +37,7 @@
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
+import org.mockito.ArgumentMatchers;
import org.mockito.Mockito;
import java.io.IOException;
@@ -349,9 +351,48 @@ public void close()
Assert.assertEquals(ROWS.size(), rowIngestionMeters.getUnparseable());
}
+ @Test
+ public void testNoInfiniteLoopForExceptionInHasNext()
+ {
+
+ int maxAllowedParseExceptions = 3;
+
+ ParseExceptionHandler exceptionHandler = new ParseExceptionHandler(rowIngestionMeters, false, maxAllowedParseExceptions, 1);
+ ParseExceptionHandler mockedExceptionHandler = Mockito.spy(exceptionHandler);
+ Mockito.doCallRealMethod().when(mockedExceptionHandler);
+
+ // This iterator throws ParseException always in hasNext().
+ final FilteringCloseableInputRowIterator rowIterator = new FilteringCloseableInputRowIterator(
+ new CloseableIterator() {
+ @Override
+ public void close()
+ {
+ }
+
+ @Override
+ public boolean hasNext()
+ {
+ throw new ParseException("test", "abcd", "");
+ }
+
+ @Override
+ public InputRow next()
+ {
+ return null;
+ }
+ },
+ row -> true,
+ rowIngestionMeters, mockedExceptionHandler
+ );
+
+ Assert.assertThrows(RE.class, () -> rowIterator.next());
+ Mockito.verify(mockedExceptionHandler, Mockito.times(maxAllowedParseExceptions + 1)).handle(ArgumentMatchers.any(ParseException.class));
+
+ }
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/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java
index 336b4d499bc8..2e57b6d5311a 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
@@ -1218,7 +1218,7 @@ public void testWaitForSegmentAvailabilityMultipleSegmentsTimeout()
EasyMock.expect(mockToolbox.getSegmentHandoffNotifierFactory()).andReturn(mockFactory).once();
EasyMock.expect(mockToolbox.getEmitter()).andReturn(new NoopServiceEmitter()).anyTimes();
EasyMock.expect(mockDataSegment1.getDataSource()).andReturn("MockDataSource").once();
- EasyMock.expect(mockFactory.createSegmentHandoffNotifier("MockDataSource")).andReturn(mockNotifier).once();
+ EasyMock.expect(mockFactory.createSegmentHandoffNotifier("MockDataSource", indexTask.getId())).andReturn(mockNotifier).once();
mockNotifier.start();
EasyMock.expectLastCall().once();
mockNotifier.registerSegmentHandoffCallback(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject());
@@ -1567,7 +1567,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.FILTERED, 0
),
RowIngestionMeters.BUILD_SEGMENTS,
ImmutableMap.of(
@@ -1575,7 +1576,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.FILTERED, 0
)
);
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
@@ -1756,7 +1758,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.FILTERED, 0
),
RowIngestionMeters.BUILD_SEGMENTS,
ImmutableMap.of(
@@ -1764,7 +1767,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.FILTERED, 0
)
);
@@ -1883,7 +1887,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.FILTERED, 0
),
RowIngestionMeters.BUILD_SEGMENTS,
ImmutableMap.of(
@@ -1891,7 +1896,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.FILTERED, 0
)
);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 43253a10bccc..68b34071a6b0 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -949,7 +949,7 @@ public void registerQueryFuture(Query query, ListenableFuture future)
final SegmentHandoffNotifierFactory handoffNotifierFactory = new SegmentHandoffNotifierFactory()
{
@Override
- public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource)
+ public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource, String taskId)
{
return new SegmentHandoffNotifier()
{
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
index f888dd76bf0f..54d02748e99e 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -831,7 +831,8 @@ protected TaskReport.ReportMap buildExpectedTaskReportSequential(
"processedBytes", 0.0,
"unparseable", 0.0,
"thrownAway", 0.0,
- "processedWithError", 0.0
+ "processedWithError", 0.0,
+ "filtered", 0.0
);
Map emptyAverages = ImmutableMap.of(
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..65f94925a7d4 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, 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 5ad774386b65..176132313bdd 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
@@ -508,7 +508,7 @@ public void testRunInParallelTaskReports()
1L
)
),
- new RowIngestionMetersTotals(10, 335, 1, 1, 1)
+ new RowIngestionMetersTotals(10, 335, 1, 1, 1, 0)
);
compareTaskReports(expectedReports, actualReports);
}
@@ -543,7 +543,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, 1, 1, 0);
List expectedUnparseableEvents = ImmutableList.of(
new ParseExceptionReport(
"{ts=2017unparseable}",
@@ -564,7 +564,7 @@ public void testRunInSequential()
expectedReports = buildExpectedTaskReportSequential(
task.getId(),
expectedUnparseableEvents,
- new RowIngestionMetersTotals(0, 0, 0, 0, 0),
+ new RowIngestionMetersTotals(0, 0, 0, 0, 0, 0),
expectedTotals
);
} else {
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
index ece18aa852d2..c1455e36fa46 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java
@@ -500,7 +500,7 @@ private SegmentHandoffNotifierFactory setUpSegmentHandOffNotifierFactory()
return new SegmentHandoffNotifierFactory()
{
@Override
- public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource)
+ public SegmentHandoffNotifier createSegmentHandoffNotifier(String dataSource, String taskId)
{
return new SegmentHandoffNotifier()
{
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
index 06a4bcb5b759..bea72e5315fa 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
@@ -634,7 +634,7 @@ public boolean checkPointDataSourceMetadata(
taskActionToolbox,
new TaskAuditLogConfig(false)
);
- final SegmentHandoffNotifierFactory handoffNotifierFactory = dataSource -> new SegmentHandoffNotifier()
+ final SegmentHandoffNotifierFactory handoffNotifierFactory = (dataSource, taskId) -> new SegmentHandoffNotifier()
{
@Override
public boolean registerSegmentHandoffCallback(
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
index 489315cc2495..8de42210161d 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
@@ -76,6 +76,7 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
@@ -114,8 +115,13 @@
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
@@ -281,6 +287,181 @@ public void testRunningStreamGetSequenceNumberReturnsNull()
verifyAll();
}
+ @Test
+ public void testAddDiscoveredTaskToPendingCompletionTaskGroups() throws Exception
+ {
+ EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes();
+ EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes();
+ EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
+ EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes();
+
+ replayAll();
+ ExecutorService threadExecutor = Execs.multiThreaded(3, "my-thread-pool-%d");
+
+ SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor();
+ Map startingPartitions = new HashMap<>();
+ startingPartitions.put("partition", "offset");
+
+ // Test concurrent threads adding to same task group
+ Callable task1 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_1", startingPartitions);
+ return true;
+ };
+ Callable task2 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_2", startingPartitions);
+ return true;
+ };
+ Callable task3 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_3", startingPartitions);
+ return true;
+ };
+
+ // Create a list to hold the Callable tasks
+ List> tasks = new ArrayList<>();
+ tasks.add(task1);
+ tasks.add(task2);
+ tasks.add(task3);
+ List> futures = threadExecutor.invokeAll(tasks);
+ // Wait for all tasks to complete
+ for (Future future : futures) {
+ try {
+ Boolean result = future.get();
+ Assert.assertTrue(result);
+ }
+ catch (ExecutionException e) {
+ Assert.fail();
+ }
+ }
+ CopyOnWriteArrayList taskGroups = supervisor.getPendingCompletionTaskGroups(0);
+ Assert.assertEquals(1, taskGroups.size());
+ Assert.assertEquals(3, taskGroups.get(0).tasks.size());
+
+ // Test concurrent threads adding to different task groups
+ task1 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(1, "task_1", startingPartitions);
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(1, "task_1", startingPartitions);
+ return true;
+ };
+ task2 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(2, "task_1", startingPartitions);
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(2, "task_1", startingPartitions);
+ return true;
+ };
+ task3 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(1, "task_2", startingPartitions);
+ return true;
+ };
+ Callable task4 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(2, "task_2", startingPartitions);
+ return true;
+ };
+ Callable task5 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(1, "task_3", startingPartitions);
+ return true;
+ };
+ Callable task6 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(1, "task_1", startingPartitions);
+ return true;
+ };
+
+ tasks = new ArrayList<>();
+ tasks.add(task1);
+ tasks.add(task2);
+ tasks.add(task3);
+ tasks.add(task4);
+ tasks.add(task5);
+ tasks.add(task6);
+ futures = threadExecutor.invokeAll(tasks);
+ for (Future future : futures) {
+ try {
+ Boolean result = future.get();
+ Assert.assertTrue(result);
+ }
+ catch (ExecutionException e) {
+ Assert.fail();
+ }
+ }
+
+ taskGroups = supervisor.getPendingCompletionTaskGroups(1);
+ Assert.assertEquals(1, taskGroups.size());
+ Assert.assertEquals(3, taskGroups.get(0).tasks.size());
+
+ taskGroups = supervisor.getPendingCompletionTaskGroups(2);
+ Assert.assertEquals(1, taskGroups.size());
+ Assert.assertEquals(2, taskGroups.get(0).tasks.size());
+ }
+
+ @Test
+ public void testAddDiscoveredTaskToPendingCompletionMultipleTaskGroups() throws Exception
+ {
+ EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes();
+ EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes();
+ EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
+ EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes();
+
+ replayAll();
+
+ // Test adding tasks with same task group and different partition offsets.
+ SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor();
+ ExecutorService threadExecutor = Execs.multiThreaded(3, "my-thread-pool-%d");
+ Map startingPartiions = new HashMap<>();
+ startingPartiions.put("partition", "offset");
+
+ Map startingPartiions1 = new HashMap<>();
+ startingPartiions.put("partition", "offset1");
+
+ Callable task1 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_1", startingPartiions);
+ return true;
+ };
+ Callable task2 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_2", startingPartiions);
+ return true;
+ };
+ Callable task3 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_3", startingPartiions);
+ return true;
+ };
+ Callable task4 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_7", startingPartiions1);
+ return true;
+ };
+ Callable task5 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_8", startingPartiions1);
+ return true;
+ };
+ Callable task6 = () -> {
+ supervisor.addDiscoveredTaskToPendingCompletionTaskGroups(0, "task_9", startingPartiions1);
+ return true;
+ };
+
+ List> tasks = new ArrayList<>();
+ tasks.add(task1);
+ tasks.add(task2);
+ tasks.add(task3);
+ tasks.add(task4);
+ tasks.add(task5);
+ tasks.add(task6);
+
+ List> futures = threadExecutor.invokeAll(tasks);
+
+ for (Future future : futures) {
+ try {
+ Boolean result = future.get();
+ Assert.assertTrue(result);
+ }
+ catch (ExecutionException e) {
+ Assert.fail();
+ }
+ }
+
+ CopyOnWriteArrayList taskGroups = supervisor.getPendingCompletionTaskGroups(0);
+
+ Assert.assertEquals(2, taskGroups.size());
+ Assert.assertEquals(3, taskGroups.get(0).tasks.size());
+ Assert.assertEquals(3, taskGroups.get(1).tasks.size());
+ }
+
@Test
public void testConnectingToStreamFail()
{
diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml
index 2d329c5d21eb..0d96180801a2 100644
--- a/integration-tests-ex/cases/pom.xml
+++ b/integration-tests-ex/cases/pom.xml
@@ -273,6 +273,17 @@
kafka-clients
${apache.kafka.version}
test
+
+
+ org.lz4
+ lz4-java
+
+
+
+
+ at.yawk.lz4
+ lz4-java
+ test
com.google.code.findbugs
@@ -339,6 +350,8 @@
com.mysql:mysql-connector-j:jar
+
+ at.yawk.lz4:lz4-java
diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml
index d576f8ddf3c2..fbca9ee90f5b 100644
--- a/integration-tests/pom.xml
+++ b/integration-tests/pom.xml
@@ -43,19 +43,6 @@
org.apache.hadoop.fs.s3a.S3AFileSystem
-
-
-
-
- org.yaml
- snakeyaml
- 1.33
-
-
-
-
com.amazonaws
@@ -290,6 +277,16 @@
org.apache.kafka
kafka-clients
${apache.kafka.version}
+
+
+ org.lz4
+ lz4-java
+
+
+
+
+ at.yawk.lz4
+ lz4-java
javax.ws.rs
@@ -562,6 +559,16 @@
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+
+ at.yawk.lz4:lz4-java:jar:1.10.1
+
+
+
diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
index e8983b474ccb..2016489a4706 100644
--- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
+++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITTLSTest.java
@@ -29,6 +29,8 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.emitter.core.NoopEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.CredentialedHttpClient;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.HttpClientConfig;
@@ -392,7 +394,8 @@ private HttpClient makeCustomHttpClient(
HttpClient client = HttpClientInit.createClient(
builder.build(),
- lifecycle
+ lifecycle,
+ new ServiceEmitter("", "", new NoopEmitter())
);
HttpClient adminClient = new CredentialedHttpClient(
@@ -419,7 +422,8 @@ private HttpClient makeCertlessClient()
HttpClient client = HttpClientInit.createClient(
builder.build(),
- lifecycle
+ lifecycle,
+ new ServiceEmitter("", "", new NoopEmitter())
);
HttpClient adminClient = new CredentialedHttpClient(
diff --git a/licenses.yaml b/licenses.yaml
index d7fbdbeeda13..1acf04607dbc 100644
--- a/licenses.yaml
+++ b/licenses.yaml
@@ -223,10 +223,34 @@ name: Jackson
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 2.12.7
+version: 2.18.4.1
libraries:
- - com.fasterxml.jackson.core: jackson-annotations
- com.fasterxml.jackson.core: jackson-core
+notice: |
+ # Jackson JSON processor
+ Jackson is a high-performance, Free/Open Source JSON processing library.
+ It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+ been in development since 2007.
+ It is currently developed by a community of developers, as well as supported
+ commercially by FasterXML.com.
+ ## Licensing
+ Jackson core and extension components may licensed under different licenses.
+ To find the details that apply to this artifact see the accompanying LICENSE file.
+ For more information, including possible other licensing options, contact
+ FasterXML.com (http://fasterxml.com).
+ ## Credits
+ A list of contributors may be found from CREDITS file, which is included
+ in some artifacts (usually source distributions); but is always available
+ from the source code management (SCM) system project uses.
+---
+
+name: Jackson
+license_category: binary
+module: java-core
+license_name: Apache License version 2.0
+version: 2.18.4
+libraries:
+ - com.fasterxml.jackson.core: jackson-annotations
- com.fasterxml.jackson.dataformat: jackson-dataformat-cbor
- com.fasterxml.jackson.dataformat: jackson-dataformat-smile
- com.fasterxml.jackson.dataformat: jackson-dataformat-xml
@@ -264,9 +288,9 @@ notice: |
name: Jackson
license_category: binary
-module: java-core
+module: extensions-contrib/druid-deltalake-extensions
license_name: Apache License version 2.0
-version: 2.12.7.1
+version: 2.18.4
libraries:
- com.fasterxml.jackson.core: jackson-databind
notice: |
@@ -493,7 +517,7 @@ name: Apache Commons BeanUtils
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 1.9.4
+version: 1.11.0
libraries:
- commons-beanutils: commons-beanutils
notices:
@@ -546,7 +570,7 @@ name: Apache Commons IO
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 2.11.0
+version: 2.14.0
libraries:
- commons-io: commons-io
notices:
@@ -676,7 +700,7 @@ name: Apache Commons Lang
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 3.12.0
+version: 3.18.0
libraries:
- org.apache.commons: commons-lang3
notices:
@@ -778,7 +802,7 @@ name: pac4j-oidc java security library
license_category: binary
module: extensions/druid-pac4j
license_name: Apache License version 2.0
-version: 4.5.7
+version: 5.7.3
libraries:
- org.pac4j: pac4j-oidc
@@ -788,7 +812,7 @@ name: pac4j-core java security library
license_category: binary
module: extensions/druid-pac4j
license_name: Apache License version 2.0
-version: 4.5.7
+version: 5.7.3
libraries:
- org.pac4j: pac4j-core
@@ -809,17 +833,27 @@ name: com.nimbusds nimbus-jose-jwt
license_category: binary
module: extensions/druid-pac4j
license_name: Apache License version 2.0
-version: 9.37.2
+version: 10.0.2
libraries:
- com.nimbusds: nimbus-jose-jwt
---
+name: pac4j-javaee java security library
+license_category: binary
+module: extensions/druid-pac4j
+license_name: Apache License version 2.0
+version: 5.7.3
+libraries:
+ - org.pac4j: pac4j-javaee
+
+---
+
name: com.nimbusds content-type
license_category: binary
module: extensions/druid-pac4j
license_name: Apache License version 2.0
-version: 2.1
+version: 2.2
libraries:
- com.nimbusds: content-type
@@ -829,7 +863,7 @@ name: com.nimbusds oauth2-oidc-sdk
license_category: binary
module: extensions/druid-pac4j
license_name: Apache License version 2.0
-version: 8.22
+version: 10.8
libraries:
- com.nimbusds: oauth2-oidc-sdk
@@ -859,7 +893,7 @@ name: kubernetes fabric java client
license_category: binary
module: extensions-contrib/kubernetes-overlord-extensions
license_name: Apache License version 2.0
-version: 6.7.2
+version: 6.8.0
libraries:
- io.fabric8: kubernetes-client
@@ -914,7 +948,7 @@ name: com.squareup.okio okio
license_category: binary
module: extensions/druid-kubernetes-extensions
license_name: Apache License version 2.0
-version: 1.17.2
+version: 1.17.6
libraries:
- com.squareup.okio: okio
@@ -1249,7 +1283,7 @@ name: Netty
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 4.1.108.Final
+version: 4.2.5.Final
libraries:
- io.netty: netty-buffer
- io.netty: netty-codec
@@ -1937,7 +1971,7 @@ name: AsyncHttpClient asynchttpclient
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 2.5.3
+version: 2.12.4
libraries:
- org.asynchttpclient: async-http-client
- org.asynchttpclient: async-http-client-netty-utils
@@ -1999,7 +2033,7 @@ name: Jetty
license_category: binary
module: java-core
license_name: Apache License version 2.0
-version: 9.4.54.v20240208
+version: 9.4.57.v20241219
libraries:
- org.eclipse.jetty: jetty-client
- org.eclipse.jetty: jetty-continuation
@@ -2333,7 +2367,7 @@ module: java-core
license_name: Apache License version 2.0
version: 1.8.0
libraries:
- - org.lz4: lz4-java
+ - at.yawk.lz4: lz4-java
---
@@ -2418,7 +2452,7 @@ name: Apache Hadoop
license_category: binary
module: hadoop-client
license_name: Apache License version 2.0
-version: 3.3.6
+version: 3.4.1
libraries:
- org.apache.hadoop: hadoop-auth
- org.apache.hadoop: hadoop-common
@@ -2783,7 +2817,7 @@ libraries:
---
name: Jackson Dataformat Yaml
-version: 2.12.7
+version: 2.18.4
license_category: binary
module: extensions/druid-avro-extensions
license_name: Apache License version 2.0
@@ -2967,7 +3001,7 @@ name: Apache Avro
license_category: binary
module: extensions/druid-avro-extensions
license_name: Apache License version 2.0
-version: 1.11.3
+version: 1.11.4
libraries:
- org.apache.avro: avro
- org.apache.avro: avro-mapred
@@ -3120,14 +3154,14 @@ name: Hadoop Client API
license_category: binary
module: extensions/druid-hdfs-storage
license_name: Apache License version 2.0
-version: 3.3.6
+version: 3.4.1
libraries:
- org.apache.hadoop: hadoop-client-api
---
name: Apache Kafka
-version: 3.6.1
+version: 3.9.1
license_category: binary
module: extensions/druid-kafka-indexing-service
license_name: Apache License version 2.0
@@ -3341,7 +3375,7 @@ name: Protocol Buffers
license_category: binary
module: java-core
license_name: BSD-3-Clause License
-version: 3.24.0
+version: 3.25.5
copyright: Google, Inc.
license_file_path:
- licenses/bin/protobuf-java.BSD3
@@ -3507,7 +3541,7 @@ name: Protocol Buffers
license_category: binary
module: extensions/druid-protobuf-extensions
license_name: BSD-3-Clause License
-version: 3.24.0
+version: 3.25.5
copyright: Google, Inc.
license_file_path: licenses/bin/protobuf-java.BSD3
libraries:
@@ -4154,7 +4188,7 @@ name: Apache Kafka
license_category: binary
module: extensions/kafka-extraction-namespace
license_name: Apache License version 2.0
-version: 3.6.1
+version: 3.9.1
libraries:
- org.apache.kafka: kafka-clients
notices:
diff --git a/owasp-dependency-check-suppressions.xml b/owasp-dependency-check-suppressions.xml
index f861c8c065ac..a869f4b19236 100644
--- a/owasp-dependency-check-suppressions.xml
+++ b/owasp-dependency-check-suppressions.xml
@@ -346,7 +346,7 @@
CVE-2021-44878
@@ -453,15 +453,6 @@
CVE-2021-4277
-
-
-
- ^pkg:maven/com\.squareup\.okio/okio@..*$
- CVE-2023-3635
-
-
^pkg:maven/org\.codehaus\.plexus/plexus-interpolation@.*$
diff --git a/pom.xml b/pom.xml
index 17cd202ea67f..a98f15899405 100644
--- a/pom.xml
+++ b/pom.xml
@@ -76,14 +76,14 @@
UTF-8
0.9.0.M2
5.5.0
- 3.6.1
+ 3.9.1
2.4.0
2.10.1
2.13.11
1.23.0
- 1.11.3
+ 1.11.4
@@ -136,6 +137,7 @@
maven.org
Maven Central Repository
https://repo1.maven.org/maven2/
+ 0.19.0-alpha
3
@@ -235,6 +237,9 @@
extensions-contrib/spectator-histogram
extensions-contrib/rabbit-stream-indexing-service
+ extensions-contrib/opencensus-extensions
+ extensions-contrib/confluent-extensions
+ extensions-contrib/opentelemetry-extensions
distribution
@@ -292,7 +297,7 @@
commons-io
commons-io
- 2.11.0
+ 2.14.0
commons-logging
@@ -317,7 +322,7 @@
org.apache.commons
commons-lang3
- 3.12.0
+ 3.18.0
org.apache.commons
@@ -602,7 +607,7 @@
com.fasterxml.jackson
jackson-bom
- ${jackson.version}
+ ${jackson.core.version}
import
pom
@@ -638,9 +643,9 @@
provided
- javax.xml.bind
- jaxb-api
- 2.3.1
+ jakarta.xml.bind
+ jakarta.xml.bind-api
+ 2.3.3
org.glassfish
@@ -655,7 +660,7 @@
org.glassfish.jaxb
jaxb-runtime
- 2.3.1
+ 2.3.3
org.jdbi
@@ -849,9 +854,9 @@
2.0.1
- org.lz4
+ at.yawk.lz4
lz4-java
- 1.8.0
+ 1.10.1
org.xerial.snappy
@@ -863,6 +868,11 @@
protobuf-java
${protobuf.version}
+
+ com.google.protobuf
+ protobuf-java-util
+ ${protobuf.version}
+
io.tesla.aether
tesla-aether
@@ -937,8 +947,7 @@
commons-beanutils
commons-beanutils
-
- 1.9.4
+ 1.11.0
com.jayway.jsonpath
@@ -974,7 +983,7 @@
org.asynchttpclient
async-http-client
- 2.5.3
+ 2.12.4
net.java.dev.jna
@@ -1355,6 +1364,33 @@
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
+ ${jackson.version}
+
+
+ javax.xml.bind
+ jaxb-api
+
+
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-smile-provider
+ ${jackson.version}
+
+
+ javax.xml.bind
+ jaxb-api
+
+
+
+
+ io.opentelemetry.proto
+ opentelemetry-proto
+ ${opentelemetry.proto.version}
+
@@ -1521,7 +1557,7 @@
de.thetaphi
forbiddenapis
- 3.2
+ 3.5.1
true
@@ -1540,6 +1576,10 @@
**/DruidSqlParserImpl.class
**/DruidSqlParserImplTokenManager.class
**/SimpleCharStream.class
+
+ **/*_jmhType_*.class
+ **/*_jmhTest_*.class
+ **/*_generated*.class
**.SuppressForbidden
@@ -1623,6 +1663,8 @@
com.google.code.findbugs:annotations
+
+ javax.xml.bind:jaxb-api
diff --git a/processing/pom.xml b/processing/pom.xml
index fcc16a65c500..e7310dcee66c 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -156,8 +156,8 @@
runtime
- javax.xml.bind
- jaxb-api
+ jakarta.xml.bind
+ jakarta.xml.bind-api
runtime
@@ -255,7 +255,7 @@
compress-lzf
- org.lz4
+ at.yawk.lz4
lz4-java
diff --git a/processing/src/main/java/org/apache/druid/collections/BlockingPool.java b/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
index 4fb3ff66d8bf..2ea9e5a5244c 100644
--- a/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/BlockingPool.java
@@ -49,4 +49,9 @@ public interface BlockingPool
* @return count of pending requests
*/
long getPendingRequests();
+
+ /**
+ * @return number of buffers used/polled from the pool at that time.
+ */
+ int getUsedBufferCount();
}
diff --git a/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java b/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
index e41a9e5d75d4..ef81ecb2295d 100644
--- a/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java
@@ -212,4 +212,10 @@ private void offer(T theObject)
lock.unlock();
}
}
+
+ @Override
+ public int getUsedBufferCount()
+ {
+ return maxSize - objects.size();
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java b/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
index 2553f9ab425f..2d70d8cadd18 100644
--- a/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
+++ b/processing/src/main/java/org/apache/druid/collections/DummyBlockingPool.java
@@ -61,4 +61,10 @@ public long getPendingRequests()
{
return 0;
}
+
+ @Override
+ public int getUsedBufferCount()
+ {
+ return 0;
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/data/input/KafkaEntity.java b/processing/src/main/java/org/apache/druid/data/input/KafkaEntity.java
new file mode 100644
index 000000000000..dba30486b097
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/data/input/KafkaEntity.java
@@ -0,0 +1,33 @@
+/*
+ * 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.data.input;
+
+/**
+ * Interface for record entities that contain timestamp information.
+ */
+public interface KafkaEntity
+{
+ /**
+ * Returns the timestamp of the record in milliseconds.
+ *
+ * @return the timestamp in milliseconds
+ */
+ long getRecordTimestampMillis();
+}
diff --git a/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java b/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java
index 573af0a0828c..ae89d90bb044 100644
--- a/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java
+++ b/processing/src/main/java/org/apache/druid/jackson/JodaStuff.java
@@ -117,7 +117,8 @@ public DateTime deserialize(JsonParser jp, DeserializationContext ctxt) throws I
// make sure to preserve time zone information when parsing timestamps
return DateTimes.ISO_DATE_OR_TIME_WITH_OFFSET.parse(str);
}
- throw ctxt.mappingException(getValueClass());
+ ctxt.reportWrongTokenException(handledType(), JsonToken.VALUE_NUMBER_INT, "expected int or string token");
+ return null; // unreachable ... required for compiler, but ctxt.reportWrongTokenException always throws
}
}
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java
index c0ccb967dbdf..8b2fba4ef4d6 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java
@@ -23,6 +23,7 @@
import com.google.common.base.Strings;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.druid.java.util.emitter.EmittingLogger;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
@@ -51,6 +52,8 @@ public static ExecutorService dummy()
return DummyExecutorService.INSTANCE;
}
+ private static final EmittingLogger log = new EmittingLogger(Execs.class);
+
public static ExecutorService singleThreaded(@NotNull String nameFormat)
{
return singleThreaded(nameFormat, null);
@@ -159,6 +162,10 @@ public static ExecutorService newBlockingThreaded(
@Override
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
{
+ if (executor.isShutdown()) {
+ log.debug("Executor is shutdown, rejecting task");
+ return;
+ }
try {
executor.getQueue().put(r);
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
index 303f835d889e..da476f2d04b6 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java
@@ -27,6 +27,9 @@ public class Granularities
{
public static final Granularity SECOND = GranularityType.SECOND.getDefaultGranularity();
public static final Granularity MINUTE = GranularityType.MINUTE.getDefaultGranularity();
+ public static final Granularity TWO_MINUTE = GranularityType.TWO_MINUTE.getDefaultGranularity();
+ public static final Granularity THREE_MINUTE = GranularityType.THREE_MINUTE.getDefaultGranularity();
+ public static final Granularity FOUR_MINUTE = GranularityType.FOUR_MINUTE.getDefaultGranularity();
public static final Granularity FIVE_MINUTE = GranularityType.FIVE_MINUTE.getDefaultGranularity();
public static final Granularity TEN_MINUTE = GranularityType.TEN_MINUTE.getDefaultGranularity();
public static final Granularity FIFTEEN_MINUTE = GranularityType.FIFTEEN_MINUTE.getDefaultGranularity();
diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/GranularityType.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/GranularityType.java
index b4b78390605d..ccdb0590f233 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/GranularityType.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/GranularityType.java
@@ -48,6 +48,9 @@ public enum GranularityType
5,
"PT1M"
),
+ TWO_MINUTE(MINUTE, "PT2M"),
+ THREE_MINUTE(MINUTE, "PT3M"),
+ FOUR_MINUTE(MINUTE, "PT4M"),
FIVE_MINUTE(MINUTE, "PT5M"),
TEN_MINUTE(MINUTE, "PT10M"),
FIFTEEN_MINUTE(MINUTE, "PT15M"),
@@ -235,6 +238,12 @@ public static GranularityType fromPeriod(Period period)
return GranularityType.TEN_MINUTE;
} else if (vals[index] == 5) {
return GranularityType.FIVE_MINUTE;
+ } else if (vals[index] == 4) {
+ return GranularityType.FOUR_MINUTE;
+ } else if (vals[index] == 3) {
+ return GranularityType.THREE_MINUTE;
+ } else if (vals[index] == 2) {
+ return GranularityType.TWO_MINUTE;
} else if (vals[index] == 1) {
return GranularityType.MINUTE;
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java
index 7e249f72d0a6..732db1d3cecb 100644
--- a/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java
+++ b/processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java
@@ -97,6 +97,37 @@ public String getFeed()
{
return "segment_metadata";
}
+
+ public DateTime getCreatedTime()
+ {
+ return createdTime;
+ }
+
+ public DateTime getStartTime()
+ {
+ return startTime;
+ }
+
+ public DateTime getEndTime()
+ {
+ return endTime;
+ }
+
+ public String getDataSource()
+ {
+ return dataSource;
+ }
+
+ public String getVersion()
+ {
+ return version;
+ }
+
+ public boolean isCompacted()
+ {
+ return isCompacted;
+ }
+
@Override
@JsonValue
public EventMap toMap()
diff --git a/processing/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java b/processing/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
index 602a8a6e2612..9a4680a5fa3b 100644
--- a/processing/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
+++ b/processing/src/main/java/org/apache/druid/java/util/http/client/HttpClientInit.java
@@ -22,9 +22,11 @@
import com.google.common.base.Throwables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.netty.HttpClientPipelineFactory;
import org.apache.druid.java.util.http.client.pool.ChannelResourceFactory;
-import org.apache.druid.java.util.http.client.pool.ResourcePool;
+import org.apache.druid.java.util.http.client.pool.DefaultResourcePoolImpl;
+import org.apache.druid.java.util.http.client.pool.MetricsEmittingResourcePoolImpl;
import org.apache.druid.java.util.http.client.pool.ResourcePoolConfig;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.channel.socket.nio.NioClientBossPool;
@@ -48,7 +50,7 @@
*/
public class HttpClientInit
{
- public static HttpClient createClient(HttpClientConfig config, Lifecycle lifecycle)
+ public static HttpClient createClient(HttpClientConfig config, Lifecycle lifecycle, ServiceEmitter emitter)
{
try {
// We need to use the full constructor in order to set a ThreadNameDeterminer. The other parameters are taken
@@ -80,19 +82,22 @@ public void stop()
);
return lifecycle.addMaybeStartManagedInstance(
new NettyHttpClient(
- new ResourcePool<>(
- new ChannelResourceFactory(
+ new MetricsEmittingResourcePoolImpl<>(
+ new DefaultResourcePoolImpl<>(
+ new ChannelResourceFactory(
createBootstrap(lifecycle, timer, config.getBossPoolSize(), config.getWorkerPoolSize()),
config.getSslContext(),
config.getProxyConfig(),
timer,
config.getSslHandshakeTimeout() == null ? -1 : config.getSslHandshakeTimeout().getMillis()
- ),
- new ResourcePoolConfig(
+ ),
+ new ResourcePoolConfig(
config.getNumConnections(),
config.getUnusedConnectionTimeoutDuration().getMillis()
+ ),
+ config.isEagerInitialization()
),
- config.isEagerInitialization()
+ emitter
),
config.getReadTimeout(),
config.getCompressionCodec(),
diff --git a/processing/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java b/processing/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
index 3ab3719180fd..b5f05bad02e8 100644
--- a/processing/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
+++ b/processing/src/main/java/org/apache/druid/java/util/http/client/NettyHttpClient.java
@@ -55,6 +55,7 @@
import org.jboss.netty.util.Timer;
import org.joda.time.Duration;
+import java.io.IOException;
import java.net.URL;
import java.util.Collection;
import java.util.Map;
@@ -100,7 +101,7 @@ public void start()
}
@LifecycleStop
- public void stop()
+ public void stop() throws IOException
{
pool.close();
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/http/client/pool/DefaultResourcePoolImpl.java b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/DefaultResourcePoolImpl.java
new file mode 100644
index 000000000000..7d0fcd800f89
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/DefaultResourcePoolImpl.java
@@ -0,0 +1,381 @@
+/*
+ * 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.java.util.http.client.pool;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import javax.annotation.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * A resource pool based on {@link LoadingCache}. When a resource is first requested for a new key,
+ * If the flag: eagerInitialization is true: use {@link EagerCreationResourceHolder}
+ * {@link ResourcePoolConfig#getMaxPerKey()} resources are initialized and cached in the {@link #pool}.
+ * Else:
+ * Initialize a single resource and further lazily using {@link LazyCreationResourceHolder}
+ * The individual resource in {@link ResourceHolderPerKey} is valid while (current time - last access time)
+ * <= {@link ResourcePoolConfig#getUnusedConnectionTimeoutMillis()}.
+ *
+ * A resource is closed and reinitialized if {@link ResourceFactory#isGood} returns false or it's expired based on
+ * {@link ResourcePoolConfig#getUnusedConnectionTimeoutMillis()}.
+ *
+ * {@link ResourcePoolConfig#getMaxPerKey() is a hard limit for the max number of resources per cache entry. The total
+ * number of resources in {@link ResourceHolderPerKey} cannot be larger than the limit in any case.
+ */
+public class DefaultResourcePoolImpl implements ResourcePool
+{
+ private static final Logger log = new Logger(DefaultResourcePoolImpl.class);
+ private final LoadingCache> pool;
+ private final AtomicBoolean closed = new AtomicBoolean(false);
+
+ public DefaultResourcePoolImpl(final ResourceFactory factory, final ResourcePoolConfig config,
+ final boolean eagerInitialization)
+ {
+ this.pool = CacheBuilder.newBuilder().build(
+ new CacheLoader>()
+ {
+ @Override
+ public ResourceHolderPerKey load(K input)
+ {
+ if (eagerInitialization) {
+ return new EagerCreationResourceHolder<>(
+ config.getMaxPerKey(),
+ config.getUnusedConnectionTimeoutMillis(),
+ input,
+ factory
+ );
+ } else {
+ return new LazyCreationResourceHolder<>(
+ config.getMaxPerKey(),
+ config.getUnusedConnectionTimeoutMillis(),
+ input,
+ factory
+ );
+ }
+ }
+ }
+ );
+ }
+
+ /**
+ * Returns a {@link ResourceContainer} for the given key or null if this pool is already closed.
+ */
+ @Override
+ public ResourceContainer take(final K key)
+ {
+ if (closed.get()) {
+ log.error(StringUtils.format("take(%s) called even though I'm closed.", key));
+ return null;
+ }
+
+ final ResourceHolderPerKey holder;
+ try {
+ holder = pool.get(key);
+ }
+ catch (ExecutionException e) {
+ throw new RuntimeException(e);
+ }
+ final V value = holder.get();
+
+ return new ResourceContainer()
+ {
+ private final AtomicBoolean returned = new AtomicBoolean(false);
+
+ @Override
+ public V get()
+ {
+ Preconditions.checkState(!returned.get(), "Resource for key[%s] has been returned, cannot get().", key);
+ return value;
+ }
+
+ @Override
+ public void returnResource()
+ {
+ if (returned.getAndSet(true)) {
+ log.warn("Resource at key[%s] was returned multiple times?", key);
+ } else {
+ holder.giveBack(value);
+ }
+ }
+
+ @Override
+ protected void finalize() throws Throwable
+ {
+ if (!returned.get()) {
+ log.warn(
+ StringUtils.format(
+ "Resource[%s] at key[%s] was not returned before Container was finalized, potential resource leak.",
+ value,
+ key
+ )
+ );
+ returnResource();
+ }
+ super.finalize();
+ }
+ };
+ }
+
+ @Override
+ public void close()
+ {
+ closed.set(true);
+ final ConcurrentMap> mapView = pool.asMap();
+ Closer closer = Closer.create();
+ for (Iterator>> iterator =
+ mapView.entrySet().iterator(); iterator.hasNext(); ) {
+ Map.Entry> e = iterator.next();
+ iterator.remove();
+ closer.register(e.getValue());
+ }
+ try {
+ closer.close();
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static class EagerCreationResourceHolder extends LazyCreationResourceHolder
+ {
+ private EagerCreationResourceHolder(
+ int maxSize,
+ long unusedResourceTimeoutMillis,
+ K key,
+ ResourceFactory factory
+ )
+ {
+ super(maxSize, unusedResourceTimeoutMillis, key, factory);
+ // Eagerly Instantiate
+ for (int i = 0; i < maxSize; i++) {
+ resourceHolderList.add(
+ new ResourceHolder<>(
+ System.currentTimeMillis(),
+ Preconditions.checkNotNull(
+ factory.generate(key),
+ "factory.generate(key)"
+ )
+ )
+ );
+ }
+ }
+ }
+
+ private static class LazyCreationResourceHolder extends ResourceHolderPerKey
+ {
+ private LazyCreationResourceHolder(
+ int maxSize,
+ long unusedResourceTimeoutMillis,
+ K key,
+ ResourceFactory factory
+ )
+ {
+ super(maxSize, unusedResourceTimeoutMillis, key, factory);
+ }
+ }
+
+ private static class ResourceHolderPerKey implements Closeable
+ {
+ protected final int maxSize;
+ private final K key;
+ private final ResourceFactory factory;
+ private final long unusedResourceTimeoutMillis;
+ // Hold previously created / returned resources
+ protected final ArrayDeque> resourceHolderList;
+ // To keep track of resources that have been successfully returned to caller.
+ private int numLentResources = 0;
+ private boolean closed = false;
+
+ protected ResourceHolderPerKey(
+ int maxSize,
+ long unusedResourceTimeoutMillis,
+ K key,
+ ResourceFactory factory
+ )
+ {
+ this.maxSize = maxSize;
+ this.key = key;
+ this.factory = factory;
+ this.unusedResourceTimeoutMillis = unusedResourceTimeoutMillis;
+ this.resourceHolderList = new ArrayDeque<>();
+ }
+
+ /**
+ * Returns a resource or null if this holder is already closed or the current thread is interrupted.
+ *
+ * Try to return a previously created resource if it isGood(). Else, generate a new resource
+ */
+ @Nullable
+ V get()
+ {
+ final V poolVal;
+ // resourceHolderList can't have nulls, so we'll use a null to signal that we need to create a new resource.
+ boolean expired = false;
+ synchronized (this) {
+ while (!closed && (numLentResources == maxSize)) {
+ try {
+ log.debug("Thread [%s] is blocked waiting for resource for key [%s]", Thread.currentThread().getName(), key);
+ this.wait();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return null;
+ }
+ }
+
+ if (closed) {
+ log.info(StringUtils.format("get() called even though I'm closed. key[%s]", key));
+ return null;
+ } else if (numLentResources < maxSize) {
+ // Attempt to take an existing resource or create one if list is empty, and increment numLentResources
+ if (resourceHolderList.isEmpty()) {
+ poolVal = factory.generate(key);
+ } else {
+ ResourceHolder holder = resourceHolderList.removeFirst();
+ poolVal = holder.getResource();
+ if (System.currentTimeMillis() - holder.getLastAccessedTime() > unusedResourceTimeoutMillis) {
+ expired = true;
+ }
+ }
+ numLentResources++;
+ } else {
+ throw new IllegalStateException("Unexpected state: More objects lent than permissible");
+ }
+ }
+
+ final V retVal;
+ // At this point, we must either return a valid resource. Or throw and exception decrement "numLentResources"
+ try {
+ if (poolVal != null && !expired && factory.isGood(poolVal)) {
+ retVal = poolVal;
+ } else {
+ if (poolVal != null) {
+ factory.close(poolVal);
+ }
+ retVal = factory.generate(key);
+ }
+ }
+ catch (Throwable e) {
+ synchronized (this) {
+ numLentResources--;
+ this.notifyAll();
+ }
+ Throwables.propagateIfPossible(e);
+ throw new RuntimeException(e);
+ }
+
+ return retVal;
+ }
+
+ void giveBack(V object)
+ {
+ Preconditions.checkNotNull(object, "object");
+
+ synchronized (this) {
+ if (closed) {
+ log.info(StringUtils.format("giveBack called after being closed. key[%s]", key));
+ factory.close(object);
+ return;
+ }
+
+ if (resourceHolderList.size() >= maxSize) {
+ if (holderListContains(object)) {
+ log.warn(
+ new Exception("Exception for stacktrace"),
+ StringUtils.format(
+ "Returning object[%s] at key[%s] that has already been returned!? Skipping",
+ object,
+ key
+ )
+ );
+ } else {
+ log.warn(
+ new Exception("Exception for stacktrace"),
+ StringUtils.format(
+ "Returning object[%s] at key[%s] even though we already have all that we can hold[%s]!? Skipping",
+ object,
+ key,
+ resourceHolderList
+ )
+ );
+ }
+ return;
+ }
+
+ resourceHolderList.addLast(new ResourceHolder<>(System.currentTimeMillis(), object));
+ numLentResources--;
+ this.notifyAll();
+ }
+ }
+
+ private boolean holderListContains(V object)
+ {
+ return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object));
+ }
+
+ @Override
+ public void close()
+ {
+ synchronized (this) {
+ closed = true;
+ resourceHolderList.forEach(v -> factory.close(v.getResource()));
+ resourceHolderList.clear();
+ this.notifyAll();
+ }
+ }
+ }
+
+ private static class ResourceHolder
+ {
+ private final long lastAccessedTime;
+ private final V resource;
+
+ private ResourceHolder(long lastAccessedTime, V resource)
+ {
+ this.resource = resource;
+ this.lastAccessedTime = lastAccessedTime;
+ }
+
+ private long getLastAccessedTime()
+ {
+ return lastAccessedTime;
+ }
+
+ public V getResource()
+ {
+ return resource;
+ }
+
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/java/util/http/client/pool/MetricsEmittingResourcePoolImpl.java b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/MetricsEmittingResourcePoolImpl.java
new file mode 100644
index 000000000000..a02a9ed4a909
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/MetricsEmittingResourcePoolImpl.java
@@ -0,0 +1,55 @@
+/*
+ * 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.java.util.http.client.pool;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+
+import java.io.IOException;
+
+public class MetricsEmittingResourcePoolImpl implements ResourcePool
+{
+ private final ServiceEmitter emitter;
+ private final ResourcePool resourcePool;
+
+ public MetricsEmittingResourcePoolImpl(ResourcePool resourcePool, ServiceEmitter emitter)
+ {
+ this.resourcePool = resourcePool;
+ Preconditions.checkNotNull(emitter, "emitter cannot be null");
+ this.emitter = emitter;
+ }
+
+ @Override
+ public ResourceContainer take(final K key)
+ {
+ long startTime = System.nanoTime();
+ ResourceContainer retVal = resourcePool.take(key);
+ long totalduration = System.nanoTime() - startTime;
+ emitter.emit(ServiceMetricEvent.builder().setDimension("server", key.toString()).setMetric("httpClient/channelAcquire/timeNs", totalduration));
+ return retVal;
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ this.resourcePool.close();
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
index 2476e4c9ebfd..dd44e4fb9c13 100644
--- a/processing/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
+++ b/processing/src/main/java/org/apache/druid/java/util/http/client/pool/ResourcePool.java
@@ -19,362 +19,11 @@
package org.apache.druid.java.util.http.client.pool;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.io.Closer;
-import org.apache.druid.java.util.common.logger.Logger;
-
-import javax.annotation.Nullable;
import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.atomic.AtomicBoolean;
-/**
- * A resource pool based on {@link LoadingCache}. When a resource is first requested for a new key,
- * If the flag: eagerInitialization is true: use {@link EagerCreationResourceHolder}
- * {@link ResourcePoolConfig#getMaxPerKey()} resources are initialized and cached in the {@link #pool}.
- * Else:
- * Initialize a single resource and further lazily using {@link LazyCreationResourceHolder}
- * The individual resource in {@link ResourceHolderPerKey} is valid while (current time - last access time)
- * <= {@link ResourcePoolConfig#getUnusedConnectionTimeoutMillis()}.
- *
- * A resource is closed and reinitialized if {@link ResourceFactory#isGood} returns false or it's expired based on
- * {@link ResourcePoolConfig#getUnusedConnectionTimeoutMillis()}.
- *
- * {@link ResourcePoolConfig#getMaxPerKey() is a hard limit for the max number of resources per cache entry. The total
- * number of resources in {@link ResourceHolderPerKey} cannot be larger than the limit in any case.
- */
-public class ResourcePool implements Closeable
+public interface ResourcePool extends Closeable
{
- private static final Logger log = new Logger(ResourcePool.class);
- private final LoadingCache> pool;
- private final AtomicBoolean closed = new AtomicBoolean(false);
-
- public ResourcePool(final ResourceFactory factory, final ResourcePoolConfig config,
- final boolean eagerInitialization)
- {
- this.pool = CacheBuilder.newBuilder().build(
- new CacheLoader>()
- {
- @Override
- public ResourceHolderPerKey load(K input)
- {
- if (eagerInitialization) {
- return new EagerCreationResourceHolder<>(
- config.getMaxPerKey(),
- config.getUnusedConnectionTimeoutMillis(),
- input,
- factory
- );
- } else {
- return new LazyCreationResourceHolder<>(
- config.getMaxPerKey(),
- config.getUnusedConnectionTimeoutMillis(),
- input,
- factory
- );
- }
- }
- }
- );
- }
-
- /**
- * Returns a {@link ResourceContainer} for the given key or null if this pool is already closed.
- */
- @Nullable
- public ResourceContainer take(final K key)
- {
- if (closed.get()) {
- log.error(StringUtils.format("take(%s) called even though I'm closed.", key));
- return null;
- }
-
- final ResourceHolderPerKey holder;
- try {
- holder = pool.get(key);
- }
- catch (ExecutionException e) {
- throw new RuntimeException(e);
- }
- final V value = holder.get();
-
- return new ResourceContainer()
- {
- private final AtomicBoolean returned = new AtomicBoolean(false);
-
- @Override
- public V get()
- {
- Preconditions.checkState(!returned.get(), "Resource for key[%s] has been returned, cannot get().", key);
- return value;
- }
-
- @Override
- public void returnResource()
- {
- if (returned.getAndSet(true)) {
- log.warn("Resource at key[%s] was returned multiple times?", key);
- } else {
- holder.giveBack(value);
- }
- }
-
- @Override
- protected void finalize() throws Throwable
- {
- if (!returned.get()) {
- log.warn(
- StringUtils.format(
- "Resource[%s] at key[%s] was not returned before Container was finalized, potential resource leak.",
- value,
- key
- )
- );
- returnResource();
- }
- super.finalize();
- }
- };
- }
-
- @Override
- public void close()
- {
- closed.set(true);
- final ConcurrentMap> mapView = pool.asMap();
- Closer closer = Closer.create();
- for (Iterator>> iterator =
- mapView.entrySet().iterator(); iterator.hasNext(); ) {
- Map.Entry> e = iterator.next();
- iterator.remove();
- closer.register(e.getValue());
- }
- try {
- closer.close();
- }
- catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- private static class EagerCreationResourceHolder extends LazyCreationResourceHolder
- {
- private EagerCreationResourceHolder(
- int maxSize,
- long unusedResourceTimeoutMillis,
- K key,
- ResourceFactory factory
- )
- {
- super(maxSize, unusedResourceTimeoutMillis, key, factory);
- // Eagerly Instantiate
- for (int i = 0; i < maxSize; i++) {
- resourceHolderList.add(
- new ResourceHolder<>(
- System.currentTimeMillis(),
- Preconditions.checkNotNull(
- factory.generate(key),
- "factory.generate(key)"
- )
- )
- );
- }
- }
- }
-
- private static class LazyCreationResourceHolder extends ResourceHolderPerKey
- {
- private LazyCreationResourceHolder(
- int maxSize,
- long unusedResourceTimeoutMillis,
- K key,
- ResourceFactory factory
- )
- {
- super(maxSize, unusedResourceTimeoutMillis, key, factory);
- }
- }
-
- private static class ResourceHolderPerKey implements Closeable
- {
- protected final int maxSize;
- private final K key;
- private final ResourceFactory factory;
- private final long unusedResourceTimeoutMillis;
- // Hold previously created / returned resources
- protected final ArrayDeque> resourceHolderList;
- // To keep track of resources that have been successfully returned to caller.
- private int numLentResources = 0;
- private boolean closed = false;
-
- protected ResourceHolderPerKey(
- int maxSize,
- long unusedResourceTimeoutMillis,
- K key,
- ResourceFactory factory
- )
- {
- this.maxSize = maxSize;
- this.key = key;
- this.factory = factory;
- this.unusedResourceTimeoutMillis = unusedResourceTimeoutMillis;
- this.resourceHolderList = new ArrayDeque<>();
- }
-
- /**
- * Returns a resource or null if this holder is already closed or the current thread is interrupted.
- *
- * Try to return a previously created resource if it isGood(). Else, generate a new resource
- */
- @Nullable
- V get()
- {
- final V poolVal;
- // resourceHolderList can't have nulls, so we'll use a null to signal that we need to create a new resource.
- boolean expired = false;
- synchronized (this) {
- while (!closed && (numLentResources == maxSize)) {
- try {
- this.wait();
- }
- catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return null;
- }
- }
-
- if (closed) {
- log.info(StringUtils.format("get() called even though I'm closed. key[%s]", key));
- return null;
- } else if (numLentResources < maxSize) {
- // Attempt to take an existing resource or create one if list is empty, and increment numLentResources
- if (resourceHolderList.isEmpty()) {
- poolVal = factory.generate(key);
- } else {
- ResourceHolder holder = resourceHolderList.removeFirst();
- poolVal = holder.getResource();
- if (System.currentTimeMillis() - holder.getLastAccessedTime() > unusedResourceTimeoutMillis) {
- expired = true;
- }
- }
- numLentResources++;
- } else {
- throw new IllegalStateException("Unexpected state: More objects lent than permissible");
- }
- }
-
- final V retVal;
- // At this point, we must either return a valid resource. Or throw and exception decrement "numLentResources"
- try {
- if (poolVal != null && !expired && factory.isGood(poolVal)) {
- retVal = poolVal;
- } else {
- if (poolVal != null) {
- factory.close(poolVal);
- }
- retVal = factory.generate(key);
- }
- }
- catch (Throwable e) {
- synchronized (this) {
- numLentResources--;
- this.notifyAll();
- }
- Throwables.propagateIfPossible(e);
- throw new RuntimeException(e);
- }
-
- return retVal;
- }
-
- void giveBack(V object)
- {
- Preconditions.checkNotNull(object, "object");
-
- synchronized (this) {
- if (closed) {
- log.info(StringUtils.format("giveBack called after being closed. key[%s]", key));
- factory.close(object);
- return;
- }
-
- if (resourceHolderList.size() >= maxSize) {
- if (holderListContains(object)) {
- log.warn(
- new Exception("Exception for stacktrace"),
- StringUtils.format(
- "Returning object[%s] at key[%s] that has already been returned!? Skipping",
- object,
- key
- )
- );
- } else {
- log.warn(
- new Exception("Exception for stacktrace"),
- StringUtils.format(
- "Returning object[%s] at key[%s] even though we already have all that we can hold[%s]!? Skipping",
- object,
- key,
- resourceHolderList
- )
- );
- }
- return;
- }
-
- resourceHolderList.addLast(new ResourceHolder<>(System.currentTimeMillis(), object));
- numLentResources--;
- this.notifyAll();
- }
- }
-
- private boolean holderListContains(V object)
- {
- return resourceHolderList.stream().anyMatch(a -> a.getResource().equals(object));
- }
-
- @Override
- public void close()
- {
- synchronized (this) {
- closed = true;
- resourceHolderList.forEach(v -> factory.close(v.getResource()));
- resourceHolderList.clear();
- this.notifyAll();
- }
- }
- }
-
- private static class ResourceHolder
- {
- private final long lastAccessedTime;
- private final V resource;
-
- private ResourceHolder(long lastAccessedTime, V resource)
- {
- this.resource = resource;
- this.lastAccessedTime = lastAccessedTime;
- }
-
- private long getLastAccessedTime()
- {
- return lastAccessedTime;
- }
- public V getResource()
- {
- return resource;
- }
+ ResourceContainer take(K key);
- }
}
diff --git a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
index e63e4ee07d94..843c2c46671d 100644
--- a/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
+++ b/processing/src/main/java/org/apache/druid/query/CPUTimeMetricQueryRunner.java
@@ -63,7 +63,9 @@ public Sequence run(final QueryPlus queryPlus, final ResponseContext respo
final QueryPlus queryWithMetrics = queryPlus.withQueryMetrics(queryToolChest);
final Sequence baseSequence = delegate.run(queryWithMetrics, responseContext);
- cpuTimeAccumulator.addAndGet(JvmUtils.getCurrentThreadCpuTime() - startRun);
+ long cpuTimeDelta = JvmUtils.getCurrentThreadCpuTime() - startRun;
+ cpuTimeAccumulator.addAndGet(cpuTimeDelta);
+ responseContext.addCpuNanos(cpuTimeDelta);
return Sequences.wrap(
baseSequence,
@@ -77,7 +79,9 @@ public RetType wrap(Supplier sequenceProcessing)
return sequenceProcessing.get();
}
finally {
- cpuTimeAccumulator.addAndGet(JvmUtils.getCurrentThreadCpuTime() - start);
+ long cpuTimeDelta = JvmUtils.getCurrentThreadCpuTime() - start;
+ cpuTimeAccumulator.addAndGet(cpuTimeDelta);
+ responseContext.addCpuNanos(cpuTimeDelta);
}
}
@@ -85,9 +89,8 @@ public RetType wrap(Supplier sequenceProcessing)
public void after(boolean isDone, Throwable thrown)
{
if (report) {
- final long cpuTimeNs = cpuTimeAccumulator.get();
+ final long cpuTimeNs = responseContext.getCpuNanos();
if (cpuTimeNs > 0) {
- responseContext.addCpuNanos(cpuTimeNs);
queryWithMetrics.getQueryMetrics().reportCpuTime(cpuTimeNs).emit(emitter);
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
index db7346517976..b6c37fb326b1 100644
--- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java
@@ -220,6 +220,12 @@ public QueryMetrics reportQueryTime(long timeNs)
return reportMillisTimeMetric("query/time", timeNs);
}
+ @Override
+ public QueryMetrics reportQueryPlanningTime(long timeNs)
+ {
+ return reportMillisTimeMetric("query/planningTime", timeNs);
+ }
+
@Override
public QueryMetrics reportQueryBytes(long byteCount)
{
@@ -250,6 +256,12 @@ public QueryMetrics reportCpuTime(long timeNs)
return reportMetric("query/cpu/time", TimeUnit.NANOSECONDS.toMicros(timeNs));
}
+ @Override
+ public QueryMetrics reportRowsScannedCount(long rowsScannedCount)
+ {
+ return reportMetric("query/rows/scanned", rowsScannedCount);
+ }
+
@Override
public QueryMetrics reportNodeTimeToFirstByte(long timeNs)
{
@@ -259,8 +271,7 @@ public QueryMetrics reportNodeTimeToFirstByte(long timeNs)
@Override
public QueryMetrics reportBackPressureTime(long timeNs)
{
- // Don't emit by default.
- return this;
+ return reportMillisTimeMetric("query/node/backpressure", timeNs);
}
@Override
@@ -278,64 +289,55 @@ public QueryMetrics reportNodeBytes(long byteCount)
@Override
public QueryMetrics reportBitmapConstructionTime(long timeNs)
{
- // Don't emit by default.
- return this;
+ return reportMillisTimeMetric("query/node/bitmapConstructionTime", timeNs);
}
@Override
public QueryMetrics reportSegmentRows(long numRows)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/segment/rows", numRows);
}
@Override
public QueryMetrics reportPreFilteredRows(long numRows)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/preFiltered/rows", numRows);
}
@Override
public QueryMetrics reportParallelMergeParallelism(int parallelism)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/parallelMerge/parallelism", parallelism);
}
@Override
public QueryMetrics reportParallelMergeInputSequences(long numSequences)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/parallelMerge/inputSequences", numSequences);
}
@Override
public QueryMetrics reportParallelMergeInputRows(long numRows)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/parallelMerge/inputRows", numRows);
}
@Override
public QueryMetrics reportParallelMergeOutputRows(long numRows)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/parallelMerge/outputRows", numRows);
}
@Override
public QueryMetrics reportParallelMergeTaskCount(long numTasks)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/parallelMerge/taskCount", numTasks);
}
@Override
public QueryMetrics reportParallelMergeTotalCpuTime(long timeNs)
{
- // Don't emit by default.
- return this;
+ return reportMillisTimeMetric("query/parallelMerge/totalCpuTime", timeNs);
}
@Override
@@ -362,8 +364,7 @@ public QueryMetrics reportParallelMergeSlowestPartitionTime(long time
@Override
public QueryMetrics reportQueriedSegmentCount(long segmentCount)
{
- // Don't emit by default.
- return this;
+ return reportMetric("query/queriedSegment/count", segmentCount);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/MetricsEmittingMergingBlockingPool.java b/processing/src/main/java/org/apache/druid/query/MetricsEmittingMergingBlockingPool.java
new file mode 100644
index 000000000000..10a89a48e6f0
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/MetricsEmittingMergingBlockingPool.java
@@ -0,0 +1,47 @@
+/*
+ * 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.query;
+
+import com.google.common.base.Supplier;
+import org.apache.druid.collections.DefaultBlockingPool;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
+
+public class MetricsEmittingMergingBlockingPool extends DefaultBlockingPool
+ implements ExecutorServiceMonitor.MetricEmitter
+{
+
+ public MetricsEmittingMergingBlockingPool(
+ Supplier generator,
+ int limit,
+ ExecutorServiceMonitor executorServiceMonitor
+ )
+ {
+ super(generator, limit);
+ executorServiceMonitor.add(this);
+ }
+
+ @Override
+ public void emitMetrics(ServiceEmitter emitter, ServiceMetricEvent.Builder metricBuilder)
+ {
+ emitter.emit(metricBuilder.setMetric("query/merge/buffersUsed", getUsedBufferCount()));
+ emitter.emit(metricBuilder.setMetric("query/merge/totalBuffers", maxSize()));
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
index 6dfad11fae40..c2a7e09aae87 100644
--- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java
@@ -308,6 +308,14 @@ default void filterBundle(FilterBundle.BundleInfo bundleInfo)
*/
QueryMetrics reportQueryTime(long timeNs);
+ /**
+ * Registers "query planning time" metric.
+ *
+ * Measures the time taken to plan the query. This includes time spent in determining segments for the given
+ * time interval and determining the Data nodes responsible for the segments.
+ */
+ QueryMetrics reportQueryPlanningTime(long timeNs);
+
/**
* Registers "query bytes" metric.
*
@@ -355,6 +363,11 @@ default void filterBundle(FilterBundle.BundleInfo bundleInfo)
*/
QueryMetrics reportCpuTime(long timeNs);
+ /**
+ * Registers "rows scanned count" metric.
+ */
+ QueryMetrics reportRowsScannedCount(long rowsScannedCount);
+
/**
* Registers "time to first byte" metric.
*/
diff --git a/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java b/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
index 6727782cc406..5e037d6eb86f 100644
--- a/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
+++ b/processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
@@ -44,6 +44,7 @@
import java.util.Comparator;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
@@ -415,6 +416,13 @@ public Object mergeValues(Object oldValue, Object newValue)
}
};
+ /**
+ * Query Segment Count.
+ */
+ public static final Key QUERY_SEGMENT_COUNT = new LongKey(
+ "querySegmentCount",
+ false);
+
/**
* Query fail time (current time + timeout).
*/
@@ -476,21 +484,21 @@ public Object mergeValues(Object oldValue, Object newValue)
private final ConcurrentMap registeredKeys = new ConcurrentSkipListMap<>();
static {
- instance().registerKeys(
- new Key[]{
- UNCOVERED_INTERVALS,
- UNCOVERED_INTERVALS_OVERFLOWED,
- REMAINING_RESPONSES_FROM_QUERY_SERVERS,
- MISSING_SEGMENTS,
- ETAG,
- QUERY_TOTAL_BYTES_GATHERED,
- QUERY_FAIL_DEADLINE_MILLIS,
- TIMEOUT_AT,
- NUM_SCANNED_ROWS,
- CPU_CONSUMED_NANOS,
- TRUNCATED,
- }
- );
+ instance().registerKeys(new Key[]
+ {
+ UNCOVERED_INTERVALS,
+ UNCOVERED_INTERVALS_OVERFLOWED,
+ REMAINING_RESPONSES_FROM_QUERY_SERVERS,
+ MISSING_SEGMENTS,
+ ETAG,
+ QUERY_TOTAL_BYTES_GATHERED,
+ QUERY_FAIL_DEADLINE_MILLIS,
+ TIMEOUT_AT,
+ NUM_SCANNED_ROWS,
+ CPU_CONSUMED_NANOS,
+ TRUNCATED,
+ QUERY_SEGMENT_COUNT,
+ });
}
/**
@@ -698,6 +706,17 @@ public Long getCpuNanos()
return (Long) get(Keys.CPU_CONSUMED_NANOS);
}
+ public Long getQuerySegmentCount()
+ {
+ return (Long) get(Keys.QUERY_SEGMENT_COUNT);
+ }
+
+ public Long getValueOrDefaultZero(Function getter)
+ {
+ Long value = getter.apply(this);
+ return Objects.nonNull(value) ? value : 0L;
+ }
+
public Object remove(Key key)
{
return getDelegate().remove(key);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
index 81ec050ce088..b4e4d5da4032 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
@@ -265,7 +265,7 @@ private Sequence mergeGroupByResultsWithoutPushDown(
return groupingEngine.processSubtotalsSpec(
query,
resource,
- groupingEngine.processSubqueryResult(subquery, query, resource, finalizingResults, false)
+ groupingEngine.processSubqueryResult(subquery, query, resource, finalizingResults, false, context)
);
} else {
return groupingEngine.applyPostProcessing(
@@ -274,7 +274,8 @@ private Sequence mergeGroupByResultsWithoutPushDown(
query,
resource,
finalizingResults,
- false
+ false,
+ context
),
query
);
@@ -309,7 +310,8 @@ private Sequence mergeResultsWithNestedQueryPushDown(
rewrittenQuery,
resource,
finalizedResults,
- true
+ true,
+ context
),
query
);
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
index adf43e8cc16c..dfaac4ffe198 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactory.java
@@ -102,7 +102,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r
throw new ISE("Got a [%s] which isn't a %s", query.getClass(), GroupByQuery.class);
}
- return groupingEngine.process((GroupByQuery) query, adapter, (GroupByQueryMetrics) queryPlus.getQueryMetrics());
+ return groupingEngine.process((GroupByQuery) query, adapter, (GroupByQueryMetrics) queryPlus.getQueryMetrics(), responseContext);
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
index 6451fb9b943d..516b9ab1313d 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java
@@ -188,7 +188,7 @@ public static GroupByQueryResources prepareResource(
);
} else {
return new GroupByQueryResources(
- mergeBufferHolders.subList(0, requiredMergeBufferNumForToolchestMerge),
+ mergeBufferHolders.subList(0, requiredMergeBufferNumForToolchestMerge),
mergeBufferHolders.subList(requiredMergeBufferNumForToolchestMerge, requiredMergeBufferNum)
);
}
@@ -420,7 +420,7 @@ public Sequence mergeResults(
/**
* Merges a variety of single-segment query runners into a combined runner. Used by
* {@link GroupByQueryRunnerFactory#mergeRunners(QueryProcessingPool, Iterable)}. In
- * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)} (the runners created
+ * that sense, it is intended to go along with {@link #process(GroupByQuery, StorageAdapter, GroupByQueryMetrics, ResponseContext)} (the runners created
* by that method will be fed into this method).
*
* This is primarily called on the data servers, to merge the results from processing on the segments. This method can
@@ -468,7 +468,8 @@ public QueryRunner mergeRunners(
public Sequence process(
GroupByQuery query,
StorageAdapter storageAdapter,
- @Nullable GroupByQueryMetrics groupByQueryMetrics
+ @Nullable GroupByQueryMetrics groupByQueryMetrics,
+ ResponseContext responseContext
)
{
final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query);
@@ -514,7 +515,8 @@ public Sequence process(
interval,
querySpecificConfig,
processingConfig,
- groupByQueryMetrics
+ groupByQueryMetrics,
+ responseContext
);
} else {
result = GroupByQueryEngine.process(
@@ -526,7 +528,8 @@ public Sequence process(
processingConfig,
filter,
interval,
- groupByQueryMetrics
+ groupByQueryMetrics,
+ responseContext
);
}
@@ -575,7 +578,8 @@ public Sequence processSubqueryResult(
GroupByQuery query,
GroupByQueryResources resource,
Sequence subqueryResult,
- boolean wasQueryPushedDown
+ boolean wasQueryPushedDown,
+ ResponseContext context
)
{
// Keep a reference to resultSupplier outside the "try" so we can close it if something goes wrong
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
index 085e6022aabd..c9b8240da3e1 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngine.java
@@ -33,6 +33,7 @@
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.groupby.GroupByQuery;
@@ -50,6 +51,7 @@
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.RowCountingCursorDecorator;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
@@ -100,7 +102,8 @@ public static Sequence process(
final DruidProcessingConfig processingConfig,
@Nullable final Filter filter,
final Interval interval,
- @Nullable final GroupByQueryMetrics groupByQueryMetrics
+ @Nullable final GroupByQueryMetrics groupByQueryMetrics,
+ ResponseContext responseContext
)
{
final Sequence cursors = storageAdapter.makeCursors(
@@ -112,7 +115,7 @@ public static Sequence process(
groupByQueryMetrics
);
- return cursors.flatMap(
+ return cursors.map(cursor -> new RowCountingCursorDecorator(cursor, responseContext)).flatMap(
cursor -> new BaseSequence<>(
new BaseSequence.IteratorMaker>()
{
@@ -515,9 +518,9 @@ protected void aggregateSingleValueDims(Grouper grouper)
for (GroupByColumnSelectorPlus dim : dims) {
final GroupByColumnSelectorStrategy strategy = dim.getColumnSelectorStrategy();
selectorInternalFootprint += strategy.writeToKeyBuffer(
- dim.getKeyBufferPosition(),
- dim.getSelector(),
- keyBuffer
+ dim.getKeyBufferPosition(),
+ dim.getSelector(),
+ keyBuffer
);
}
keyBuffer.rewind();
@@ -554,16 +557,16 @@ protected void aggregateMultiValueDims(Grouper grouper)
for (int i = 0; i < dims.length; i++) {
GroupByColumnSelectorStrategy strategy = dims[i].getColumnSelectorStrategy();
selectorInternalFootprint += strategy.initColumnValues(
- dims[i].getSelector(),
- i,
- valuess
+ dims[i].getSelector(),
+ i,
+ valuess
);
strategy.initGroupingKeyColumnValue(
- dims[i].getKeyBufferPosition(),
- i,
- valuess[i],
- keyBuffer,
- stack
+ dims[i].getKeyBufferPosition(),
+ i,
+ valuess[i],
+ keyBuffer,
+ stack
);
}
}
@@ -584,10 +587,10 @@ protected void aggregateMultiValueDims(Grouper grouper)
if (stackPointer >= 0) {
doAggregate = dims[stackPointer].getColumnSelectorStrategy().checkRowIndexAndAddValueToGroupingKey(
- dims[stackPointer].getKeyBufferPosition(),
- valuess[stackPointer],
- stack[stackPointer],
- keyBuffer
+ dims[stackPointer].getKeyBufferPosition(),
+ valuess[stackPointer],
+ stack[stackPointer],
+ keyBuffer
);
if (doAggregate) {
@@ -597,11 +600,11 @@ protected void aggregateMultiValueDims(Grouper grouper)
stack[stackPointer]++;
for (int i = stackPointer + 1; i < stack.length; i++) {
dims[i].getColumnSelectorStrategy().initGroupingKeyColumnValue(
- dims[i].getKeyBufferPosition(),
- i,
- valuess[i],
- keyBuffer,
- stack
+ dims[i].getKeyBufferPosition(),
+ i,
+ valuess[i],
+ keyBuffer,
+ stack
);
}
stackPointer = stack.length - 1;
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
index a2fc9cec8a6e..88b2624f453e 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/VectorGroupByEngine.java
@@ -29,6 +29,7 @@
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.aggregation.AggregatorAdapters;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.groupby.GroupByQuery;
@@ -89,7 +90,8 @@ public static Sequence process(
final Interval interval,
final GroupByQueryConfig config,
final DruidProcessingConfig processingConfig,
- @Nullable final GroupByQueryMetrics groupByQueryMetrics
+ @Nullable final GroupByQueryMetrics groupByQueryMetrics,
+ ResponseContext responseContext
)
{
if (!canVectorize(query, storageAdapter, filter)) {
@@ -147,15 +149,16 @@ public void close()
).collect(Collectors.toList());
return new VectorGroupByEngineIterator(
- query,
- config,
- processingConfig,
- storageAdapter,
- cursor,
- interval,
- dimensions,
- processingBuffer,
- fudgeTimestamp
+ query,
+ config,
+ processingConfig,
+ storageAdapter,
+ cursor,
+ interval,
+ dimensions,
+ processingBuffer,
+ fudgeTimestamp,
+ responseContext
);
}
catch (Throwable e) {
@@ -249,6 +252,7 @@ static class VectorGroupByEngineIterator implements CloseableIterator
private final int keySize;
private final WritableMemory keySpace;
private final VectorGrouper vectorGrouper;
+ private final ResponseContext responseContext;
@Nullable
private final VectorCursorGranularizer granulizer;
@@ -278,7 +282,8 @@ static class VectorGroupByEngineIterator implements CloseableIterator
final Interval queryInterval,
final List selectors,
final ByteBuffer processingBuffer,
- @Nullable final DateTime fudgeTimestamp
+ @Nullable final DateTime fudgeTimestamp,
+ ResponseContext responseContext
)
{
this.query = query;
@@ -293,6 +298,7 @@ static class VectorGroupByEngineIterator implements CloseableIterator
this.keySpace = WritableMemory.allocate(keySize * cursor.getMaxVectorSize());
this.vectorGrouper = makeGrouper();
this.granulizer = VectorCursorGranularizer.create(storageAdapter, cursor, query.getGranularity(), queryInterval);
+ this.responseContext = responseContext;
if (granulizer != null) {
this.bucketIterator = granulizer.getBucketIterable().iterator();
@@ -400,6 +406,7 @@ private CloseableGrouperIterator initNewDelegate()
? fudgeTimestamp
: query.getGranularity().toDateTime(bucketInterval.getStartMillis());
+ long numRowsScanned = 0l;
while (!cursor.isDone()) {
final int startOffset;
@@ -428,6 +435,7 @@ private CloseableGrouperIterator initNewDelegate()
startOffset,
granulizer.getEndOffset()
);
+ numRowsScanned += granulizer.getEndOffset() - startOffset;
if (result.isOk()) {
partiallyAggregatedRows = -1;
@@ -452,6 +460,9 @@ private CloseableGrouperIterator initNewDelegate()
break;
}
}
+ if (this.responseContext != null) {
+ this.responseContext.addRowScanCount(numRowsScanned);
+ }
final boolean resultRowHasTimestamp = query.getResultRowHasTimestamp();
final int resultRowDimensionStart = query.getResultRowDimensionStart();
diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
index ef484df5f183..76bdc45103bc 100644
--- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
+++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java
@@ -187,6 +187,12 @@ public QueryMetrics reportQueryTime(long timeNs)
return delegateQueryMetrics.reportQueryTime(timeNs);
}
+ @Override
+ public QueryMetrics reportQueryPlanningTime(long timeNs)
+ {
+ return delegateQueryMetrics.reportQueryPlanningTime(timeNs);
+ }
+
@Override
public QueryMetrics reportQueryBytes(long byteCount)
{
@@ -217,6 +223,12 @@ public QueryMetrics reportCpuTime(long timeNs)
return delegateQueryMetrics.reportCpuTime(timeNs);
}
+ @Override
+ public QueryMetrics reportRowsScannedCount(long rowsScannedCount)
+ {
+ return delegateQueryMetrics.reportRowsScannedCount(rowsScannedCount);
+ }
+
@Override
public QueryMetrics reportNodeTimeToFirstByte(long timeNs)
{
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
index c5e83b84e87c..e374ca8fbd7b 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java
@@ -36,9 +36,11 @@
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.vector.VectorCursorGranularizer;
import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.RowCountingCursorDecorator;
import org.apache.druid.segment.SegmentMissingException;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
@@ -84,7 +86,8 @@ public TimeseriesQueryEngine(
public Sequence> process(
final TimeseriesQuery query,
final StorageAdapter adapter,
- @Nullable final TimeseriesQueryMetrics timeseriesQueryMetrics
+ @Nullable final TimeseriesQueryMetrics timeseriesQueryMetrics,
+ ResponseContext responseContext
)
{
if (adapter == null) {
@@ -109,9 +112,9 @@ public Sequence> process(
final Sequence> result;
if (doVectorize) {
- result = processVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics);
+ result = processVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics, responseContext);
} else {
- result = processNonVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics);
+ result = processNonVectorized(query, adapter, filter, interval, gran, descending, timeseriesQueryMetrics, responseContext);
}
final int limit = query.getLimit();
@@ -129,7 +132,8 @@ private Sequence> processVectorized(
final Interval queryInterval,
final Granularity gran,
final boolean descending,
- final TimeseriesQueryMetrics timeseriesQueryMetrics
+ final TimeseriesQueryMetrics timeseriesQueryMetrics,
+ final ResponseContext responseContext
)
{
final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
@@ -187,6 +191,7 @@ private Sequence> processVectorized(
bucketInterval -> {
// Whether or not the current bucket is empty
boolean emptyBucket = true;
+ long numRowsScanned = 0;
while (!cursor.isDone()) {
granularizer.setCurrentOffsets(bucketInterval);
@@ -202,7 +207,7 @@ private Sequence> processVectorized(
granularizer.getStartOffset(),
granularizer.getEndOffset()
);
-
+ numRowsScanned += granularizer.getEndOffset() - granularizer.getStartOffset();
emptyBucket = false;
}
@@ -210,6 +215,9 @@ private Sequence> processVectorized(
break;
}
}
+ if (responseContext != null) {
+ responseContext.addRowScanCount(numRowsScanned);
+ }
if (emptyBucket && skipEmptyBuckets) {
// Return null, will get filtered out later by the Objects::nonNull filter.
@@ -256,12 +264,12 @@ private Sequence> processNonVectorized(
final Interval queryInterval,
final Granularity gran,
final boolean descending,
- final TimeseriesQueryMetrics timeseriesQueryMetrics
+ final TimeseriesQueryMetrics timeseriesQueryMetrics,
+ final ResponseContext responseContext
)
{
final boolean skipEmptyBuckets = query.isSkipEmptyBuckets();
final List aggregatorSpecs = query.getAggregatorSpecs();
-
return QueryRunnerHelper.makeCursorBasedQuery(
adapter,
Collections.singletonList(queryInterval),
@@ -269,7 +277,8 @@ private Sequence> processNonVectorized(
query.getVirtualColumns(),
descending,
gran,
- cursor -> {
+ c -> {
+ RowCountingCursorDecorator cursor = new RowCountingCursorDecorator(c, responseContext);
if (skipEmptyBuckets && cursor.isDone()) {
return null;
}
@@ -289,7 +298,6 @@ private Sequence> processNonVectorized(
}
cursor.advance();
}
-
TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime());
for (int i = 0; i < aggregatorSpecs.size(); i++) {
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
index fe3d420e5662..add1e52ab6d9 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerFactory.java
@@ -99,7 +99,7 @@ public Sequence> run(
throw new ISE("Got a [%s] which isn't a %s", input.getClass(), TimeseriesQuery.class);
}
- return engine.process((TimeseriesQuery) input, adapter, (TimeseriesQueryMetrics) queryPlus.getQueryMetrics());
+ return engine.process((TimeseriesQuery) input, adapter, (TimeseriesQueryMetrics) queryPlus.getQueryMetrics(), responseContext);
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
index 2180b73253ce..994b9e2df8a0 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/AggregateTopNMetricFirstAlgorithm.java
@@ -26,6 +26,7 @@
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.StorageAdapter;
@@ -72,7 +73,8 @@ public void run(
TopNParams params,
TopNResultBuilder resultBuilder,
int[] ints,
- @Nullable TopNQueryMetrics queryMetrics
+ @Nullable TopNQueryMetrics queryMetrics,
+ ResponseContext responseContext
)
{
final String metric = query.getTopNMetricSpec().getMetricName(query.getDimensionSpec());
@@ -98,7 +100,8 @@ public void run(
singleMetricParam,
singleMetricResultBuilder,
null,
- null // Don't collect metrics during the preparation run.
+ null, // Don't collect metrics during the preparation run.
+ responseContext
);
// Get only the topN dimension values
@@ -117,7 +120,8 @@ public void run(
allMetricsParam,
resultBuilder,
dimValSelector,
- queryMetrics
+ queryMetrics,
+ responseContext
);
}
finally {
diff --git a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
index f34464a49d0e..89e7f5aee29a 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/BaseTopNAlgorithm.java
@@ -25,6 +25,7 @@
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IdLookup;
@@ -75,13 +76,14 @@ public void run(
Parameters params,
TopNResultBuilder resultBuilder,
DimValSelector dimValSelector,
- @Nullable TopNQueryMetrics queryMetrics
+ @Nullable TopNQueryMetrics queryMetrics,
+ ResponseContext responseContext
)
{
if (params.getCardinality() != TopNParams.CARDINALITY_UNKNOWN) {
- runWithCardinalityKnown(params, resultBuilder, dimValSelector, queryMetrics);
+ runWithCardinalityKnown(params, resultBuilder, dimValSelector, queryMetrics, responseContext);
} else {
- runWithCardinalityUnknown(params, resultBuilder, queryMetrics);
+ runWithCardinalityUnknown(params, resultBuilder, queryMetrics, responseContext);
}
}
@@ -89,7 +91,8 @@ private void runWithCardinalityKnown(
Parameters params,
TopNResultBuilder resultBuilder,
DimValSelector dimValSelector,
- @Nullable TopNQueryMetrics queryMetrics
+ @Nullable TopNQueryMetrics queryMetrics,
+ ResponseContext responseContext
)
{
if (queryMetrics != null) {
@@ -125,6 +128,7 @@ private void runWithCardinalityKnown(
numProcessed += numToProcess;
params.getCursor().reset();
}
+ responseContext.addRowScanCount(processedRows);
if (queryMetrics != null) {
queryMetrics.addProcessedRows(processedRows);
queryMetrics.stopRecordingScanTime();
@@ -142,7 +146,8 @@ private void runWithCardinalityKnown(
private void runWithCardinalityUnknown(
Parameters params,
TopNResultBuilder resultBuilder,
- @Nullable TopNQueryMetrics queryMetrics
+ @Nullable TopNQueryMetrics queryMetrics,
+ ResponseContext responseContext
)
{
DimValAggregateStore aggregatesStore = makeDimValAggregateStore(params);
@@ -153,6 +158,7 @@ private void runWithCardinalityUnknown(
updateResults(params, null, aggregatesStore, resultBuilder);
resetAggregators(aggregatesStore);
params.getCursor().reset();
+ responseContext.addRowScanCount(processedRows);
if (queryMetrics != null) {
queryMetrics.addProcessedRows(processedRows);
queryMetrics.stopRecordingScanTime();
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java
index 31b4d9204e8d..8c502aa5046c 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNAlgorithm.java
@@ -21,6 +21,7 @@
import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.segment.Cursor;
@@ -40,7 +41,8 @@ void run(
Parameters params,
TopNResultBuilder resultBuilder,
DimValSelector dimValSelector,
- @Nullable TopNQueryMetrics queryMetrics
+ @Nullable TopNQueryMetrics queryMetrics,
+ ResponseContext responseContext
);
void cleanup(Parameters params);
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java
index 96fb62f9012a..029390a8e4c0 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNMapFn.java
@@ -21,6 +21,7 @@
import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.Result;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
import org.apache.druid.segment.Cursor;
@@ -44,7 +45,7 @@ public TopNMapFn(
@SuppressWarnings("unchecked")
@Nullable
- public Result apply(final Cursor cursor, final @Nullable TopNQueryMetrics queryMetrics)
+ public Result apply(final Cursor cursor, final @Nullable TopNQueryMetrics queryMetrics, ResponseContext responseContext)
{
final ColumnSelectorPlus> selectorPlus =
DimensionHandlerUtils.createColumnSelectorPlus(
@@ -67,7 +68,7 @@ public Result apply(final Cursor cursor, final @Nullable TopNQu
TopNResultBuilder resultBuilder = BaseTopNAlgorithm.makeResultBuilder(params, query);
- topNAlgorithm.run(params, resultBuilder, null, queryMetrics);
+ topNAlgorithm.run(params, resultBuilder, null, queryMetrics, responseContext);
return resultBuilder.build();
}
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
index 50b8a30d1028..016b0a30a585 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java
@@ -27,6 +27,7 @@
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.SegmentMissingException;
@@ -61,9 +62,10 @@ public TopNQueryEngine(NonBlockingPool bufferPool)
* {@link AggregatorFactory} and create or update {@link TopNResultValue}
*/
public Sequence> query(
- final TopNQuery query,
- final StorageAdapter adapter,
- final @Nullable TopNQueryMetrics queryMetrics
+ final TopNQuery query,
+ final StorageAdapter adapter,
+ final @Nullable TopNQueryMetrics queryMetrics,
+ final ResponseContext responseContext
)
{
if (adapter == null) {
@@ -97,7 +99,7 @@ public Sequence> query(
if (queryMetrics != null) {
queryMetrics.cursor(input);
}
- return mapFn.apply(input, queryMetrics);
+ return mapFn.apply(input, queryMetrics, responseContext);
}
),
Predicates.notNull()
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
index 821d37e3a199..6378f02987ed 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryRunnerFactory.java
@@ -74,7 +74,7 @@ public Sequence> run(
}
TopNQuery query = (TopNQuery) input.getQuery();
- return queryEngine.query(query, segment.asStorageAdapter(), (TopNQueryMetrics) input.getQueryMetrics());
+ return queryEngine.query(query, segment.asStorageAdapter(), (TopNQueryMetrics) input.getQueryMetrics(), responseContext);
}
};
diff --git a/processing/src/main/java/org/apache/druid/segment/RowCountingCursorDecorator.java b/processing/src/main/java/org/apache/druid/segment/RowCountingCursorDecorator.java
new file mode 100644
index 000000000000..56906745701f
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/segment/RowCountingCursorDecorator.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+import org.apache.druid.query.context.ResponseContext;
+import org.joda.time.DateTime;
+
+public class RowCountingCursorDecorator implements Cursor
+{
+ private Cursor cursor;
+ private ResponseContext responseContext;
+
+ public RowCountingCursorDecorator(Cursor cursor, ResponseContext responseContext)
+ {
+ this.cursor = cursor;
+ this.responseContext = responseContext;
+ }
+
+ @Override
+ public ColumnSelectorFactory getColumnSelectorFactory()
+ {
+ return this.cursor.getColumnSelectorFactory();
+ }
+
+ @Override
+ public DateTime getTime()
+ {
+ return this.cursor.getTime();
+ }
+
+ @Override
+ public void advance()
+ {
+ this.cursor.advance();
+ if (this.responseContext != null) {
+ this.responseContext.addRowScanCount(1);
+ }
+ }
+
+ @Override
+ public void advanceUninterruptibly()
+ {
+ this.cursor.advanceUninterruptibly();
+ }
+
+ @Override
+ public boolean isDone()
+ {
+ return this.cursor.isDone();
+ }
+
+ @Override
+ public boolean isDoneOrInterrupted()
+ {
+ return this.cursor.isDoneOrInterrupted();
+ }
+
+ @Override
+ public void reset()
+ {
+ this.cursor.reset();
+ }
+}
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..06e2168e0d30 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
@@ -29,7 +29,7 @@
*/
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, 0, 0, 0);
@Override
public long getProcessed()
@@ -79,6 +79,18 @@ public void incrementThrownAway()
}
+ @Override
+ public long getFiltered()
+ {
+ return 0;
+ }
+
+ @Override
+ public void incrementFiltered()
+ {
+
+ }
+
@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 3085376b8223..d4b6e8c1363c 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 FILTERED = "filtered";
/**
* Number of bytes read by an ingestion task.
@@ -75,6 +76,9 @@ default long getProcessedBytes()
long getThrownAway();
void incrementThrownAway();
+ long getFiltered();
+ void incrementFiltered();
+
RowIngestionMetersTotals getTotals();
Map getMovingAverages();
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..4e201c0242f0 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
@@ -31,6 +31,7 @@ public class RowIngestionMetersTotals
private final long processedWithError;
private final long thrownAway;
private final long unparseable;
+ private final long filtered;
@JsonCreator
public RowIngestionMetersTotals(
@@ -38,7 +39,8 @@ public RowIngestionMetersTotals(
@JsonProperty("processedBytes") long processedBytes,
@JsonProperty("processedWithError") long processedWithError,
@JsonProperty("thrownAway") long thrownAway,
- @JsonProperty("unparseable") long unparseable
+ @JsonProperty("unparseable") long unparseable,
+ @JsonProperty("filtered") long filtered
)
{
this.processed = processed;
@@ -46,6 +48,7 @@ public RowIngestionMetersTotals(
this.processedWithError = processedWithError;
this.thrownAway = thrownAway;
this.unparseable = unparseable;
+ this.filtered = filtered;
}
@JsonProperty
@@ -78,6 +81,12 @@ public long getUnparseable()
return unparseable;
}
+ @JsonProperty
+ public long getFiltered()
+ {
+ return filtered;
+ }
+
@Override
public boolean equals(Object o)
{
@@ -92,13 +101,14 @@ public boolean equals(Object o)
&& processedBytes == that.processedBytes
&& processedWithError == that.processedWithError
&& thrownAway == that.thrownAway
- && unparseable == that.unparseable;
+ && unparseable == that.unparseable
+ && filtered == that.filtered;
}
@Override
public int hashCode()
{
- return Objects.hash(processed, processedBytes, processedWithError, thrownAway, unparseable);
+ return Objects.hash(processed, processedBytes, processedWithError, thrownAway, unparseable, filtered);
}
@Override
@@ -110,6 +120,7 @@ public String toString()
", processedWithError=" + processedWithError +
", thrownAway=" + thrownAway +
", unparseable=" + unparseable +
+ ", filtered=" + filtered +
'}';
}
}
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..bb8109a575d3 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
@@ -27,6 +27,7 @@ public class SimpleRowIngestionMeters implements RowIngestionMeters
private long processedWithError;
private long unparseable;
private long thrownAway;
+ private long filtered;
private long processedBytes;
@Override
@@ -89,6 +90,18 @@ public void incrementThrownAway()
thrownAway++;
}
+ @Override
+ public long getFiltered()
+ {
+ return filtered;
+ }
+
+ @Override
+ public void incrementFiltered()
+ {
+ filtered++;
+ }
+
@Override
public RowIngestionMetersTotals getTotals()
{
@@ -97,7 +110,8 @@ public RowIngestionMetersTotals getTotals()
processedBytes,
processedWithError,
thrownAway,
- unparseable
+ unparseable,
+ filtered
);
}
@@ -113,6 +127,7 @@ public void addRowIngestionMetersTotals(RowIngestionMetersTotals rowIngestionMet
this.processedWithError += rowIngestionMetersTotals.getProcessedWithError();
this.unparseable += rowIngestionMetersTotals.getUnparseable();
this.thrownAway += rowIngestionMetersTotals.getThrownAway();
+ this.filtered += rowIngestionMetersTotals.getFiltered();
this.processedBytes += rowIngestionMetersTotals.getProcessedBytes();
}
}
diff --git a/processing/src/test/java/org/apache/druid/collections/BlockingPoolTest.java b/processing/src/test/java/org/apache/druid/collections/BlockingPoolTest.java
index cc5b82ba26e0..3b934bef17f5 100644
--- a/processing/src/test/java/org/apache/druid/collections/BlockingPoolTest.java
+++ b/processing/src/test/java/org/apache/druid/collections/BlockingPoolTest.java
@@ -292,4 +292,14 @@ public void testConcurrentTakeBatchClose() throws ExecutionException, Interrupte
r2.forEach(ReferenceCountingResourceHolder::close);
Assert.assertEquals(pool.maxSize(), pool.getPoolSize());
}
+
+ @Test(timeout = 60_000L)
+ public void testGetUsedBufferCount()
+ {
+ final List> holder = pool.takeBatch(6, 100L);
+ Assert.assertNotNull(holder);
+ Assert.assertEquals(6, pool.getUsedBufferCount());
+ holder.forEach(ReferenceCountingResourceHolder::close);
+ Assert.assertEquals(0, pool.getUsedBufferCount());
+ }
}
diff --git a/processing/src/test/java/org/apache/druid/concurrent/ExecsTest.java b/processing/src/test/java/org/apache/druid/concurrent/ExecsTest.java
index de76af7cb7bb..454eb43fb935 100644
--- a/processing/src/test/java/org/apache/druid/concurrent/ExecsTest.java
+++ b/processing/src/test/java/org/apache/druid/concurrent/ExecsTest.java
@@ -19,15 +19,20 @@
package org.apache.druid.concurrent;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
import org.junit.Assert;
import org.junit.Test;
+import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
public class ExecsTest
@@ -118,6 +123,37 @@ public void run()
producer.shutdown();
}
+ @Test
+ public void testSynchronousQueueSingleThreadedExecutor() throws Exception
+ {
+ // The implementation of Execs.newBlockingSingleThreaded() rejectedExecutionHandler should not add tasks when it's in shutDown state
+ // When capacity is 0, a SynchronousQueue is used and if a task is put in it in ShutDown state, it will forever stuck in WAITING state
+ // as executor will not take() the task to schedule it.
+ final ListeningExecutorService intermediateTempExecutor = MoreExecutors.listeningDecorator(
+ Execs.newBlockingSingleThreaded("[TASK_ID]-appenderator-abandon", 0)
+ );
+ Callable task = () -> {
+ try {
+ Thread.sleep(500); // Simulate long-running task
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt(); // Restore interrupted status
+ }
+ return null;
+ };
+
+ // Submit multiple tasks together
+ ListenableFuture unused = intermediateTempExecutor.submit(task);
+ unused = intermediateTempExecutor.submit(task);
+ unused = intermediateTempExecutor.submit(task);
+
+ intermediateTempExecutor.shutdownNow();
+ // Submit task after shutDown / shutDownNow should not be added in queue
+ unused = intermediateTempExecutor.submit(task);
+ Assert.assertTrue(intermediateTempExecutor.awaitTermination(10, TimeUnit.SECONDS));
+ Assert.assertTrue(intermediateTempExecutor.isShutdown());
+ }
+
@Test
public void testDirectExecutorFactory()
{
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 d71964eed668..6a36757019dc 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
@@ -141,7 +141,8 @@ public void testWritePlainMapAndReadAsReportMap() throws Exception
"processedBytes", 0,
"unparseable", 0,
"thrownAway", 0,
- "processedWithError", 0
+ "processedWithError", 0,
+ "filtered", 0
);
final Map emptyAverages = ImmutableMap.of(
@@ -352,7 +353,8 @@ private void verifyTotalRowStats(
"processedBytes", (int) determinePartitionTotalStats.getProcessedBytes(),
"processedWithError", (int) determinePartitionTotalStats.getProcessedWithError(),
"thrownAway", (int) determinePartitionTotalStats.getThrownAway(),
- "unparseable", (int) determinePartitionTotalStats.getUnparseable()
+ "unparseable", (int) determinePartitionTotalStats.getUnparseable(),
+ "filtered", (int) determinePartitionTotalStats.getFiltered()
),
observedTotals.get("determinePartitions")
);
@@ -362,7 +364,8 @@ private void verifyTotalRowStats(
"processedBytes", (int) buildSegmentTotalStats.getProcessedBytes(),
"processedWithError", (int) buildSegmentTotalStats.getProcessedWithError(),
"thrownAway", (int) buildSegmentTotalStats.getThrownAway(),
- "unparseable", (int) buildSegmentTotalStats.getUnparseable()
+ "unparseable", (int) buildSegmentTotalStats.getUnparseable(),
+ "filtered", (int) buildSegmentTotalStats.getFiltered()
),
observedTotals.get("buildSegments")
);
diff --git a/processing/src/test/java/org/apache/druid/java/util/common/GranularityTest.java b/processing/src/test/java/org/apache/druid/java/util/common/GranularityTest.java
index f4106fe99d1d..45e7a5d2b1a7 100644
--- a/processing/src/test/java/org/apache/druid/java/util/common/GranularityTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/common/GranularityTest.java
@@ -978,6 +978,9 @@ public void testGranularitiesFinerThanDay()
Granularities.FIFTEEN_MINUTE,
Granularities.TEN_MINUTE,
Granularities.FIVE_MINUTE,
+ Granularities.FOUR_MINUTE,
+ Granularities.THREE_MINUTE,
+ Granularities.TWO_MINUTE,
Granularities.MINUTE,
Granularities.SECOND
),
@@ -995,6 +998,9 @@ public void testGranularitiesFinerThanHour()
Granularities.FIFTEEN_MINUTE,
Granularities.TEN_MINUTE,
Granularities.FIVE_MINUTE,
+ Granularities.FOUR_MINUTE,
+ Granularities.THREE_MINUTE,
+ Granularities.TWO_MINUTE,
Granularities.MINUTE,
Granularities.SECOND
),
@@ -1016,6 +1022,9 @@ public void testGranularitiesFinerThanWeek()
Granularities.FIFTEEN_MINUTE,
Granularities.TEN_MINUTE,
Granularities.FIVE_MINUTE,
+ Granularities.FOUR_MINUTE,
+ Granularities.THREE_MINUTE,
+ Granularities.TWO_MINUTE,
Granularities.MINUTE,
Granularities.SECOND
),
@@ -1040,6 +1049,9 @@ public void testGranularitiesFinerThanAll()
Granularities.FIFTEEN_MINUTE,
Granularities.TEN_MINUTE,
Granularities.FIVE_MINUTE,
+ Granularities.FOUR_MINUTE,
+ Granularities.THREE_MINUTE,
+ Granularities.TWO_MINUTE,
Granularities.MINUTE,
Granularities.SECOND
),
diff --git a/processing/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java b/processing/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
index 4df73564389e..d636e67d6f77 100644
--- a/processing/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/http/client/FriendlyServersTest.java
@@ -22,6 +22,8 @@
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.emitter.core.NoopEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
import org.eclipse.jetty.server.Connector;
@@ -95,7 +97,7 @@ public void run()
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final StatusResponseHolder response = client
.go(
new Request(
@@ -165,7 +167,7 @@ public void run()
new HttpClientProxyConfig("localhost", serverSocket.getLocalPort(), "bob", "sally")
)
.build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final StatusResponseHolder response = client
.go(
new Request(
@@ -232,7 +234,7 @@ public void run()
final HttpClientConfig config = HttpClientConfig.builder()
.withCompressionCodec(HttpClientConfig.CompressionCodec.IDENTITY)
.build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final StatusResponseHolder response = client
.go(
new Request(
@@ -283,12 +285,12 @@ public void testFriendlySelfSignedHttpsServer() throws Exception
try {
final SSLContext mySsl = HttpClientInit.sslContextWithTrustedKeyStore(keyStorePath, "abc123");
final HttpClientConfig trustingConfig = HttpClientConfig.builder().withSslContext(mySsl).build();
- final HttpClient trustingClient = HttpClientInit.createClient(trustingConfig, lifecycle);
+ final HttpClient trustingClient = HttpClientInit.createClient(trustingConfig, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final HttpClientConfig skepticalConfig = HttpClientConfig.builder()
.withSslContext(SSLContext.getDefault())
.build();
- final HttpClient skepticalClient = HttpClientInit.createClient(skepticalConfig, lifecycle);
+ final HttpClient skepticalClient = HttpClientInit.createClient(skepticalConfig, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
// Correct name ("localhost")
{
@@ -364,7 +366,7 @@ public void testHttpBin() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
{
final HttpResponseStatus status = client
diff --git a/processing/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java b/processing/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
index ec54bd13500a..537a684388ed 100644
--- a/processing/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/http/client/JankyServersTest.java
@@ -22,6 +22,8 @@
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.emitter.core.NoopEmitter;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.http.client.response.StatusResponseHandler;
import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
import org.jboss.netty.channel.ChannelException;
@@ -155,7 +157,7 @@ public void testHttpSilentServerWithGlobalTimeout() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withReadTimeout(new Duration(100)).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture future = client
.go(
new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", silentServerSocket.getLocalPort()))),
@@ -183,7 +185,7 @@ public void testHttpSilentServerWithRequestTimeout() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withReadTimeout(new Duration(86400L * 365)).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture future = client
.go(
new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", silentServerSocket.getLocalPort()))),
@@ -215,7 +217,7 @@ public void testHttpsSilentServer() throws Throwable
.withSslContext(SSLContext.getDefault())
.withSslHandshakeTimeout(new Duration(100))
.build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture response = client
.go(
@@ -244,7 +246,7 @@ public void testHttpConnectionClosingServer() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture response = client
.go(
new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", closingServerSocket.getLocalPort()))),
@@ -272,7 +274,7 @@ public void testHttpsConnectionClosingServer() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture response = client
.go(
@@ -302,7 +304,7 @@ public void testHttpConnectionRefused() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
// Need to select a port that isn't being listened to. This approach finds an unused port in a racey way.
// Hopefully it works most of the time.
@@ -338,7 +340,7 @@ public void testHttpsConnectionRefused() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
// Need to select a port that isn't being listened to. This approach finds an unused port in a racey way.
// Hopefully it works most of the time.
@@ -381,7 +383,7 @@ public void testHttpEchoServer() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture response = client
.go(
new Request(HttpMethod.GET, new URL(StringUtils.format("http://localhost:%d/", echoServerSocket.getLocalPort()))),
@@ -404,7 +406,7 @@ public void testHttpsEchoServer() throws Throwable
final Lifecycle lifecycle = new Lifecycle();
try {
final HttpClientConfig config = HttpClientConfig.builder().withSslContext(SSLContext.getDefault()).build();
- final HttpClient client = HttpClientInit.createClient(config, lifecycle);
+ final HttpClient client = HttpClientInit.createClient(config, lifecycle, new ServiceEmitter("", "", new NoopEmitter()));
final ListenableFuture response = client
.go(
diff --git a/processing/src/test/java/org/apache/druid/java/util/http/client/pool/ResourcePoolTest.java b/processing/src/test/java/org/apache/druid/java/util/http/client/pool/ResourcePoolTest.java
index 2961f6504695..57e2516b0831 100644
--- a/processing/src/test/java/org/apache/druid/java/util/http/client/pool/ResourcePoolTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/http/client/pool/ResourcePoolTest.java
@@ -34,7 +34,7 @@
public class ResourcePoolTest
{
ResourceFactory resourceFactory;
- ResourcePool pool;
+ DefaultResourcePoolImpl pool;
@Before
public void setUp()
@@ -52,7 +52,7 @@ public void setUpPool(boolean eagerInitialization)
resourceFactory = (ResourceFactory) EasyMock.createMock(ResourceFactory.class);
EasyMock.replay(resourceFactory);
- pool = new ResourcePool(
+ pool = new DefaultResourcePoolImpl(
resourceFactory,
new ResourcePoolConfig(2, TimeUnit.MINUTES.toMillis(4)),
eagerInitialization
@@ -418,7 +418,7 @@ public void testTimedOutResource() throws Exception
{
resourceFactory = (ResourceFactory