From 24f5e2c68fe6012d55aa4f7514a193c93a2089e8 Mon Sep 17 00:00:00 2001 From: Jamie Chapman-Brown Date: Wed, 8 Mar 2023 00:06:38 -0500 Subject: [PATCH] Add support for fetching from Rabbit MQ Super Streams --- distribution/pom.xml | 2 + .../rabbit-stream-ingestion.md | 239 +++++++ .../rabbit-stream-indexing-service/pom.xml | 172 +++++ ...PublishingRabbitStreamIndexTaskRunner.java | 158 +++++ .../rabbitstream/RabbitSequenceNumber.java | 47 ++ .../RabbitStreamDataSourceMetadata.java | 57 ++ .../rabbitstream/RabbitStreamIndexTask.java | 150 +++++ .../RabbitStreamIndexTaskClientFactory.java | 52 ++ .../RabbitStreamIndexTaskIOConfig.java | 118 ++++ .../RabbitStreamIndexTaskModule.java | 58 ++ .../RabbitStreamIndexTaskTuningConfig.java | 258 ++++++++ .../RabbitStreamRecordSupplier.java | 481 ++++++++++++++ .../rabbitstream/RabbitStreamSamplerSpec.java | 72 +++ .../supervisor/RabbitStreamSupervisor.java | 420 +++++++++++++ .../RabbitStreamSupervisorIOConfig.java | 142 +++++ .../RabbitStreamSupervisorIngestionSpec.java | 65 ++ .../RabbitStreamSupervisorReportPayload.java | 90 +++ .../RabbitStreamSupervisorSpec.java | 167 +++++ .../RabbitStreamSupervisorTuningConfig.java | 248 ++++++++ ...rg.apache.druid.initialization.DruidModule | 16 + .../RabbitStreamIndexTaskIOConfigTest.java | 88 +++ ...RabbitStreamIndexTaskTuningConfigTest.java | 116 ++++ .../RabbitStreamRecordSupplierTest.java | 591 ++++++++++++++++++ .../RabbitStreamSupervisorIOConfigTest.java | 139 ++++ .../RabbitStreamSupervisorTest.java | 368 +++++++++++ ...abbitStreamSupervisorTuningConfigTest.java | 126 ++++ .../kinesis/KinesisIndexTaskTuningConfig.java | 7 - licenses.yaml | 10 + pom.xml | 1 + 29 files changed, 4451 insertions(+), 7 deletions(-) create mode 100644 docs/development/extensions-contrib/rabbit-stream-ingestion.md create mode 100644 extensions-contrib/rabbit-stream-indexing-service/pom.xml create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/IncrementalPublishingRabbitStreamIndexTaskRunner.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitSequenceNumber.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskClientFactory.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskModule.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamSamplerSpec.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIngestionSpec.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorReportPayload.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplierTest.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java create mode 100644 extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java diff --git a/distribution/pom.xml b/distribution/pom.xml index ab8e082c9ab88..f4e89d69eb799 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -450,6 +450,8 @@ org.apache.druid.extensions.contrib:opentelemetry-emitter -c org.apache.druid.extensions:druid-iceberg-extensions + -c + org.apache.druid.extensions.contrib:druid-rabbit-indexing-service diff --git a/docs/development/extensions-contrib/rabbit-stream-ingestion.md b/docs/development/extensions-contrib/rabbit-stream-ingestion.md new file mode 100644 index 0000000000000..7929a65be759b --- /dev/null +++ b/docs/development/extensions-contrib/rabbit-stream-ingestion.md @@ -0,0 +1,239 @@ +--- +id: rabbit-super-stream-injestion +title: "RabbitMQ superstream ingestion" +sidebar_label: "Rabbitmq superstream" +--- + + + +When you enable the rabbit stream indexing service, you can configure *supervisors* on the Overlord to manage the creation and lifetime of rabbit indexing tasks. These indexing tasks read events from a rabbit super-stream. The supervisor oversees the state of the indexing tasks to: + - coordinate handoffs + - manage failures + - ensure that scalability and replication requirements are maintained. + + To use the rabbit stream indexing service, load the `druid-rabbit-indexing-service` community druid extension (see +[Including Extensions](../../configuration/extensions.md#loading-extensions)). + + +## Submitting a Supervisor Spec + +To use the rabbit stream indexing service, load the `druid-rabbit-indexing-service` extension on both the Overlord and the MiddleManagers. Druid starts a supervisor for a dataSource when you submit a supervisor spec. Submit your supervisor spec to the following endpoint: + + +`http://:/druid/indexer/v1/supervisor` + +For example: + +``` +curl -X POST -H 'Content-Type: application/json' -d @supervisor-spec.json http://localhost:8090/druid/indexer/v1/supervisor +``` + +Where the file `supervisor-spec.json` contains a rabbit supervisor spec: + +```json +{ + "type": "rabbit", + "spec": { + "dataSchema": { + "dataSource": "metrics-rabbit", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [ + "timestamp", + "value" + ] + }, + "metricsSpec": [ + { + "name": "count", + "type": "count" + }, + { + "name": "value_sum", + "fieldName": "value", + "type": "doubleSum" + }, + { + "name": "value_min", + "fieldName": "value", + "type": "doubleMin" + }, + { + "name": "value_max", + "fieldName": "value", + "type": "doubleMax" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "HOUR", + "queryGranularity": "NONE" + } + }, + "ioConfig": { + "stream": "metrics", + "inputFormat": { + "type": "json" + }, + "uri": "rabbitmq-stream://localhost:5552", + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H" + }, + "tuningConfig": { + "type": "rabbit", + "maxRowsPerSegment": 5000000 + } + } +} +``` + +## Supervisor Spec + +|Field|Description|Required| +|--------|-----------|---------| +|`type`|The supervisor type; this should always be `rabbit`.|yes| +|`spec`|Container object for the supervisor configuration.|yes| +|`dataSchema`|The schema that will be used by the rabbit indexing task during ingestion. See [`dataSchema`](../../ingestion/ingestion-spec.md#dataschema).|yes| +|`ioConfig`|An [`ioConfig`](#ioconfig) object for configuring rabbit super stream connection and I/O-related settings for the supervisor and indexing task.|yes| +|`tuningConfig`|A [`tuningConfig`](#tuningconfig) object for configuring performance-related settings for the supervisor and indexing tasks.|no| + +### `ioConfig` + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`stream`|String|The Rabbitmq super stream to read.|yes| +|`inputFormat`|Object|[`inputFormat`](../../ingestion/data-formats.md#input-format) to specify how to parse input data. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|yes| +|`uri`|String|The uri to connect to rabbitmq with. |yes | +|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against process failure.|no (default == 1)| +|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. |no (default == 1)| +|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)| +|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)| +|`period`|ISO8601 Period|How often the supervisor will execute its management logic. Note that the supervisor will also run in response to certain events (such as tasks succeeding, failing, and reaching their taskDuration) so this value specifies the maximum time between iterations.|no (default == PT30S)| +|`useEarliestSequenceNumber`|Boolean|If a supervisor is managing a dataSource for the first time, it will obtain a set of starting sequence numbers from rabbitmq. This flag determines whether it retrieves the earliest or latest sequence numbers in the stream. Under normal circumstances, subsequent tasks will start from where the previous segments ended so this flag will only be used on first run.|no (default == false)| +|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If this is set too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT6H)| +|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)| +|`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*. Messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)| +|`Consumer Properties`|Object| a dynamic map used to provide |no (default == none)| + + + +### `tuningConfig` + +The `tuningConfig` is optional. If no `tuningConfig` is specified, default parameters are used. + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`| String|The indexing task type, this should always be `rabbit`.|yes| +|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with `maxRowsInMemory * (2 + maxPendingPersists)`.|no (default == 100000)| +|`maxBytesInMemory`|Long| The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally and user does not need to set it. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|no (default == One-sixth of max JVM memory)| +|`maxRowsPerSegment`|Integer|The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == 5000000)| +|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.|no (default == unlimited)| +|`intermediatePersistPeriod`|ISO8601 Period|The period that determines the rate at which intermediate persists occur.|no (default == PT10M)| +|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with `maxRowsInMemory * (2 + maxPendingPersists)`.|no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up)| +|`indexSpec`|Object|Tune how data is indexed. See [IndexSpec](#indexspec) for more information.|no| +|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. This can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values.| no (default = same as `indexSpec`)| +|`reportParseExceptions`|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)| +|`handoffConditionTimeout`|Long| Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.| no (default == 0)| +|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read rabbitmq messages that are no longer available. Not supported. |no (default == false)| +|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular RabbitMQ stream. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`.|no (default == false)| +|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|no (default == min(10, taskCount))| +|`chatAsync`|Boolean| If true, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If false, use synchronous communication in a thread pool of size `chatThreads`. | no (default == true) | +|`chatThreads`|Integer| The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true` (the default).| no (default == min(10, taskCount * replicas))| +|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive.| no (default == 8)| +|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| +|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| +|`recordBufferSize`|Integer|Size of the buffer (number of events) used between the rabbitmq consumers and the main ingestion thread.|no (see [Determining fetch settings](#determining-fetch-settings) for defaults)| +|`recordBufferOfferTimeout`|Integer|Length of time in milliseconds to wait for space to become available in the buffer before timing out.| no (default == 5000)| | +|`segmentWriteOutMediumFactory`|Object|Segment write-out medium to use when creating segments. See below for more information.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used)| +|`intermediateHandoffPeriod`|ISO8601 Period|How often the tasks should hand off segments. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier.| no (default == P2147483647D)| +|`logParseExceptions`|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no, default == false| +|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|no, unlimited default| +|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid can keep track of the most recent parse exceptions. "maxSavedParseExceptions" limits how many exception instances will be saved. These saved exceptions will be made available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|no, default == 0| +|`maxRecordsPerPoll`|Integer|The maximum number of records/events to be fetched from buffer per poll. The actual maximum will be `Max(maxRecordsPerPoll, Max(bufferSize, 1))`|no (see [Determining fetch settings](#determining-fetch-settings) for defaults)| +|`repartitionTransitionDuration`|ISO8601 Period|When shards are split or merged, the supervisor will recompute shard -> task group mappings, and signal any running tasks created under the old mappings to stop early at (current time + `repartitionTransitionDuration`). Stopping the tasks early allows Druid to begin reading from the new shards more quickly. The repartition transition wait time controlled by this property gives the stream additional time to write records to the new shards after the split/merge, which helps avoid the issues with empty shard handling described at https://github.com/apache/druid/issues/7600.|no, (default == PT2M)| +|`offsetFetchPeriod`|ISO8601 Period|How often the supervisor queries rabbitmq and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value (`PT5S`), the supervisor ignores the value and uses the minimum value instead.|no (default == PT30S, min == PT5S)| + + +#### IndexSpec + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object. See [Bitmap types](#bitmap-types) below for options.|no (defaults to Roaring)| +|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for primitive type metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using sequence number or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| + +##### Bitmap types + +For Roaring bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|Must be `roaring`.|yes| + +For Concise bitmaps: + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|Must be `concise`.|yes| + +#### SegmentWriteOutMediumFactory + +|Field|Type|Description|Required| +|-----|----|-----------|--------| +|`type`|String|See [Additional Peon Configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|yes| + + + +## Operations + +This section describes how some supervisor APIs work in the Rabbit Stream Indexing Service. +For all supervisor APIs, check [Supervisor APIs](../../operations/api-reference.md#supervisors). + +### RabbitMQ Authentication + +To authenticate with rabbitmq securely, you must provide a username and password, as well as configure +a certificate if you aren't using a standard certificate provider. + +In order to configure these, use the dynamic configuration provider of the ioConfig +``` + "ioConfig": { + "type": "rabbit", + "stream": "api-audit", + "uri": "rabbitmq-stream://localhost:5552", + "taskCount": 1, + "replicas": 1, + "taskDuration": "PT1H", + "consumerProperties": { + "druid.dynamic.config.provider" : { + "type": "environment", + "variables": { + "username": "RABBIT_USERNAME", + "password": "RABBIT_PASSWORD" + } + } + } + }, + ``` \ No newline at end of file diff --git a/extensions-contrib/rabbit-stream-indexing-service/pom.xml b/extensions-contrib/rabbit-stream-indexing-service/pom.xml new file mode 100644 index 0000000000000..5ba3a0426ad19 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/pom.xml @@ -0,0 +1,172 @@ + + + + + 4.0.0 + + org.apache.druid.extensions.contrib + druid-rabbit-indexing-service + druid-rabbit-indexing-service + druid-rabbit-indexing-service + + + org.apache.druid + druid + 29.0.0-SNAPSHOT + ../../pom.xml + + + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + io.netty + netty + 3.10.6.Final + provided + + + com.google.code.findbugs + jsr305 + provided + + + commons-io + commons-io + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + joda-time + joda-time + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + javax.ws.rs + jsr311-api + provided + + + org.hamcrest + hamcrest-core + test + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.google.guava + guava + provided + + + javax.validation + validation-api + provided + + + com.rabbitmq + stream-client + 0.9.0 + + + + + junit + junit + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + test-jar + test + + + org.apache.curator + curator-test + test + + + org.easymock + easymock + test + + + org.assertj + assertj-core + test + + + nl.jqno.equalsverifier + equalsverifier + test + + + + + diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/IncrementalPublishingRabbitStreamIndexTaskRunner.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/IncrementalPublishingRabbitStreamIndexTaskRunner.java new file mode 100644 index 0000000000000..edb023ddd3d6d --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/IncrementalPublishingRabbitStreamIndexTaskRunner.java @@ -0,0 +1,158 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SequenceMetadata; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.security.AuthorizerMapper; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +/** + * RabbitStream indexing task runner supporting incremental segments publishing + */ +public class IncrementalPublishingRabbitStreamIndexTaskRunner + extends SeekableStreamIndexTaskRunner +{ + private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingRabbitStreamIndexTaskRunner.class); + private final RabbitStreamIndexTask task; + + IncrementalPublishingRabbitStreamIndexTaskRunner( + RabbitStreamIndexTask task, + @Nullable InputRowParser parser, + AuthorizerMapper authorizerMapper, + LockGranularity lockGranularityToUse) + { + super( + task, + parser, + authorizerMapper, + lockGranularityToUse); + this.task = task; + } + + @Override + protected Long getNextStartOffset(@NotNull Long sequenceNumber) + { + return sequenceNumber + 1; + } + + @Nonnull + @Override + protected List> getRecords( + RecordSupplier recordSupplier, + TaskToolbox toolbox) throws Exception + { + return recordSupplier.poll(task.getIOConfig().getPollTimeout()); + } + + @Override + protected SeekableStreamEndSequenceNumbers deserializePartitionsFromMetadata( + ObjectMapper mapper, + Object object) + { + return mapper.convertValue(object, mapper.getTypeFactory().constructParametrizedType( + SeekableStreamEndSequenceNumbers.class, + SeekableStreamEndSequenceNumbers.class, + String.class, + Long.class)); + } + + @Override + protected SeekableStreamDataSourceMetadata createDataSourceMetadata( + SeekableStreamSequenceNumbers partitions) + { + return new RabbitStreamDataSourceMetadata(partitions); + } + + @Override + protected OrderedSequenceNumber createSequenceNumber(Long sequenceNumber) + { + return RabbitSequenceNumber.of(sequenceNumber); + } + + @Override + protected void possiblyResetDataSourceMetadata( + TaskToolbox toolbox, + RecordSupplier recordSupplier, + Set> assignment) + { + // do nothing + } + + @Override + protected boolean isEndOffsetExclusive() + { + return true; + } + + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + + @Override + public TypeReference>> getSequenceMetadataTypeReference() + { + return new TypeReference>>() { + }; + } + + @Nullable + @Override + protected TreeMap> getCheckPointsFromContext( + TaskToolbox toolbox, + String checkpointsString) throws IOException + { + if (checkpointsString != null) { + log.debug("Got checkpoints from task context[%s].", checkpointsString); + return toolbox.getJsonMapper().readValue( + checkpointsString, + new TypeReference>>() { + }); + } else { + return null; + } + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitSequenceNumber.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitSequenceNumber.java new file mode 100644 index 0000000000000..0f5197d7a55bc --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitSequenceNumber.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.indexing.rabbitstream; + +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; + +import javax.validation.constraints.NotNull; + +// OrderedSequenceNumber.equals() should be used instead. +@SuppressWarnings("ComparableImplementedButEqualsNotOverridden") +public class RabbitSequenceNumber extends OrderedSequenceNumber +{ + private RabbitSequenceNumber(Long sequenceNumber) + { + super(sequenceNumber, false); + } + + public static RabbitSequenceNumber of(Long sequenceNumber) + { + return new RabbitSequenceNumber(sequenceNumber); + } + + @Override + public int compareTo( + @NotNull OrderedSequenceNumber o) + { + return this.get().compareTo(o.get()); + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.java new file mode 100644 index 0000000000000..5cafa1d7925dc --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamDataSourceMetadata.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.rabbitstream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamSequenceNumbers; + +public class RabbitStreamDataSourceMetadata extends SeekableStreamDataSourceMetadata +{ + + @JsonCreator + public RabbitStreamDataSourceMetadata( + @JsonProperty("partitions") SeekableStreamSequenceNumbers partitions) + { + super(partitions); + } + + @Override + public DataSourceMetadata asStartMetadata() + { + final SeekableStreamSequenceNumbers sequenceNumbers = getSeekableStreamSequenceNumbers(); + if (sequenceNumbers instanceof SeekableStreamEndSequenceNumbers) { + return createConcreteDataSourceMetaData( + ((SeekableStreamEndSequenceNumbers) sequenceNumbers).asStartPartitions(true)); + } else { + return this; + } + } + + @Override + protected SeekableStreamDataSourceMetadata createConcreteDataSourceMetaData( + SeekableStreamSequenceNumbers seekableStreamSequenceNumbers) + { + return new RabbitStreamDataSourceMetadata(seekableStreamSequenceNumbers); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java new file mode 100644 index 0000000000000..6e9fab2c60a1c --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTask.java @@ -0,0 +1,150 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.utils.RuntimeInfo; + +import java.util.HashMap; +import java.util.Map; + +public class RabbitStreamIndexTask extends SeekableStreamIndexTask +{ + private static final String TYPE = "index_rabbit"; + private final ObjectMapper configMapper; + // This value can be tuned in some tests + private long pollRetryMs = 30000; + private RuntimeInfo runtimeInfo; + + @JsonCreator + public RabbitStreamIndexTask( + @JsonProperty("id") String id, + @JsonProperty("resource") TaskResource taskResource, + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("tuningConfig") RabbitStreamIndexTaskTuningConfig tuningConfig, + @JsonProperty("ioConfig") RabbitStreamIndexTaskIOConfig ioConfig, + @JsonProperty("context") Map context, + @JacksonInject ObjectMapper configMapper) + { + super( + getOrMakeId(id, dataSchema.getDataSource(), TYPE), + taskResource, + dataSchema, + tuningConfig, + ioConfig, + context, + getFormattedGroupId(dataSchema.getDataSource(), TYPE)); + this.configMapper = configMapper; + + Preconditions.checkArgument( + ioConfig.getStartSequenceNumbers().getExclusivePartitions().isEmpty(), + "All startSequenceNumbers must be inclusive"); + } + + long getPollRetryMs() + { + return pollRetryMs; + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) + { + this.runtimeInfo = toolbox.getAdjustedRuntimeInfo(); + return super.runTask(toolbox); + } + + @Override + protected SeekableStreamIndexTaskRunner createTaskRunner() + { + // noinspection unchecked + return new IncrementalPublishingRabbitStreamIndexTaskRunner( + this, + dataSchema.getParser(), + authorizerMapper, + lockGranularityToUse); + } + + @Override + protected RabbitStreamRecordSupplier newTaskRecordSupplier() + { + + RabbitStreamIndexTaskIOConfig ioConfig = ((RabbitStreamIndexTaskIOConfig) super.ioConfig); + RabbitStreamIndexTaskTuningConfig tuningConfig = ((RabbitStreamIndexTaskTuningConfig) super.tuningConfig); + final Map props = new HashMap<>(ioConfig.getConsumerProperties()); + + final int recordBufferSize = + tuningConfig.getRecordBufferSizeOrDefault(runtimeInfo.getMaxHeapSizeBytes()); + final int maxRecordsPerPoll = tuningConfig.getMaxRecordsPerPollOrDefault(); + + + return new RabbitStreamRecordSupplier( + props, + configMapper, + ioConfig.getUri(), + recordBufferSize, + tuningConfig.getRecordBufferOfferTimeout(), + maxRecordsPerPoll + ); + } + + @Override + @JsonProperty + public RabbitStreamIndexTaskTuningConfig getTuningConfig() + { + return (RabbitStreamIndexTaskTuningConfig) super.getTuningConfig(); + } + + @VisibleForTesting + void setPollRetryMs(long retryMs) + { + this.pollRetryMs = retryMs; + } + + @Override + @JsonProperty("ioConfig") + public RabbitStreamIndexTaskIOConfig getIOConfig() + { + return (RabbitStreamIndexTaskIOConfig) super.getIOConfig(); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean supportsQueries() + { + return true; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskClientFactory.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskClientFactory.java new file mode 100644 index 0000000000000..83cdde8191fff --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskClientFactory.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.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClientFactory; +import org.apache.druid.rpc.ServiceClientFactory; + +@LazySingleton +public class RabbitStreamIndexTaskClientFactory extends SeekableStreamIndexTaskClientFactory +{ + @Inject + public RabbitStreamIndexTaskClientFactory( + @EscalatedGlobal ServiceClientFactory serviceClientFactory, + @Json ObjectMapper mapper) + { + super(serviceClientFactory, mapper); + } + + @Override + public Class getPartitionType() + { + return String.class; + } + + @Override + public Class getSequenceType() + { + return Long.class; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.java new file mode 100644 index 0000000000000..4c053d89455d2 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfig.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.indexing.rabbitstream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Map; + +public class RabbitStreamIndexTaskIOConfig extends SeekableStreamIndexTaskIOConfig +{ + private final long pollTimeout; + private final String uri; + private final Map consumerProperties; + + @JsonCreator + public RabbitStreamIndexTaskIOConfig( + @JsonProperty("taskGroupId") @Nullable Integer taskGroupId, // can be null + // for + // backward + // compabitility + @JsonProperty("baseSequenceName") String baseSequenceName, + @JsonProperty("startSequenceNumbers") SeekableStreamStartSequenceNumbers startSequenceNumbers, + @JsonProperty("endSequenceNumbers") SeekableStreamEndSequenceNumbers endSequenceNumbers, + @JsonProperty("consumerProperties") Map consumerProperties, + @JsonProperty("pollTimeout") Long pollTimeout, + @JsonProperty("useTransaction") Boolean useTransaction, + @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, + @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, + @JsonProperty("uri") String uri) + { + super( + taskGroupId, + baseSequenceName, + startSequenceNumbers, + endSequenceNumbers, + useTransaction, + minimumMessageTime, + maximumMessageTime, + inputFormat); + + this.pollTimeout = pollTimeout != null ? pollTimeout : RabbitStreamSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS; + this.uri = uri; + + this.consumerProperties = consumerProperties; + + final SeekableStreamEndSequenceNumbers myEndSequenceNumbers = getEndSequenceNumbers(); + for (String partition : myEndSequenceNumbers.getPartitionSequenceNumberMap().keySet()) { + Preconditions.checkArgument( + myEndSequenceNumbers.getPartitionSequenceNumberMap() + .get(partition) + .compareTo(getStartSequenceNumbers().getPartitionSequenceNumberMap().get(partition)) >= 0, + "end offset must be >= start offset for partition[%s]", + partition); + } + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @JsonProperty + public long getPollTimeout() + { + return pollTimeout; + } + + @JsonProperty + public String getUri() + { + return this.uri; + } + + @Override + public String toString() + { + return "RabbitStreamIndexTaskIOConfig{" + + "taskGroupId=" + getTaskGroupId() + + ", baseSequenceName='" + getBaseSequenceName() + '\'' + + ", startSequenceNumbers=" + getStartSequenceNumbers() + + ", endSequenceNumbers=" + getEndSequenceNumbers() + + ", consumerProperties=" + consumerProperties + + ", pollTimeout=" + pollTimeout + + ", useTransaction=" + isUseTransaction() + + ", minimumMessageTime=" + getMinimumMessageTime() + + ", maximumMessageTime=" + getMaximumMessageTime() + + ", uri=" + getUri() + + '}'; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskModule.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskModule.java new file mode 100644 index 0000000000000..070e97c9ebd81 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskModule.java @@ -0,0 +1,58 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorSpec; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorTuningConfig; +import org.apache.druid.initialization.DruidModule; + +import java.util.List; + +public class RabbitStreamIndexTaskModule implements DruidModule +{ + + static final String PROPERTY_BASE = "druid.rabbit"; + + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(getClass().getSimpleName()) + .registerSubtypes( + new NamedType(RabbitStreamIndexTask.class, "index_rabbit"), + new NamedType(RabbitStreamDataSourceMetadata.class, "rabbit"), + new NamedType(RabbitStreamIndexTaskIOConfig.class, "rabbit"), + new NamedType(RabbitStreamIndexTaskTuningConfig.class, "RabbitTuningConfig"), + new NamedType(RabbitStreamSupervisorTuningConfig.class, "rabbit"), + new NamedType(RabbitStreamSupervisorSpec.class, "rabbit"), + new NamedType(RabbitStreamSamplerSpec.class, "rabbit"))); + } + + @Override + public void configure(Binder binder) + { + // Nothing to do. + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java new file mode 100644 index 0000000000000..c0c398523bf93 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfig.java @@ -0,0 +1,258 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Ints; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Period; + +import javax.annotation.Nullable; + +import java.io.File; + +public class RabbitStreamIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningConfig +{ + private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; + private static final int DEFAULT_MAX_RECORDS_PER_POLL = 100; + + + static final int ASSUMED_RECORD_SIZE = 10_000; + + /** + * Together with {@link RabbitStreamIndexTaskIOConfig#RECORD_FETCH_MEMORY_MAX_HEAP_FRACTION}, don't take up more + * than 15% of the heap per task. + */ + private static final double RECORD_BUFFER_MEMORY_MAX_HEAP_FRACTION = 0.1; + + /** + * Together with {@link RabbitStreamIndexTaskIOConfig#MAX_RECORD_FETCH_MEMORY}, don't take up more than 200MB per task. + */ + private static final int MAX_RECORD_BUFFER_MEMORY = 100_000_000; + + + private final Integer recordBufferSize; + private final int recordBufferOfferTimeout; + + private final Integer maxRecordsPerPoll; + + public RabbitStreamIndexTaskTuningConfig( + @Nullable AppendableIndexSpec appendableIndexSpec, + @Nullable Integer maxRowsInMemory, + @Nullable Long maxBytesInMemory, + @Nullable Boolean skipBytesInMemoryOverheadCheck, + @Nullable Integer maxRowsPerSegment, + @Nullable Long maxTotalRows, + @Nullable Period intermediatePersistPeriod, + @Nullable File basePersistDirectory, + @Nullable Integer maxPendingPersists, + @Nullable IndexSpec indexSpec, + @Nullable IndexSpec indexSpecForIntermediatePersists, + @Nullable Boolean reportParseExceptions, + @Nullable Long handoffConditionTimeout, + @Nullable Boolean resetOffsetAutomatically, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @Nullable Period intermediateHandoffPeriod, + @Nullable Boolean logParseExceptions, + @Nullable Integer maxParseExceptions, + @Nullable Integer maxSavedParseExceptions, + @Nullable Integer recordBufferSize, + @Nullable Integer recordBufferOfferTimeout, + @Nullable Integer maxRecordsPerPoll) + { + super( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + indexSpecForIntermediatePersists, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + false, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions); + + this.recordBufferSize = recordBufferSize; + this.recordBufferOfferTimeout = recordBufferOfferTimeout == null + ? DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT + : recordBufferOfferTimeout; + this.maxRecordsPerPoll = maxRecordsPerPoll; + } + + @JsonCreator + private RabbitStreamIndexTaskTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, + @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") @Nullable Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, + @Deprecated @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, + @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, + @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("intermediateHandoffPeriod") @Nullable Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll + ) + { + this( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + null, + maxPendingPersists, + indexSpec, + indexSpecForIntermediatePersists, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + recordBufferSize, + recordBufferOfferTimeout, + maxRecordsPerPoll); + } + + @Nullable + @JsonProperty("recordBufferSize") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getRecordBufferSizeConfigured() + { + return recordBufferSize; + } + + @JsonProperty + public int getRecordBufferOfferTimeout() + { + return recordBufferOfferTimeout; + } + + @Nullable + @JsonProperty("maxRecordsPerPoll") + @JsonInclude(JsonInclude.Include.NON_NULL) + public Integer getMaxRecordsPerPollConfigured() + { + return maxRecordsPerPoll; + } + + public int getRecordBufferSizeOrDefault(final long maxHeapSize) + { + if (recordBufferSize != null) { + return recordBufferSize; + } else { + final long memoryToUse = Math.min( + MAX_RECORD_BUFFER_MEMORY, + (long) (maxHeapSize * RECORD_BUFFER_MEMORY_MAX_HEAP_FRACTION) + ); + + return Ints.checkedCast(Math.max(1, memoryToUse / ASSUMED_RECORD_SIZE)); + } + } + + public int getMaxRecordsPerPollOrDefault() + { + return (this.maxRecordsPerPoll != null) ? this.maxRecordsPerPoll : DEFAULT_MAX_RECORDS_PER_POLL; + } + + @Override + public RabbitStreamIndexTaskTuningConfig withBasePersistDirectory(File dir) + { + return new RabbitStreamIndexTaskTuningConfig( + getAppendableIndexSpec(), + getMaxRowsInMemory(), + getMaxBytesInMemory(), + isSkipBytesInMemoryOverheadCheck(), + getMaxRowsPerSegment(), + getMaxTotalRows(), + getIntermediatePersistPeriod(), + dir, + getMaxPendingPersists(), + getIndexSpec(), + getIndexSpecForIntermediatePersists(), + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + getSegmentWriteOutMediumFactory(), + getIntermediateHandoffPeriod(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions(), + getRecordBufferSizeConfigured(), + getRecordBufferOfferTimeout(), + getMaxRecordsPerPollConfigured() + ); + } + + @Override + public String toString() + { + return "RabbitStreamIndexTaskTuningConfig{" + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + + ", maxBytesInMemory=" + getMaxBytesInMemory() + + ", skipBytesInMemoryOverheadCheck=" + isSkipBytesInMemoryOverheadCheck() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", indexSpecForIntermediatePersists=" + getIndexSpecForIntermediatePersists() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + '}'; + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java new file mode 100644 index 0000000000000..956fa540e6ee3 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplier.java @@ -0,0 +1,481 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Queues; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.Message; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.DynamicConfigProvider; + +import javax.annotation.Nonnull; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class RabbitStreamRecordSupplier implements RecordSupplier, MessageHandler +{ + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamRecordSupplier.class); + private Environment env; + private Map streamBuilders; + private boolean closed; + private BlockingQueue> queue; + private String superStream; + private String uri; + private ObjectMapper mapper; + + private final int recordBufferOfferTimeout; + private final int maxRecordsPerPoll; + private final int recordBufferSize; + + public final Map offsetMap; + + List consumers; + private boolean isRunning; + private Semaphore stateSemaphore; + + private String password; + private String username; + + public RabbitStreamRecordSupplier( + Map consumerProperties, + ObjectMapper mapper, + String uri, + int recordBufferSize, + int recordBufferOfferTimeout, + int maxRecordsPerPoll + ) + { + this.uri = uri; + this.mapper = mapper; + + this.recordBufferSize = recordBufferSize; + this.maxRecordsPerPoll = maxRecordsPerPoll; + + this.recordBufferOfferTimeout = recordBufferOfferTimeout; + + // Messages will be added to this queue from multiple threads + queue = new LinkedBlockingQueue<>(recordBufferSize); + + offsetMap = new ConcurrentHashMap<>(); + streamBuilders = new ConcurrentHashMap<>(); + + // stateSemaphore protects isRunning and consumers + stateSemaphore = new Semaphore(1, true); + isRunning = false; + consumers = new ArrayList(); + + this.password = null; + this.username = null; + this.env = null; + + if (consumerProperties != null) { + + // Additional DynamicConfigProvider based extensible support for all consumer + // properties + Object dynamicConfigProviderJson = consumerProperties + .get(RabbitStreamSupervisorIOConfig.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY); + if (dynamicConfigProviderJson != null) { + DynamicConfigProvider dynamicConfigProvider = this.mapper.convertValue(dynamicConfigProviderJson, + DynamicConfigProvider.class); + Map dynamicConfig = dynamicConfigProvider.getConfig(); + for (Map.Entry e : dynamicConfig.entrySet()) { + if (e.getKey().equals("password")) { + this.password = e.getValue(); + } + if (e.getKey().equals("username")) { + this.username = e.getValue(); + } + } + } + } + } + + public void startBackgroundFetch() + { + try { + // aquire uninteruptibly to prevent state corruption issues + // on consumers and isRunning + stateSemaphore.acquireUninterruptibly(); + if (this.isRunning != false) { + return; + } + for (Map.Entry entry : streamBuilders.entrySet()) { + consumers.add( + entry.getValue().offset( + offsetMap.get(entry.getKey())).build()); + } + this.isRunning = true; + } + finally { + stateSemaphore.release(); + } + } + + @VisibleForTesting + public int bufferSize() + { + return queue.size(); + } + + @VisibleForTesting + public boolean isRunning() + { + return this.isRunning; + } + + @VisibleForTesting + public OffsetSpecification getOffset(StreamPartition partition) + { + return this.offsetMap.get(partition.getPartitionId()); + } + + public void stopBackgroundFetch() + { + try { + stateSemaphore.acquire(); + try { + if (this.isRunning != true) { + return; + } + for (Consumer consumer : consumers) { + consumer.close(); + } + this.consumers.clear(); + this.isRunning = false; + } + finally { + stateSemaphore.release(); + } + } + catch (InterruptedException exc) { + } + + } + + public EnvironmentBuilder getEnvBuilder() + { + return Environment.builder(); + } + + public Environment getRabbitEnvironment() + { + if (this.env != null) { + return this.env; + } + + EnvironmentBuilder envBuilder = this.getEnvBuilder().uri(this.uri); + + if (this.password != null) { + envBuilder = envBuilder.password(this.password); + } + if (this.username != null) { + envBuilder = envBuilder.username(this.username); + } + + this.env = envBuilder.build(); + return this.env; + } + + public static String getStreamFromSubstream(String partionID) + { + String[] res = partionID.split("-"); + res = Arrays.copyOf(res, res.length - 1); + return String.join("-", res); + } + + private void removeOldAssignments(Set> streamPartitionstoKeep) + { + Iterator> streamBuilderIterator = streamBuilders.entrySet().iterator(); + while (streamBuilderIterator.hasNext()) { + Map.Entry entry = streamBuilderIterator.next(); + StreamPartition comparitor = new StreamPartition(getStreamFromSubstream(entry.getKey()), entry.getKey()); + if (!streamPartitionstoKeep.contains(comparitor)) { + streamBuilderIterator.remove(); + } + } + + Iterator> offsetItterator = offsetMap.entrySet().iterator(); + while (offsetItterator.hasNext()) { + Map.Entry entry = offsetItterator.next(); + StreamPartition comparitor = new StreamPartition(getStreamFromSubstream(entry.getKey()), entry.getKey()); + if (!streamPartitionstoKeep.contains(comparitor)) { + offsetItterator.remove(); + } + } + } + + @Override + public void assign(Set> streamPartitions) + { + this.stopBackgroundFetch(); + + for (StreamPartition part : streamPartitions) { + ConsumerBuilder builder = this.getRabbitEnvironment().consumerBuilder().noTrackingStrategy(); + builder = builder.stream(part.getPartitionId()); + builder = builder.messageHandler(this); + streamBuilders.put(part.getPartitionId(), builder); + this.superStream = part.getStream(); + } + + removeOldAssignments(streamPartitions); + + + } + + private void filterBufferAndResetBackgroundFetch(Set> partitions) + { + this.stopBackgroundFetch(); + // filter records in buffer and only retain ones whose partition was not seeked + BlockingQueue> newQ = new LinkedBlockingQueue<>( + recordBufferSize); + + queue.stream() + .filter(x -> !streamBuilders.containsKey(x.getPartitionId())) + .forEachOrdered(newQ::offer); + + queue = newQ; + } + + @Override + public void seek(StreamPartition partition, Long sequenceNumber) throws InterruptedException + { + filterBufferAndResetBackgroundFetch(ImmutableSet.of(partition)); + offsetMap.put(partition.getPartitionId(), OffsetSpecification.offset(sequenceNumber)); + } + + @Override + public void seekToEarliest(Set> partitions) throws InterruptedException + { + filterBufferAndResetBackgroundFetch(partitions); + for (StreamPartition part : partitions) { + offsetMap.put(part.getPartitionId(), OffsetSpecification.first()); + } + } + + @Override + public void seekToLatest(Set> partitions) throws InterruptedException + { + filterBufferAndResetBackgroundFetch(partitions); + for (StreamPartition part : partitions) { + offsetMap.put(part.getPartitionId(), OffsetSpecification.last()); + } + } + + @Override + public Set> getAssignment() + { + return streamBuilders.keySet().stream().map(e -> StreamPartition.of(this.superStream, e)) + .collect(Collectors.toSet()); + } + + /** + * handle can be called from any consumer, so it must be + * as short as possible and thread safe + */ + @Override + public void handle(MessageHandler.Context context, Message message) + { + + OrderedPartitionableRecord currRecord; + currRecord = new OrderedPartitionableRecord<>( + this.superStream, + context.stream(), + context.offset(), + ImmutableList.of(new ByteEntity(message.getBodyAsBinary()))); + + try { + if (!queue.offer( + currRecord, + this.recordBufferOfferTimeout, + TimeUnit.MILLISECONDS)) { + log.warn("Message not accepted, message buffer full"); + stopBackgroundFetch(); + } else { + this.offsetMap.put(context.stream(), OffsetSpecification.offset(context.offset() + 1)); + } + } + catch (InterruptedException e) { + // may happen if interrupted while BlockingQueue.offer() is waiting + log.warn( + e, + "Interrupted while waiting to add record to buffer"); + stopBackgroundFetch(); + } + + } + + /** + * optionalStartBackgroundFetch ensures that a background fetch is running + * if this.queue is running low on records. We want to minimize thrashing + * around starting/stopping the consumers. + */ + public void optionalStartBackgroundFetch() + { + if (this.queue.size() < Math.min(this.maxRecordsPerPoll * 2, recordBufferSize / 2)) { + this.startBackgroundFetch(); + } + } + + @Nonnull + @Override + public List> poll(long timeout) + { + this.optionalStartBackgroundFetch(); + + try { + int expectedSize = Math.min(Math.max(queue.size(), 1), maxRecordsPerPoll); + + List> polledRecords = new ArrayList<>(expectedSize); + + Queues.drain( + queue, + polledRecords, + expectedSize, + timeout, + TimeUnit.MILLISECONDS); + + polledRecords = polledRecords.stream() + .filter(x -> streamBuilders.containsKey(x.getPartitionId())) + .collect(Collectors.toList()); + + return polledRecords; + } + catch (InterruptedException e) { + log.warn(e, "Interrupted while polling"); + + return Collections.emptyList(); + } + } + + @Override + public Long getEarliestSequenceNumber(StreamPartition partition) + { + return this.getRabbitEnvironment().queryStreamStats(partition.getPartitionId()).firstOffset(); + } + + @Override + public Long getLatestSequenceNumber(StreamPartition partition) + { + return this.getRabbitEnvironment().queryStreamStats(partition.getPartitionId()).committedChunkId(); + } + + @Override + public boolean isOffsetAvailable(StreamPartition partition, OrderedSequenceNumber offset) + { + final Long earliestOffset = getEarliestSequenceNumber(partition); + return earliestOffset != null + && offset.isAvailableWithEarliest(RabbitSequenceNumber.of(earliestOffset)); + } + + @Override + public Long getPosition(StreamPartition partition) + { + throw new UnsupportedOperationException("getPosition() is not supported in RabbitMQ streams"); + } + + + public ClientParameters getParameters() + { + return new ClientParameters(); + } + + public Client getClient(ClientParameters parameters) + { + return new Client(parameters); + } + + @Override + public Set getPartitionIds(String stream) + { + ClientParameters parameters = getParameters(); + + try { + URI parsedURI = new URI(uri); + parameters.host(parsedURI.getHost()); + + if (parsedURI.getPort() != -1) { + parameters.port(parsedURI.getPort()); + } + + if (this.password != null) { + parameters.password(password); + } + + if (this.username != null) { + parameters.username(username); + } + + Client client = getClient(parameters); + + List partitions = client.partitions(stream); + client.close(); + return new HashSet<>(partitions); + } + catch (URISyntaxException e) { + throw new IllegalArgumentException("error on uri" + uri); + } + catch (Exception e) { + throw e; + } + } + + @Override + public void close() + { + if (closed) { + return; + } + this.stopBackgroundFetch(); + closed = true; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamSamplerSpec.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamSamplerSpec.java new file mode 100644 index 0000000000000..a27e6cbc163a9 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/RabbitStreamSamplerSpec.java @@ -0,0 +1,72 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; +import org.apache.druid.indexing.overlord.sampler.SamplerConfig; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorIOConfig; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorSpec; +import org.apache.druid.indexing.rabbitstream.supervisor.RabbitStreamSupervisorTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamSamplerSpec; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; + +public class RabbitStreamSamplerSpec extends SeekableStreamSamplerSpec +{ + private final ObjectMapper objectMapper; + + @JsonCreator + public RabbitStreamSamplerSpec( + @JsonProperty("spec") final RabbitStreamSupervisorSpec ingestionSpec, + @JsonProperty("samplerConfig") @Nullable final SamplerConfig samplerConfig, + @JacksonInject InputSourceSampler inputSourceSampler, + @JacksonInject ObjectMapper objectMapper) + { + super(ingestionSpec, samplerConfig, inputSourceSampler); + this.objectMapper = objectMapper; + } + + @Override + protected RabbitStreamRecordSupplier createRecordSupplier() + { + RabbitStreamSupervisorIOConfig ioConfig = (RabbitStreamSupervisorIOConfig) RabbitStreamSamplerSpec.this.ioConfig; + final Map props = new HashMap<>(((RabbitStreamSupervisorIOConfig) ioConfig).getConsumerProperties()); + + RabbitStreamSupervisorTuningConfig tuningConfig = ((RabbitStreamSupervisorTuningConfig) RabbitStreamSamplerSpec.this.tuningConfig); + + + return new RabbitStreamRecordSupplier( + props, + objectMapper, + ioConfig.getUri(), + tuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory()), + tuningConfig.getRecordBufferOfferTimeout(), + tuningConfig.getMaxRecordsPerPollOrDefault() + ); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java new file mode 100644 index 0000000000000..6d63acb04c9d7 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisor.java @@ -0,0 +1,420 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.overlord.DataSourceMetadata; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats; +import org.apache.druid.indexing.rabbitstream.RabbitSequenceNumber; +import org.apache.druid.indexing.rabbitstream.RabbitStreamDataSourceMetadata; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTask; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskClientFactory; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskIOConfig; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.rabbitstream.RabbitStreamRecordSupplier; +import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTask; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; +import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamException; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; +import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; + +/** + * Supervisor responsible for managing the RabbitStreamIndexTasks for a single + * dataSource. At a high level, the class accepts a + * {@link RabbitStreamSupervisorSpec} which includes the rabbit super stream and + * configuration as well as an ingestion spec which will be used to generate the + * indexing tasks. The run loop periodically refreshes its view of the super stream's + * partitions and the list of running indexing tasks and ensures that + * all partitions are being read from and that there are enough tasks to satisfy + * the desired number of replicas. As tasks complete, new tasks are queued to + * process the next range of rabbit stream offsets. + */ +public class RabbitStreamSupervisor extends SeekableStreamSupervisor +{ + public static final TypeReference>> CHECKPOINTS_TYPE_REF = new TypeReference>>() { + }; + + private static final EmittingLogger log = new EmittingLogger(RabbitStreamSupervisor.class); + private static final Long NOT_SET = -1L; + private static final Long END_OF_PARTITION = Long.MAX_VALUE; + + private final ServiceEmitter emitter; + private final DruidMonitorSchedulerConfig monitorSchedulerConfig; + private volatile Map latestSequenceFromStream; + + private final RabbitStreamSupervisorSpec spec; + + public RabbitStreamSupervisor( + final TaskStorage taskStorage, + final TaskMaster taskMaster, + final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + final RabbitStreamIndexTaskClientFactory taskClientFactory, + final ObjectMapper mapper, + final RabbitStreamSupervisorSpec spec, + final RowIngestionMetersFactory rowIngestionMetersFactory) + { + super( + StringUtils.format("RabbitSupervisor-%s", spec.getDataSchema().getDataSource()), + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + taskClientFactory, + mapper, + spec, + rowIngestionMetersFactory, + false); + + this.spec = spec; + this.emitter = spec.getEmitter(); + this.monitorSchedulerConfig = spec.getMonitorSchedulerConfig(); + } + + @Override + protected RecordSupplier setupRecordSupplier() + { + RabbitStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); + RabbitStreamIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig(); + + return new RabbitStreamRecordSupplier( + spec.getIoConfig().getConsumerProperties(), + sortingMapper, + spec.getIoConfig().getUri(), + taskTuningConfig.getRecordBufferSizeOrDefault(Runtime.getRuntime().maxMemory()), + taskTuningConfig.getRecordBufferOfferTimeout(), + taskTuningConfig.getMaxRecordsPerPollOrDefault() + ); + } + + @Override + protected int getTaskGroupIdForPartition(String partitionId) + { + return partitionId.hashCode() % spec.getIoConfig().getTaskCount(); + } + + @Override + protected boolean checkSourceMetadataMatch(DataSourceMetadata metadata) + { + return metadata instanceof RabbitStreamDataSourceMetadata; + } + + @Override + protected boolean doesTaskTypeMatchSupervisor(Task task) + { + return task instanceof RabbitStreamIndexTask; + } + + @Override + protected SeekableStreamSupervisorReportPayload createReportPayload( + int numPartitions, + boolean includeOffsets) + { + RabbitStreamSupervisorIOConfig ioConfig = spec.getIoConfig(); + Map partitionLag = getRecordLagPerPartitionInLatestSequences(getHighestCurrentOffsets()); + return new RabbitStreamSupervisorReportPayload( + spec.getDataSchema().getDataSource(), + ioConfig.getStream(), + numPartitions, + ioConfig.getReplicas(), + ioConfig.getTaskDuration().getMillis() / 1000, + includeOffsets ? latestSequenceFromStream : null, + includeOffsets ? partitionLag : null, + includeOffsets ? partitionLag.values().stream().mapToLong(x -> Math.max(x, 0)).sum() : null, + includeOffsets ? sequenceLastUpdated : null, + spec.isSuspended(), + stateManager.isHealthy(), + stateManager.getSupervisorState().getBasicState(), + stateManager.getSupervisorState(), + stateManager.getExceptionEvents()); + } + + @Override + protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( + int groupId, + Map startPartitions, + Map endPartitions, + String baseSequenceName, + DateTime minimumMessageTime, + DateTime maximumMessageTime, + Set exclusiveStartSequenceNumberPartitions, + SeekableStreamSupervisorIOConfig ioConfig) + { + RabbitStreamSupervisorIOConfig rabbitConfig = (RabbitStreamSupervisorIOConfig) ioConfig; + return new RabbitStreamIndexTaskIOConfig( + groupId, + baseSequenceName, + new SeekableStreamStartSequenceNumbers<>(ioConfig.getStream(), startPartitions, Collections.emptySet()), + new SeekableStreamEndSequenceNumbers<>(ioConfig.getStream(), endPartitions), + rabbitConfig.getConsumerProperties(), + rabbitConfig.getPollTimeout(), + true, + minimumMessageTime, + maximumMessageTime, + ioConfig.getInputFormat(), + rabbitConfig.getUri()); + } + + @Override + protected List> createIndexTasks( + int replicas, + String baseSequenceName, + ObjectMapper sortingMapper, + TreeMap> sequenceOffsets, + SeekableStreamIndexTaskIOConfig taskIoConfig, + SeekableStreamIndexTaskTuningConfig taskTuningConfig, + RowIngestionMetersFactory rowIngestionMetersFactory) throws JsonProcessingException + { + final String checkpoints = sortingMapper.writerFor(CHECKPOINTS_TYPE_REF).writeValueAsString(sequenceOffsets); + final Map context = createBaseTaskContexts(); + + List> taskList = new ArrayList<>(); + for (int i = 0; i < replicas; i++) { + String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName); + taskList.add(new RabbitStreamIndexTask( + taskId, + new TaskResource(baseSequenceName, 1), + spec.getDataSchema(), + (RabbitStreamIndexTaskTuningConfig) taskTuningConfig, + (RabbitStreamIndexTaskIOConfig) taskIoConfig, + context, + sortingMapper)); + } + return taskList; + } + + @Override + protected Map getPartitionRecordLag() + { + Map highestCurrentOffsets = getHighestCurrentOffsets(); + + if (latestSequenceFromStream == null) { + return null; + } + + if (!latestSequenceFromStream.keySet().equals(highestCurrentOffsets.keySet())) { + log.warn( + "Lag metric: rabbit partitions %s do not match task partitions %s", + latestSequenceFromStream.keySet(), + highestCurrentOffsets.keySet()); + } + + return getRecordLagPerPartitionInLatestSequences(highestCurrentOffsets); + } + + @Nullable + @Override + protected Map getPartitionTimeLag() + { + // time lag not currently support with rabbit + return null; + } + + // suppress use of CollectionUtils.mapValues() since the valueMapper function + // is + // dependent on map key here + @SuppressWarnings("SSBasedInspection") + // Used while calculating cummulative lag for entire stream + private Map getRecordLagPerPartitionInLatestSequences(Map currentOffsets) + { + if (latestSequenceFromStream == null) { + return Collections.emptyMap(); + } + + return latestSequenceFromStream + .entrySet() + .stream() + .collect( + Collectors.toMap( + Entry::getKey, + e -> e.getValue() != null + ? e.getValue() + 1 - Optional.ofNullable(currentOffsets.get(e.getKey())).orElse(0L) + : 0)); + } + + @Override + // suppress use of CollectionUtils.mapValues() since the valueMapper function + // is + // dependent on map key here + @SuppressWarnings("SSBasedInspection") + // Used while generating Supervisor lag reports per task + protected Map getRecordLagPerPartition(Map currentOffsets) + { + if (latestSequenceFromStream == null || currentOffsets == null) { + return Collections.emptyMap(); + } + + return currentOffsets + .entrySet() + .stream() + .filter(e -> latestSequenceFromStream.get(e.getKey()) != null) + .collect( + Collectors.toMap( + Entry::getKey, + e -> e.getValue() != null + ? latestSequenceFromStream.get(e.getKey()) + 1 - e.getValue() + : 0)); + } + + @Override + protected Map getTimeLagPerPartition(Map currentOffsets) + { + return null; + } + + @Override + protected RabbitStreamDataSourceMetadata createDataSourceMetaDataForReset(String topic, Map map) + { + return new RabbitStreamDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, map)); + } + + @Override + protected OrderedSequenceNumber makeSequenceNumber(Long seq, boolean isExclusive) + { + return RabbitSequenceNumber.of(seq); + } + + @Override + protected Long getNotSetMarker() + { + return NOT_SET; + } + + @Override + protected Long getEndOfPartitionMarker() + { + return END_OF_PARTITION; + } + + @Override + protected boolean isEndOfShard(Long seqNum) + { + return false; + } + + @Override + protected boolean isShardExpirationMarker(Long seqNum) + { + return false; + } + + @Override + protected boolean useExclusiveStartSequenceNumberForNonFirstSequence() + { + return false; + } + + @Override + public LagStats computeLagStats() + { + Map partitionRecordLag = getPartitionRecordLag(); + if (partitionRecordLag == null) { + return new LagStats(0, 0, 0); + } + + return computeLags(partitionRecordLag); + } + + @Override + protected void updatePartitionLagFromStream() + { + getRecordSupplierLock().lock(); + + Set partitionIds; + try { + partitionIds = recordSupplier.getPartitionIds(getIoConfig().getStream()); + } + catch (Exception e) { + log.warn("Could not fetch partitions for topic/stream [%s]", getIoConfig().getStream()); + getRecordSupplierLock().unlock(); + throw new StreamException(e); + } + + Set> partitions = partitionIds + .stream() + .map(e -> new StreamPartition<>(getIoConfig().getStream(), e)) + .collect(Collectors.toSet()); + + latestSequenceFromStream = partitions.stream() + .collect(Collectors.toMap(StreamPartition::getPartitionId, recordSupplier::getLatestSequenceNumber)); + + getRecordSupplierLock().unlock(); + + } + + @Override + protected Map getLatestSequencesFromStream() + { + return latestSequenceFromStream != null ? latestSequenceFromStream : new HashMap<>(); + } + + @Override + protected String baseTaskName() + { + return "index_rabbit"; + } + + @Override + @VisibleForTesting + public RabbitStreamSupervisorIOConfig getIoConfig() + { + return spec.getIoConfig(); + } + + @VisibleForTesting + public RabbitStreamSupervisorTuningConfig getTuningConfig() + { + return spec.getTuningConfig(); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java new file mode 100644 index 0000000000000..bce4164c5d2aa --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfig.java @@ -0,0 +1,142 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; +import org.joda.time.DateTime; +import org.joda.time.Period; + +import javax.annotation.Nullable; + +import java.util.Map; + +public class RabbitStreamSupervisorIOConfig extends SeekableStreamSupervisorIOConfig +{ + public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider"; + + public static final String USERNAME_KEY = "username"; + public static final String PASSWORD_KEY = "password"; + + private final Map consumerProperties; + + public static final long DEFAULT_POLL_TIMEOUT_MILLIS = 100; + + private final String uri; + private final long pollTimeout; + + @JsonCreator + public RabbitStreamSupervisorIOConfig( + @JsonProperty("stream") String stream, + @JsonProperty("uri") String uri, + @JsonProperty("inputFormat") InputFormat inputFormat, + @JsonProperty("replicas") Integer replicas, + @JsonProperty("taskCount") Integer taskCount, + @JsonProperty("taskDuration") Period taskDuration, + @JsonProperty("consumerProperties") Map consumerProperties, + @Nullable @JsonProperty("autoScalerConfig") AutoScalerConfig autoScalerConfig, + @JsonProperty("pollTimeout") Long pollTimeout, + @JsonProperty("startDelay") Period startDelay, + @JsonProperty("period") Period period, + @JsonProperty("completionTimeout") Period completionTimeout, + @JsonProperty("useEarliestOffset") Boolean useEarliestOffset, + @JsonProperty("lateMessageRejectionPeriod") Period lateMessageRejectionPeriod, + @JsonProperty("earlyMessageRejectionPeriod") Period earlyMessageRejectionPeriod, + @JsonProperty("lateMessageRejectionStartDateTime") DateTime lateMessageRejectionStartDateTime, + @JsonProperty("stopTaskCount") Integer stopTaskCount + ) + { + super( + Preconditions.checkNotNull(stream, "stream"), + inputFormat, + replicas, + taskCount, + taskDuration, + startDelay, + period, + useEarliestOffset, + completionTimeout, + lateMessageRejectionPeriod, + earlyMessageRejectionPeriod, + autoScalerConfig, + lateMessageRejectionStartDateTime, + new IdleConfig(null, null), + stopTaskCount + ); + + this.consumerProperties = consumerProperties; + Preconditions.checkNotNull(uri, "uri"); + this.uri = uri; + + this.pollTimeout = pollTimeout != null ? pollTimeout : DEFAULT_POLL_TIMEOUT_MILLIS; + } + + @JsonProperty + public String getUri() + { + return this.uri; + } + + @JsonProperty + public Map getConsumerProperties() + { + return consumerProperties; + } + + @JsonProperty + public long getPollTimeout() + { + return pollTimeout; + } + + @JsonProperty + public boolean isUseEarliestOffset() + { + return isUseEarliestSequenceNumber(); + } + + @Override + public String toString() + { + return "RabbitStreamSupervisorIOConfig{" + + "stream='" + getStream() + '\'' + + ", replicas=" + getReplicas() + + ", uri=" + getUri() + + ", taskCount=" + getTaskCount() + + ", taskDuration=" + getTaskDuration() + + ", autoScalerConfig=" + getAutoScalerConfig() + + ", pollTimeout=" + pollTimeout + + ", startDelay=" + getStartDelay() + + ", period=" + getPeriod() + + ", useEarliestOffset=" + isUseEarliestOffset() + + ", completionTimeout=" + getCompletionTimeout() + + ", earlyMessageRejectionPeriod=" + getEarlyMessageRejectionPeriod() + + ", lateMessageRejectionPeriod=" + getLateMessageRejectionPeriod() + + ", lateMessageRejectionStartDateTime=" + getLateMessageRejectionStartDateTime() + + ", idleConfig=" + getIdleConfig() + + '}'; + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIngestionSpec.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIngestionSpec.java new file mode 100644 index 0000000000000..7fd48b41f8409 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIngestionSpec.java @@ -0,0 +1,65 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIngestionSpec; +import org.apache.druid.segment.indexing.DataSchema; + +public class RabbitStreamSupervisorIngestionSpec extends SeekableStreamSupervisorIngestionSpec +{ + private final DataSchema dataSchema; + private final RabbitStreamSupervisorIOConfig ioConfig; + private final RabbitStreamSupervisorTuningConfig tuningConfig; + + @JsonCreator + public RabbitStreamSupervisorIngestionSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("ioConfig") RabbitStreamSupervisorIOConfig ioConfig, + @JsonProperty("tuningConfig") RabbitStreamSupervisorTuningConfig tuningConfig) + { + super(dataSchema, ioConfig, tuningConfig); + this.dataSchema = dataSchema; + this.ioConfig = ioConfig; + this.tuningConfig = tuningConfig == null ? RabbitStreamSupervisorTuningConfig.defaultConfig() : tuningConfig; + } + + @Override + @JsonProperty("dataSchema") + public DataSchema getDataSchema() + { + return dataSchema; + } + + @Override + @JsonProperty("ioConfig") + public RabbitStreamSupervisorIOConfig getIOConfig() + { + return ioConfig; + } + + @Override + @JsonProperty("tuningConfig") + public RabbitStreamSupervisorTuningConfig getTuningConfig() + { + return tuningConfig; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorReportPayload.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorReportPayload.java new file mode 100644 index 0000000000000..510ebe3ccc867 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorReportPayload.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.indexing.rabbitstream.supervisor; + +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; + +public class RabbitStreamSupervisorReportPayload extends SeekableStreamSupervisorReportPayload +{ + public RabbitStreamSupervisorReportPayload( + String dataSource, + String stream, + int partitions, + int replicas, + long durationSeconds, + @Nullable Map latestOffsets, + @Nullable Map minimumLag, + @Nullable Long aggregateLag, + @Nullable DateTime offsetsLastUpdated, + boolean suspended, + boolean healthy, + SupervisorStateManager.State state, + SupervisorStateManager.State detailedState, + List recentErrors) + { + super( + dataSource, + stream, + partitions, + replicas, + durationSeconds, + latestOffsets, + minimumLag, + aggregateLag, + null, + null, + offsetsLastUpdated, + suspended, + healthy, + state, + detailedState, + recentErrors); + } + + @Override + public String toString() + { + return "RabbitStreamSupervisorReportPayload{" + + "dataSource='" + getDataSource() + '\'' + + ", stream='" + getStream() + '\'' + + ", partitions=" + getPartitions() + + ", replicas=" + getReplicas() + + ", durationSeconds=" + getDurationSeconds() + + ", active=" + getActiveTasks() + + ", publishing=" + getPublishingTasks() + + (getLatestOffsets() != null ? ", latestOffsets=" + getLatestOffsets() : "") + + (getMinimumLag() != null ? ", minimumLag=" + getMinimumLag() : "") + + (getAggregateLag() != null ? ", aggregateLag=" + getAggregateLag() : "") + + (getOffsetsLastUpdated() != null ? ", sequenceLastUpdated=" + getOffsetsLastUpdated() : "") + + ", suspended=" + isSuspended() + + ", healthy=" + isHealthy() + + ", state=" + getState() + + ", detailedState=" + getDetailedState() + + ", recentErrors=" + getRecentErrors() + + '}'; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java new file mode 100644 index 0000000000000..e30f98876a41b --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorSpec.java @@ -0,0 +1,167 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.Supervisor; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskClientFactory; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; +import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.indexing.DataSchema; + +import javax.annotation.Nullable; + +import java.util.Map; + +public class RabbitStreamSupervisorSpec extends SeekableStreamSupervisorSpec +{ + private static final String TASK_TYPE = "rabbit"; + + @JsonCreator + public RabbitStreamSupervisorSpec( + @JsonProperty("spec") @Nullable RabbitStreamSupervisorIngestionSpec ingestionSchema, + @JsonProperty("dataSchema") @Nullable DataSchema dataSchema, + @JsonProperty("tuningConfig") @Nullable RabbitStreamSupervisorTuningConfig tuningConfig, + @JsonProperty("ioConfig") @Nullable RabbitStreamSupervisorIOConfig ioConfig, + @JsonProperty("context") Map context, + @JsonProperty("suspended") Boolean suspended, + @JacksonInject TaskStorage taskStorage, + @JacksonInject TaskMaster taskMaster, + @JacksonInject IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator, + @JacksonInject RabbitStreamIndexTaskClientFactory rabbitStreamIndexTaskClientFactory, + @JacksonInject @Json ObjectMapper mapper, + @JacksonInject ServiceEmitter emitter, + @JacksonInject DruidMonitorSchedulerConfig monitorSchedulerConfig, + @JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory, + @JacksonInject SupervisorStateManagerConfig supervisorStateManagerConfig) + { + super( + ingestionSchema != null + ? ingestionSchema + : new RabbitStreamSupervisorIngestionSpec( + dataSchema, + ioConfig, + tuningConfig != null + ? tuningConfig + : RabbitStreamSupervisorTuningConfig.defaultConfig()), + context, + suspended, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + rabbitStreamIndexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig); + } + + @Override + public String getType() + { + return TASK_TYPE; + } + + @Override + public String getSource() + { + return getIoConfig() != null ? getIoConfig().getStream() : null; + } + + @Override + public Supervisor createSupervisor() + { + return new RabbitStreamSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + (RabbitStreamIndexTaskClientFactory) indexTaskClientFactory, + mapper, + this, + rowIngestionMetersFactory); + } + + @Override + @Deprecated + @JsonProperty + public RabbitStreamSupervisorTuningConfig getTuningConfig() + { + return (RabbitStreamSupervisorTuningConfig) super.getTuningConfig(); + } + + @Override + @Deprecated + @JsonProperty + public RabbitStreamSupervisorIOConfig getIoConfig() + { + return (RabbitStreamSupervisorIOConfig) super.getIoConfig(); + } + + @Override + @JsonProperty + public RabbitStreamSupervisorIngestionSpec getSpec() + { + return (RabbitStreamSupervisorIngestionSpec) super.getSpec(); + } + + @Override + protected RabbitStreamSupervisorSpec toggleSuspend(boolean suspend) + { + return new RabbitStreamSupervisorSpec( + getSpec(), + getDataSchema(), + getTuningConfig(), + getIoConfig(), + getContext(), + suspend, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + (RabbitStreamIndexTaskClientFactory) indexTaskClientFactory, + mapper, + emitter, + monitorSchedulerConfig, + rowIngestionMetersFactory, + supervisorStateManagerConfig); + } + + @Override + public String toString() + { + return "RabbitStreamSupervisorSpec{" + + "dataSchema=" + getDataSchema() + + ", tuningConfig=" + getTuningConfig() + + ", ioConfig=" + getIoConfig() + + ", context=" + getContext() + + ", suspend=" + isSuspended() + + '}'; + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java new file mode 100644 index 0000000000000..9c300deb9ef32 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/main/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfig.java @@ -0,0 +1,248 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskTuningConfig; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.annotation.Nullable; + +public class RabbitStreamSupervisorTuningConfig extends RabbitStreamIndexTaskTuningConfig + implements SeekableStreamSupervisorTuningConfig +{ + private final Integer workerThreads; + private final Boolean chatAsync; + private final Integer chatThreads; + private final Long chatRetries; + private final Duration httpTimeout; + private final Duration shutdownTimeout; + private final Duration offsetFetchPeriod; + + public static RabbitStreamSupervisorTuningConfig defaultConfig() + { + return new RabbitStreamSupervisorTuningConfig( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + } + + public RabbitStreamSupervisorTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, + @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, + @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, + @JsonProperty("skipBytesInMemoryOverheadCheck") @Nullable Boolean skipBytesInMemoryOverheadCheck, + @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, + @JsonProperty("maxTotalRows") Long maxTotalRows, + @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, + @JsonProperty("maxPendingPersists") Integer maxPendingPersists, + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, + @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, + @JsonProperty("workerThreads") Integer workerThreads, + @JsonProperty("chatAsync") Boolean chatAsync, + @JsonProperty("chatThreads") Integer chatThreads, + @JsonProperty("chatRetries") Long chatRetries, + @JsonProperty("httpTimeout") Period httpTimeout, + @JsonProperty("shutdownTimeout") Period shutdownTimeout, + @JsonProperty("recordBufferSize") Integer recordBufferSize, + @JsonProperty("recordBufferOfferTimeout") Integer recordBufferOfferTimeout, + @JsonProperty("offsetFetchPeriod") Period offsetFetchPeriod, + @JsonProperty("intermediateHandoffPeriod") Period intermediateHandoffPeriod, + @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, + @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, + @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions, + @JsonProperty("maxRecordsPerPoll") @Nullable Integer maxRecordsPerPoll) + { + super( + appendableIndexSpec, + maxRowsInMemory, + maxBytesInMemory, + skipBytesInMemoryOverheadCheck, + maxRowsPerSegment, + maxTotalRows, + intermediatePersistPeriod, + null, + maxPendingPersists, + indexSpec, + indexSpecForIntermediatePersists, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory, + intermediateHandoffPeriod, + logParseExceptions, + maxParseExceptions, + maxSavedParseExceptions, + recordBufferSize, + recordBufferOfferTimeout, + maxRecordsPerPoll); + this.workerThreads = workerThreads; + this.chatAsync = chatAsync; + this.chatThreads = chatThreads; + this.chatRetries = (chatRetries != null ? chatRetries : DEFAULT_CHAT_RETRIES); + this.httpTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration(httpTimeout, DEFAULT_HTTP_TIMEOUT); + this.shutdownTimeout = SeekableStreamSupervisorTuningConfig.defaultDuration( + shutdownTimeout, + DEFAULT_SHUTDOWN_TIMEOUT); + this.offsetFetchPeriod = SeekableStreamSupervisorTuningConfig.defaultDuration( + offsetFetchPeriod, + DEFAULT_OFFSET_FETCH_PERIOD); + } + + @Override + @JsonProperty + public Integer getWorkerThreads() + { + return workerThreads; + } + + @Override + @JsonProperty + public Long getChatRetries() + { + return chatRetries; + } + + @Override + @JsonProperty + public Duration getHttpTimeout() + { + return httpTimeout; + } + + @Override + @JsonProperty + public Duration getShutdownTimeout() + { + return shutdownTimeout; + } + + @Override + public Duration getRepartitionTransitionDuration() + { + // just return a default for now. + return SeekableStreamSupervisorTuningConfig.defaultDuration( + null, + SeekableStreamSupervisorTuningConfig.DEFAULT_REPARTITION_TRANSITION_DURATION); + } + + @Override + @JsonProperty + public Duration getOffsetFetchPeriod() + { + return offsetFetchPeriod; + } + + @Override + public String toString() + { + return "RabbitStreamSupervisorTuningConfig{" + + "maxRowsInMemory=" + getMaxRowsInMemory() + + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + + ", maxTotalRows=" + getMaxTotalRows() + + ", maxBytesInMemory=" + getMaxBytesInMemoryOrDefault() + + ", skipBytesInMemoryOverheadCheck=" + isSkipBytesInMemoryOverheadCheck() + + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + + ", maxPendingPersists=" + getMaxPendingPersists() + + ", indexSpec=" + getIndexSpec() + + ", reportParseExceptions=" + isReportParseExceptions() + + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + + ", workerThreads=" + workerThreads + + ", chatThreads=" + chatThreads + + ", chatRetries=" + chatRetries + + ", httpTimeout=" + httpTimeout + + ", shutdownTimeout=" + shutdownTimeout + + ", recordBufferSize=" + getRecordBufferSizeConfigured() + + ", recordBufferOfferTimeout=" + getRecordBufferOfferTimeout() + + ", offsetFetchPeriod=" + offsetFetchPeriod + + ", intermediateHandoffPeriod=" + getIntermediateHandoffPeriod() + + ", logParseExceptions=" + isLogParseExceptions() + + ", maxParseExceptions=" + getMaxParseExceptions() + + ", maxSavedParseExceptions=" + getMaxSavedParseExceptions() + + ", maxRecordsPerPoll=" + getMaxRecordsPerPollConfigured() + + '}'; + } + + @Override + public RabbitStreamIndexTaskTuningConfig convertToTaskTuningConfig() + { + return new RabbitStreamIndexTaskTuningConfig( + getAppendableIndexSpec(), + getMaxRowsInMemory(), + getMaxBytesInMemory(), + isSkipBytesInMemoryOverheadCheck(), + getMaxRowsPerSegment(), + getMaxTotalRows(), + getIntermediatePersistPeriod(), + null, + getMaxPendingPersists(), + getIndexSpec(), + getIndexSpecForIntermediatePersists(), + isReportParseExceptions(), + getHandoffConditionTimeout(), + isResetOffsetAutomatically(), + getSegmentWriteOutMediumFactory(), + getIntermediateHandoffPeriod(), + isLogParseExceptions(), + getMaxParseExceptions(), + getMaxSavedParseExceptions(), + getRecordBufferSizeConfigured(), + getRecordBufferOfferTimeout(), + getMaxRecordsPerPollConfigured() + ); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/rabbit-stream-indexing-service/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 0000000000000..f88539cd55faf --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/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.indexing.rabbitstream.RabbitStreamIndexTaskModule diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java new file mode 100644 index 0000000000000..ef77c58163c22 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskIOConfigTest.java @@ -0,0 +1,88 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.indexing.IOConfig; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Collections; + +public class RabbitStreamIndexTaskIOConfigTest +{ + private final ObjectMapper mapper; + + public RabbitStreamIndexTaskIOConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new RabbitStreamIndexTaskModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"baseSequenceName\": \"my-sequence-name\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\",\n" + + " \"startSequenceNumbers\": {\"type\":\"start\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"stream-0\":1, \"stream-1\":10}},\n" + + " \"endSequenceNumbers\": {\"type\":\"end\", \"stream\":\"mystream\", \"partitionSequenceNumberMap\" : {\"stream-0\":15, \"stream-1\":200}}\n" + + "}"; + + RabbitStreamIndexTaskIOConfig config = (RabbitStreamIndexTaskIOConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + IOConfig.class)), + IOConfig.class); + + Assert.assertNull(config.getTaskGroupId()); + Assert.assertEquals("my-sequence-name", config.getBaseSequenceName()); + + Assert.assertEquals("mystream", config.getStartSequenceNumbers().getStream()); + + Assert.assertEquals(Long.class, + config.getStartSequenceNumbers().getPartitionSequenceNumberMap().get("stream-1").getClass()); + Assert.assertEquals( + ImmutableMap.of("stream-0", Long.valueOf(1), "stream-1", Long.valueOf(10)), + config.getStartSequenceNumbers().getPartitionSequenceNumberMap()); + + Assert.assertEquals("mystream", config.getEndSequenceNumbers().getStream()); + + Assert.assertEquals( + ImmutableMap.of("stream-0", Long.valueOf(15L), "stream-1", Long.valueOf(200L)), + config.getEndSequenceNumbers().getPartitionSequenceNumberMap()); + + Assert.assertTrue(config.isUseTransaction()); + Assert.assertFalse("minimumMessageTime", config.getMinimumMessageTime().isPresent()); + Assert.assertEquals(config.getUri(), "rabbitmq-stream://localhost:5552"); + Assert.assertEquals(Collections.emptySet(), config.getStartSequenceNumbers().getExclusivePartitions()); + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java new file mode 100644 index 0000000000000..50ae2a0fc615a --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamIndexTaskTuningConfigTest.java @@ -0,0 +1,116 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.indexing.TuningConfig; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.time.Duration; + +public class RabbitStreamIndexTaskTuningConfigTest +{ + private final ObjectMapper mapper; + + public RabbitStreamIndexTaskTuningConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new RabbitStreamIndexTaskModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\"type\": \"rabbit\"}"; + + RabbitStreamIndexTaskTuningConfig config = (RabbitStreamIndexTaskTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class)), + TuningConfig.class); + + Assert.assertNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); + Assert.assertEquals(150000, config.getMaxRowsInMemory()); + Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); + Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + // Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + Assert.assertEquals(false, config.isReportParseExceptions()); + Assert.assertEquals(Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); + + Assert.assertNull(config.getRecordBufferSizeConfigured()); + Assert.assertEquals(10000, config.getRecordBufferSizeOrDefault(1_000_000_000)); + Assert.assertEquals(5000, config.getRecordBufferOfferTimeout()); + + Assert.assertFalse(config.isSkipSequenceNumberAvailabilityCheck()); + Assert.assertFalse(config.isResetOffsetAutomatically()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"recordBufferSize\": 1000,\n" + + " \"recordBufferOfferTimeout\": 500,\n" + + " \"resetOffsetAutomatically\": false,\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + + "}"; + + RabbitStreamIndexTaskTuningConfig config = (RabbitStreamIndexTaskTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class)), + TuningConfig.class); + + Assert.assertNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); + Assert.assertEquals(100, config.getMaxRowsInMemory()); + Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); + Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertTrue(config.isReportParseExceptions()); + Assert.assertEquals(100, config.getHandoffConditionTimeout()); + Assert.assertEquals(1000, (int) config.getRecordBufferSizeConfigured()); + Assert.assertEquals(1000, config.getRecordBufferSizeOrDefault(1_000_000_000)); + Assert.assertEquals(500, config.getRecordBufferOfferTimeout()); + Assert.assertFalse(config.isResetOffsetAutomatically()); + } +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplierTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplierTest.java new file mode 100644 index 0000000000000..6ed475bd29530 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/RabbitStreamRecordSupplierTest.java @@ -0,0 +1,591 @@ +/* + * 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.rabbitstream; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.rabbitmq.stream.Consumer; +import com.rabbitmq.stream.ConsumerBuilder; +import com.rabbitmq.stream.Environment; +import com.rabbitmq.stream.EnvironmentBuilder; +import com.rabbitmq.stream.MessageHandler; +import com.rabbitmq.stream.OffsetSpecification; +import com.rabbitmq.stream.codec.WrapperMessageBuilder; +import com.rabbitmq.stream.impl.Client; +import com.rabbitmq.stream.impl.Client.ClientParameters; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.metadata.DynamicConfigProvider; +import org.apache.druid.metadata.MapStringDynamicConfigProvider; +import org.apache.druid.segment.TestHelper; +import org.easymock.EasyMock; +import org.easymock.EasyMockSupport; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class RabbitStreamRecordSupplierTest extends EasyMockSupport +{ + + private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static String uri = "rabbitmq-stream://localhost:5552"; + private static Environment environment; + private static EnvironmentBuilder environmentBuilder; + private static ClientParameters clientParameters; + private static Client client; + + private static final String STREAM = "stream"; + private static final String PARTITION_ID1 = "stream-1"; + private static final String PARTITION_ID0 = "stream-0"; + private static final List ALL_PARTITIONS = ImmutableList.of(PARTITION_ID0, PARTITION_ID1); + + private static class MockedRabbitStreamRecordSupplier extends RabbitStreamRecordSupplier + { + + public ClientParameters sentParameters = null; + + public MockedRabbitStreamRecordSupplier(String uri, Map consumerProperties) + { + super( + consumerProperties, + OBJECT_MAPPER, + uri, + 100, + 1000, + 100); + + } + + @Override + public EnvironmentBuilder getEnvBuilder() + { + return environmentBuilder; + } + + @Override + public ClientParameters getParameters() + { + return clientParameters; + } + + @Override + public Client getClient(ClientParameters paramaters) + { + this.sentParameters = paramaters; + return client; + } + + } + + public MockedRabbitStreamRecordSupplier makeRecordSupplierWithMockedEnvironment(String uri, Map consumerProperties) + { + EasyMock.expect(environmentBuilder.uri(uri)).andReturn(environmentBuilder); + EasyMock.expect(environmentBuilder.build()).andReturn(environment); + replayAll(); + MockedRabbitStreamRecordSupplier res = new MockedRabbitStreamRecordSupplier(uri, consumerProperties); + resetAll(); + return res; + } + + private class MessageHandlerContext implements MessageHandler.Context + { + + private final long offset; + private final long timestamp; + private final long committedOffset; + private final String stream; + + private MessageHandlerContext( + long offset, + long timestamp, + long committedOffset, + String stream) + { + this.offset = offset; + this.timestamp = timestamp; + this.committedOffset = committedOffset; + this.stream = stream; + } + + @Override + public long offset() + { + return this.offset; + } + + @Override + public void storeOffset() + { + } + + @Override + public long timestamp() + { + return this.timestamp; + } + + @Override + public long committedChunkId() + { + return this.committedOffset; + } + + @Override + public String stream() + { + return this.stream; + } + + @Override + public Consumer consumer() + { + return createMock(Consumer.class); + } + } + + @Before + public void setupTest() + { + environment = createMock(Environment.class); + environmentBuilder = createMock(EnvironmentBuilder.class); + client = createMock(Client.class); + clientParameters = createMock(ClientParameters.class); + + } + + @Test + public void testGetStreamFromSubstream() + { + String test = "stream-0"; + String res = RabbitStreamRecordSupplier.getStreamFromSubstream(test); + Assert.assertEquals("stream", res); + + test = "test-stream-0"; + res = RabbitStreamRecordSupplier.getStreamFromSubstream(test); + Assert.assertEquals("test-stream", res); + + } + + @Test + public void testAssign() + { + Set> partitions = ImmutableSet.of( + StreamPartition.of(STREAM, PARTITION_ID0), + StreamPartition.of(STREAM, PARTITION_ID1)); + + Set> res; + + RabbitStreamRecordSupplier recordSupplier = makeRecordSupplierWithMockedEnvironment( + uri, + null + ); + + + res = recordSupplier.getAssignment(); + + Assert.assertTrue(res.isEmpty()); + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andStubReturn(environment); + + ConsumerBuilder consumerBuilderMock1 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.noTrackingStrategy()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.stream(PARTITION_ID0)).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.messageHandler(recordSupplier)).andReturn(consumerBuilderMock1).once(); + + ConsumerBuilder consumerBuilderMock2 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.noTrackingStrategy()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.stream(PARTITION_ID1)).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.messageHandler(recordSupplier)).andReturn(consumerBuilderMock2).once(); + + replayAll(); + + recordSupplier.assign(partitions); + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + + verifyAll(); + } + + @Test + public void testAssignClears() + { + StreamPartition partition0 = StreamPartition.of(STREAM, PARTITION_ID0); + StreamPartition partition1 = StreamPartition.of(STREAM, PARTITION_ID1); + long offset1 = 1; + long offset2 = 2; + + Set> partitions = ImmutableSet.of( + StreamPartition.of(STREAM, PARTITION_ID0), + StreamPartition.of(STREAM, PARTITION_ID1)); + + Set> res; + + RabbitStreamRecordSupplier recordSupplier = makeRecordSupplierWithMockedEnvironment( + uri, + null + ); + + + res = recordSupplier.getAssignment(); + + Assert.assertTrue(res.isEmpty()); + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andStubReturn(environment); + + ConsumerBuilder consumerBuilderMock1 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock1).once(); + + EasyMock.expect(consumerBuilderMock1.noTrackingStrategy()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.stream(PARTITION_ID0)).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.messageHandler(recordSupplier)).andReturn(consumerBuilderMock1).once(); + + ConsumerBuilder consumerBuilderMock2 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.noTrackingStrategy()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.stream(PARTITION_ID1)).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.messageHandler(recordSupplier)).andReturn(consumerBuilderMock2).once(); + + replayAll(); + + recordSupplier.assign(partitions); + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + + verifyAll(); + + try { + recordSupplier.seek(partition0, offset1); + recordSupplier.seek(partition1, offset2); + } + catch (Exception exc) { + Assert.fail("Exception seeking:" + exc.getMessage()); + } + + resetAll(); + + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.noTrackingStrategy()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.stream(PARTITION_ID0)).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.messageHandler(recordSupplier)).andReturn(consumerBuilderMock1).once(); + replayAll(); + + recordSupplier.assign(ImmutableSet.of(partition0)); + Assert.assertEquals(ImmutableSet.of(partition0), recordSupplier.getAssignment()); + + Assert.assertNotNull(recordSupplier.getOffset(partition0)); + Assert.assertNull(recordSupplier.getOffset(partition1)); + + } + + @Test + public void testSeek() + { + StreamPartition partition0 = StreamPartition.of(STREAM, PARTITION_ID0); + StreamPartition partition1 = StreamPartition.of(STREAM, PARTITION_ID1); + + Set> partitions = ImmutableSet.of( + partition0, + partition1); + + long offset1 = 1; + long offset2 = 2; + + + RabbitStreamRecordSupplier recordSupplier = makeRecordSupplierWithMockedEnvironment( + uri, + null + ); + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andStubReturn(environment); + + ConsumerBuilder consumerBuilderMock1 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock1).once(); + + EasyMock.expect(consumerBuilderMock1.noTrackingStrategy()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.stream(PARTITION_ID0)).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.messageHandler(recordSupplier)).andReturn(consumerBuilderMock1).once(); + + ConsumerBuilder consumerBuilderMock2 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.noTrackingStrategy()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.stream(PARTITION_ID1)).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.messageHandler(recordSupplier)).andReturn(consumerBuilderMock2).once(); + + replayAll(); + + recordSupplier.assign(partitions); + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + + verifyAll(); + + try { + recordSupplier.seek(partition0, offset1); + recordSupplier.seek(partition1, offset2); + } + catch (Exception exc) { + Assert.fail("Exception seeking:" + exc.getMessage()); + } + + + Assert.assertEquals(recordSupplier.getOffset(partition0).getOffset(), offset1); + Assert.assertEquals(recordSupplier.getOffset(partition1).getOffset(), offset2); + + try { + recordSupplier.seekToEarliest(partitions); + } + catch (Exception exc) { + Assert.fail("Exception seeking:" + exc.getMessage()); + } + Assert.assertEquals(recordSupplier.getOffset(partition0), OffsetSpecification.first()); + Assert.assertEquals(recordSupplier.getOffset(partition1), OffsetSpecification.first()); + + try { + recordSupplier.seekToLatest(partitions); + } + catch (Exception exc) { + Assert.fail("Exception seeking:" + exc.getMessage()); + } + + Assert.assertEquals(recordSupplier.getOffset(partition0), OffsetSpecification.last()); + Assert.assertEquals(recordSupplier.getOffset(partition1), OffsetSpecification.last()); + + } + + + + @Test + public void testPollBothPartitions() + { + + StreamPartition partition1 = StreamPartition.of(STREAM, PARTITION_ID0); + StreamPartition partition2 = StreamPartition.of(STREAM, PARTITION_ID1); + + Set> partitions = ImmutableSet.of( + partition1, + partition2); + + long offset1 = 1; + long offset2 = 2; + + RabbitStreamRecordSupplier recordSupplier = makeRecordSupplierWithMockedEnvironment( + uri, + null + ); + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andStubReturn(environment); + + ConsumerBuilder consumerBuilderMock1 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.noTrackingStrategy()).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.stream(PARTITION_ID0)).andReturn(consumerBuilderMock1).once(); + EasyMock.expect(consumerBuilderMock1.messageHandler(recordSupplier)).andReturn(consumerBuilderMock1).once(); + + EasyMock.expect(consumerBuilderMock1.offset(OffsetSpecification.offset(offset1))).andReturn(consumerBuilderMock1) + .once(); + + Consumer consumer1 = createMock(Consumer.class); + consumer1.close(); + + MessageHandler.Context mockMessageContext1 = new MessageHandlerContext(0, 0, 0, PARTITION_ID0); + WrapperMessageBuilder mockMessageBuilder = new WrapperMessageBuilder(); + mockMessageBuilder.addData("Hello shard0".getBytes(StandardCharsets.UTF_8)); + + // when the consumerbuilder is turned into a consumer, call the handler method + // to test round trip and ensure messages come back + EasyMock.expect(consumerBuilderMock1.build()).andAnswer(() -> { + recordSupplier.handle( + mockMessageContext1, + mockMessageBuilder.build()); + return consumer1; + }).once(); + + ConsumerBuilder consumerBuilderMock2 = createMock(ConsumerBuilder.class); + EasyMock.expect(environment.consumerBuilder()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.noTrackingStrategy()).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.stream(PARTITION_ID1)).andReturn(consumerBuilderMock2).once(); + EasyMock.expect(consumerBuilderMock2.messageHandler(recordSupplier)).andReturn(consumerBuilderMock2).once(); + + EasyMock.expect(consumerBuilderMock2.offset(OffsetSpecification.offset(offset2))).andReturn(consumerBuilderMock2) + .once(); + + Consumer consumer2 = createMock(Consumer.class); + consumer2.close(); + + MessageHandler.Context mockMessageContext2 = new MessageHandlerContext(0, 0, 0, PARTITION_ID1); + WrapperMessageBuilder mockMessageBuilder2 = new WrapperMessageBuilder(); + mockMessageBuilder2.addData("Hello shard1".getBytes(StandardCharsets.UTF_8)); + + EasyMock.expect(consumerBuilderMock2.build()).andAnswer(() -> { + recordSupplier.handle( + mockMessageContext2, + mockMessageBuilder2.build()); + return consumer2; + }).once(); + + replayAll(); + + recordSupplier.assign(partitions); + + try { + recordSupplier.seek(partition1, offset1); + recordSupplier.seek(partition2, offset2); + } + catch (Exception exc) { + Assert.fail("Exception seeking:" + exc.getMessage()); + } + + List> messages = recordSupplier.poll(0); + Assert.assertEquals(2, messages.size()); + + recordSupplier.close(); + + // test double close + recordSupplier.close(); + + verifyAll(); + } + + + @Test + public void testConsumerProperties() + { + + DynamicConfigProvider dynamicConfigProvider = new MapStringDynamicConfigProvider( + ImmutableMap.of( + "username", "RABBIT_USERNAME", + "password", "RABBIT_PASSWORD" + ) + ); + Map consumerProperties = ImmutableMap.of( + "druid.dynamic.config.provider", OBJECT_MAPPER.convertValue(dynamicConfigProvider, Map.class) + ); + + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.password("RABBIT_PASSWORD")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.username("RABBIT_USERNAME")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andReturn(environment).once(); + replayAll(); + + MockedRabbitStreamRecordSupplier supplier = new MockedRabbitStreamRecordSupplier("rabbitmq-stream://localhost:5552", consumerProperties); + supplier.getRabbitEnvironment(); + verifyAll(); + supplier.close(); + + + + } + + + @Test + public void testGetPartitionIDs() + { + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andReturn(environment).once(); + replayAll(); + + MockedRabbitStreamRecordSupplier supplier = new MockedRabbitStreamRecordSupplier("rabbitmq-stream://localhost:5552", null); + supplier.getRabbitEnvironment(); + verifyAll(); + resetAll(); + + EasyMock.expect(clientParameters.host("localhost")).andReturn(clientParameters); + EasyMock.expect(clientParameters.port(5552)).andReturn(clientParameters); + + + EasyMock.expect(client.partitions(STREAM)).andReturn(ALL_PARTITIONS); + + client.close(); + replayAll(); + + Set partitions = supplier.getPartitionIds(STREAM); + verifyAll(); + + Assert.assertTrue(clientParameters == supplier.sentParameters); + + Assert.assertEquals(2, partitions.size()); + Assert.assertTrue(partitions.containsAll(ALL_PARTITIONS)); + + supplier.close(); + + } + + @Test + public void testGetPartitionIDsWithConfig() + { + + DynamicConfigProvider dynamicConfigProvider = new MapStringDynamicConfigProvider( + ImmutableMap.of( + "username", "RABBIT_USERNAME", + "password", "RABBIT_PASSWORD" + ) + ); + Map consumerProperties = ImmutableMap.of( + "druid.dynamic.config.provider", OBJECT_MAPPER.convertValue(dynamicConfigProvider, Map.class) + ); + + + EasyMock.expect(environmentBuilder.uri("rabbitmq-stream://localhost:5552")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.password("RABBIT_PASSWORD")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.username("RABBIT_USERNAME")).andReturn(environmentBuilder).once(); + EasyMock.expect(environmentBuilder.build()).andReturn(environment).once(); + replayAll(); + + MockedRabbitStreamRecordSupplier supplier = new MockedRabbitStreamRecordSupplier("rabbitmq-stream://localhost:5552", consumerProperties); + supplier.getRabbitEnvironment(); + verifyAll(); + resetAll(); + + + EasyMock.expect(clientParameters.host("localhost")).andReturn(clientParameters); + EasyMock.expect(clientParameters.port(5552)).andReturn(clientParameters); + EasyMock.expect(clientParameters.password("RABBIT_PASSWORD")).andReturn(clientParameters); + EasyMock.expect(clientParameters.username("RABBIT_USERNAME")).andReturn(clientParameters); + + EasyMock.expect(client.partitions(STREAM)).andReturn(ALL_PARTITIONS); + + client.close(); + replayAll(); + + Set partitions = supplier.getPartitionIds(STREAM); + verifyAll(); + + Assert.assertTrue(clientParameters == supplier.sentParameters); + + Assert.assertEquals(2, partitions.size()); + Assert.assertTrue(partitions.containsAll(ALL_PARTITIONS)); + + supplier.close(); + + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java new file mode 100644 index 0000000000000..347152a4bb28c --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorIOConfigTest.java @@ -0,0 +1,139 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskModule; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.hamcrest.CoreMatchers; +import org.joda.time.Duration; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class RabbitStreamSupervisorIOConfigTest +{ + private final ObjectMapper mapper; + + public RabbitStreamSupervisorIOConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new RabbitStreamIndexTaskModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\"\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue( + jsonStr, + RabbitStreamSupervisorIOConfig.class); + + Assert.assertEquals("my-stream", config.getStream()); + Assert.assertEquals(config.getUri(), "rabbitmq-stream://localhost:5552"); + Assert.assertEquals(1, (int) config.getReplicas()); + Assert.assertEquals(1, (int) config.getTaskCount()); + Assert.assertEquals(Duration.standardMinutes(60), config.getTaskDuration()); + Assert.assertEquals(Duration.standardSeconds(5), config.getStartDelay()); + Assert.assertEquals(Duration.standardSeconds(30), config.getPeriod()); + Assert.assertFalse(config.isUseEarliestSequenceNumber()); + Assert.assertEquals(Duration.standardMinutes(30), config.getCompletionTimeout()); + Assert.assertFalse("lateMessageRejectionPeriod", config.getLateMessageRejectionPeriod().isPresent()); + Assert.assertFalse("earlyMessageRejectionPeriod", config.getEarlyMessageRejectionPeriod().isPresent()); + Assert.assertFalse("lateMessageRejectionStartDateTime", config.getLateMessageRejectionStartDateTime().isPresent()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\",\n" + + " \"uri\": \"rabbitmq-stream://localhost:5552\",\n" + + " \"replicas\": 3,\n" + + " \"taskCount\": 9,\n" + + " \"taskDuration\": \"PT30M\",\n" + + " \"startDelay\": \"PT1M\",\n" + + " \"period\": \"PT10S\",\n" + + " \"useEarliestOffset\": true,\n" + + " \"completionTimeout\": \"PT45M\",\n" + + " \"lateMessageRejectionPeriod\": \"PT1H\",\n" + + " \"earlyMessageRejectionPeriod\": \"PT1H\",\n" + + " \"recordsPerFetch\": 4000\n" + + "}"; + + RabbitStreamSupervisorIOConfig config = mapper.readValue( + jsonStr, + RabbitStreamSupervisorIOConfig.class); + + Assert.assertEquals("my-stream", config.getStream()); + Assert.assertEquals(config.getUri(), "rabbitmq-stream://localhost:5552"); + Assert.assertEquals(3, (int) config.getReplicas()); + Assert.assertEquals(9, (int) config.getTaskCount()); + Assert.assertEquals(Duration.standardMinutes(30), config.getTaskDuration()); + Assert.assertEquals(Duration.standardMinutes(1), config.getStartDelay()); + Assert.assertEquals(Duration.standardSeconds(10), config.getPeriod()); + Assert.assertTrue(config.isUseEarliestSequenceNumber()); + Assert.assertEquals(Duration.standardMinutes(45), config.getCompletionTimeout()); + Assert.assertEquals(Duration.standardHours(1), config.getLateMessageRejectionPeriod().get()); + Assert.assertEquals(Duration.standardHours(1), config.getEarlyMessageRejectionPeriod().get()); + // Assert.assertEquals((Integer) 4000, config.getRecordsPerFetch()); + // Assert.assertEquals(1000, config.getFetchDelayMillis()); + } + + @Test + public void testStreamRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\"\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("stream")); + mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + } + + @Test + public void testURIRequired() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"stream\": \"my-stream\"\n" + + "}"; + + exception.expect(JsonMappingException.class); + exception.expectCause(CoreMatchers.isA(NullPointerException.class)); + exception.expectMessage(CoreMatchers.containsString("uri")); + mapper.readValue(jsonStr, RabbitStreamSupervisorIOConfig.class); + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java new file mode 100644 index 0000000000000..9b9a0ca905bfa --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -0,0 +1,368 @@ +/* + * 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.rabbitstream.supervisor; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.TaskQueue; +import org.apache.druid.indexing.overlord.TaskRunner; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskClientFactory; +import org.apache.druid.indexing.rabbitstream.RabbitStreamRecordSupplier; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskClient; +import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorReportPayload; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.incremental.RowIngestionMetersFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; +import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.easymock.EasyMockSupport; +import org.joda.time.Period; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class RabbitStreamSupervisorTest extends EasyMockSupport +{ + private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final InputFormat INPUT_FORMAT = new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false, + false, + false, + false); + private static final String DATASOURCE = "testDS"; + private static final int TEST_CHAT_THREADS = 3; + private static final long TEST_CHAT_RETRIES = 9L; + private static final Period TEST_HTTP_TIMEOUT = new Period("PT10S"); + private static final Period TEST_SHUTDOWN_TIMEOUT = new Period("PT80S"); + private static final String STREAM = "stream"; + private static final String URI = "rabbitmq-stream://localhost:5552"; + + private static DataSchema dataSchema; + private RabbitStreamRecordSupplier supervisorRecordSupplier; + + private final int numThreads = 1; + private RabbitStreamSupervisor supervisor; + private RabbitStreamSupervisorTuningConfig tuningConfig; + private TaskStorage taskStorage; + private TaskMaster taskMaster; + private TaskRunner taskRunner; + private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; + private SeekableStreamIndexTaskClient taskClient; + private TaskQueue taskQueue; + private RowIngestionMetersFactory rowIngestionMetersFactory; + private ExceptionCapturingServiceEmitter serviceEmitter; + private SupervisorStateManagerConfig supervisorConfig; + + private static DataSchema getDataSchema(String dataSource) + { + List dimensions = new ArrayList<>(); + dimensions.add(StringDimensionSchema.create("dim1")); + dimensions.add(StringDimensionSchema.create("dim2")); + + return new DataSchema( + dataSource, + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(dimensions), + new AggregatorFactory[] {new CountAggregatorFactory("rows")}, + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of()), + null); + } + + @BeforeClass + public static void setupClass() + { + dataSchema = getDataSchema(DATASOURCE); + } + + @Before + public void setupTest() + { + taskStorage = createMock(TaskStorage.class); + taskMaster = createMock(TaskMaster.class); + taskRunner = createMock(TaskRunner.class); + indexerMetadataStorageCoordinator = createMock(IndexerMetadataStorageCoordinator.class); + taskClient = createMock(SeekableStreamIndexTaskClient.class); + taskQueue = createMock(TaskQueue.class); + supervisorRecordSupplier = createMock(RabbitStreamRecordSupplier.class); + + tuningConfig = new RabbitStreamSupervisorTuningConfig( + null, + 1000, // max rows in memory + null, // max bytes + null, // skipBytes + 50000, // max rows per seg + null, // max total rows + new Period("P1Y"), // intermediatepersistPeriod + null, + null, + null, + null, + null, + null, + null, + numThreads, // worker threads + null, + TEST_CHAT_THREADS, + TEST_CHAT_RETRIES, + TEST_HTTP_TIMEOUT, + TEST_SHUTDOWN_TIMEOUT, + 1000, + 100, + null, + null, + null, + null, + null, + 100); + rowIngestionMetersFactory = new TestUtils().getRowIngestionMetersFactory(); + serviceEmitter = new ExceptionCapturingServiceEmitter(); + EmittingLogger.registerEmitter(serviceEmitter); + supervisorConfig = new SupervisorStateManagerConfig(); + } + + @After + public void tearDownTest() + { + supervisor = null; + } + + /** + * Use for tests where you don't want generateSequenceName to be overridden out + */ + private RabbitStreamSupervisor getSupervisor( + int replicas, + int taskCount, + boolean useEarliestOffset, + String duration, + Period lateMessageRejectionPeriod, + Period earlyMessageRejectionPeriod, + DataSchema dataSchema, + RabbitStreamSupervisorTuningConfig tuningConfig) + { + RabbitStreamSupervisorIOConfig rabbitStreamSupervisorIOConfig = new RabbitStreamSupervisorIOConfig( + STREAM, // stream + URI, // uri + INPUT_FORMAT, // inputFormat + replicas, // replicas + taskCount, // taskCount + new Period(duration), // taskDuration + null, // consumerProperties + null, // autoscalerConfig + 400L, // poll timeout + new Period("P1D"), // start delat + new Period("PT30M"), // period + new Period("PT30S"), // completiontimeout + false, // useearliest + lateMessageRejectionPeriod, // latemessagerejection + earlyMessageRejectionPeriod, // early message rejection + null, // latemessagerejectionstartdatetime + 1 + ); + RabbitStreamIndexTaskClientFactory clientFactory = new RabbitStreamIndexTaskClientFactory(null, + OBJECT_MAPPER); + RabbitStreamSupervisor supervisor = new RabbitStreamSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + clientFactory, + OBJECT_MAPPER, + new RabbitStreamSupervisorSpec( + null, + dataSchema, + tuningConfig, + rabbitStreamSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + clientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig()), + rowIngestionMetersFactory); + return supervisor; + } + + public RabbitStreamSupervisor getDefaultSupervisor() + { + return getSupervisor( + 1, + 1, + false, + "PT30M", + null, + null, + RabbitStreamSupervisorTest.dataSchema, + tuningConfig); + } + + @Test + public void testRecordSupplier() + { + RabbitStreamSupervisorIOConfig rabbitStreamSupervisorIOConfig = new RabbitStreamSupervisorIOConfig( + STREAM, // stream + URI, // uri + INPUT_FORMAT, // inputFormat + 1, // replicas + 1, // taskCount + new Period("PT30M"), // taskDuration + null, // consumerProperties + null, // autoscalerConfig + 400L, // poll timeout + new Period("P1D"), // start delat + new Period("PT30M"), // period + new Period("PT30S"), // completiontimeout + false, // useearliest + null, // latemessagerejection + null, // early message rejection + null, // latemessagerejectionstartdatetime + 1 + ); + RabbitStreamIndexTaskClientFactory clientFactory = new RabbitStreamIndexTaskClientFactory(null, + OBJECT_MAPPER); + RabbitStreamSupervisor supervisor = new RabbitStreamSupervisor( + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + clientFactory, + OBJECT_MAPPER, + new RabbitStreamSupervisorSpec( + null, + dataSchema, + tuningConfig, + rabbitStreamSupervisorIOConfig, + null, + false, + taskStorage, + taskMaster, + indexerMetadataStorageCoordinator, + clientFactory, + OBJECT_MAPPER, + new NoopServiceEmitter(), + new DruidMonitorSchedulerConfig(), + rowIngestionMetersFactory, + new SupervisorStateManagerConfig()), + rowIngestionMetersFactory); + + RabbitStreamRecordSupplier supplier = (RabbitStreamRecordSupplier) supervisor.setupRecordSupplier(); + Assert.assertNotNull(supplier); + Assert.assertEquals(0, supplier.bufferSize()); + Assert.assertEquals(Collections.emptySet(), supplier.getAssignment()); + Assert.assertEquals(false, supplier.isRunning()); + } + + @Test + public void testGetters() + { + supervisor = getDefaultSupervisor(); + Assert.assertNull(supervisor.getPartitionTimeLag()); + + Assert.assertNull(supervisor.getTimeLagPerPartition(null)); + Assert.assertFalse(supervisor.isEndOfShard(null)); + Assert.assertFalse(supervisor.isShardExpirationMarker(null)); + + Assert.assertEquals(Long.valueOf(Long.MAX_VALUE), supervisor.getEndOfPartitionMarker()); + + Assert.assertEquals("index_rabbit", supervisor.baseTaskName()); + + Assert.assertEquals(Long.valueOf(-1L), supervisor.getNotSetMarker()); + Assert.assertEquals(false, supervisor.useExclusiveStartSequenceNumberForNonFirstSequence()); + + } + + @Test + public void testTaskGroupID() + { + + List taskCounts = ImmutableList.of(1, 2, 3, 4); + List partitions = ImmutableList.of("a", "b", "c"); + + for (Integer taskCount : taskCounts) { + supervisor = getSupervisor( + 1, + taskCount, + false, + "PT30M", + null, + null, + RabbitStreamSupervisorTest.dataSchema, + tuningConfig); + for (String partition : partitions) { + Assert.assertEquals(partition.hashCode() % taskCount, supervisor.getTaskGroupIdForPartition(partition)); + } + } + } + + @Test + public void testReportPayload() + { + supervisor = getSupervisor( + 1, + 1, + false, + "PT30M", + null, + null, + RabbitStreamSupervisorTest.dataSchema, + tuningConfig); + + SeekableStreamSupervisorReportPayload payload = supervisor.createReportPayload(1, false); + Assert.assertEquals(STREAM, payload.getStream()); + Assert.assertEquals(1, payload.getPartitions()); + Assert.assertEquals(1, payload.getReplicas()); + Assert.assertEquals(false, payload.isSuspended()); + Assert.assertEquals(true, payload.isHealthy()); + Assert.assertEquals(30 * 60, payload.getDurationSeconds()); + } + +} diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.java new file mode 100644 index 0000000000000..22f87bcc2b3e1 --- /dev/null +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTuningConfigTest.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.indexing.rabbitstream.supervisor; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.indexing.rabbitstream.RabbitStreamIndexTaskModule; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.indexing.TuningConfig; +import org.joda.time.Duration; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class RabbitStreamSupervisorTuningConfigTest +{ + private final ObjectMapper mapper; + + public RabbitStreamSupervisorTuningConfigTest() + { + mapper = new DefaultObjectMapper(); + mapper.registerModules((Iterable) new RabbitStreamIndexTaskModule().getJacksonModules()); + } + + @Rule + public final ExpectedException exception = ExpectedException.none(); + + @Test + public void testSerdeWithDefaults() throws Exception + { + String jsonStr = "{\"type\": \"rabbit\"}"; + + RabbitStreamSupervisorTuningConfig config = (RabbitStreamSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class)), + TuningConfig.class); + + Assert.assertNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); + Assert.assertEquals(150000, config.getMaxRowsInMemory()); + Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); + Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(0, config.getMaxPendingPersists()); + // Assert.assertEquals(IndexSpec.DEFAULT, config.getIndexSpec()); + Assert.assertEquals(false, config.isReportParseExceptions()); + Assert.assertEquals(java.time.Duration.ofMinutes(15).toMillis(), config.getHandoffConditionTimeout()); + Assert.assertNull(config.getWorkerThreads()); + Assert.assertEquals(8L, (long) config.getChatRetries()); + Assert.assertEquals(Duration.standardSeconds(10), config.getHttpTimeout()); + Assert.assertEquals(Duration.standardSeconds(80), config.getShutdownTimeout()); + Assert.assertEquals(Duration.standardSeconds(120), config.getRepartitionTransitionDuration()); + Assert.assertEquals(100, config.getMaxRecordsPerPollOrDefault()); + } + + @Test + public void testSerdeWithNonDefaults() throws Exception + { + String jsonStr = "{\n" + + " \"type\": \"rabbit\",\n" + + " \"basePersistDirectory\": \"/tmp/xxx\",\n" + + " \"maxRowsInMemory\": 100,\n" + + " \"maxRowsPerSegment\": 100,\n" + + " \"intermediatePersistPeriod\": \"PT1H\",\n" + + " \"maxPendingPersists\": 100,\n" + + " \"reportParseExceptions\": true,\n" + + " \"handoffConditionTimeout\": 100,\n" + + " \"workerThreads\": 12,\n" + + " \"chatThreads\": 13,\n" + + " \"chatRetries\": 14,\n" + + " \"httpTimeout\": \"PT15S\",\n" + + " \"shutdownTimeout\": \"PT95S\",\n" + + " \"repartitionTransitionDuration\": \"PT500S\",\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" },\n" + + " \"recordBufferSize\": 15,\n" + + " \"recordBufferOfferTimeout\": 16,\n" + + " \"maxRecordsPerPoll\": 17\n" + + "}"; + + RabbitStreamSupervisorTuningConfig config = (RabbitStreamSupervisorTuningConfig) mapper.readValue( + mapper.writeValueAsString( + mapper.readValue( + jsonStr, + TuningConfig.class)), + TuningConfig.class); + + Assert.assertNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); + Assert.assertEquals(100, config.getMaxRowsInMemory()); + Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); + Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); + Assert.assertEquals(100, config.getMaxPendingPersists()); + Assert.assertEquals(true, config.isReportParseExceptions()); + Assert.assertEquals(100, config.getHandoffConditionTimeout()); + Assert.assertEquals(12, (int) config.getWorkerThreads()); + Assert.assertEquals(14L, (long) config.getChatRetries()); + Assert.assertEquals(15, (int) config.getRecordBufferSizeConfigured()); + Assert.assertEquals(16, (int) config.getRecordBufferOfferTimeout()); + Assert.assertEquals(17, (int) config.getMaxRecordsPerPollConfigured()); + Assert.assertEquals(Duration.standardSeconds(15), config.getHttpTimeout()); + Assert.assertEquals(Duration.standardSeconds(95), config.getShutdownTimeout()); + Assert.assertEquals(Duration.standardSeconds(120), config.getRepartitionTransitionDuration()); + } + +} diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index 06d8cf537230f..e31ba104d05d6 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -43,15 +43,8 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC static final int ASSUMED_RECORD_SIZE = 10_000; static final int ASSUMED_RECORD_SIZE_AGGREGATE = 1_000_000; - /** - * Together with {@link KinesisIndexTaskIOConfig#MAX_RECORD_FETCH_MEMORY}, don't take up more than 200MB per task. - */ private static final int MAX_RECORD_BUFFER_MEMORY = 100_000_000; - /** - * Together with {@link KinesisIndexTaskIOConfig#RECORD_FETCH_MEMORY_MAX_HEAP_FRACTION}, don't take up more - * than 15% of the heap per task. - */ private static final double RECORD_BUFFER_MEMORY_MAX_HEAP_FRACTION = 0.1; private static final int DEFAULT_RECORD_BUFFER_OFFER_TIMEOUT = 5000; diff --git a/licenses.yaml b/licenses.yaml index f1a4cba1159cf..57929367defe6 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -2802,6 +2802,16 @@ libraries: --- +name: com.rabbitmq stream-client +license_category: binary +version: 0.9 +module: extensions-contrib/rabbit-stream-indexing-service +license_name: Apache License version 2.0 +libraries: + - com.rabbitmq: stream-client + +--- + name: swagger-annotations version: 1.6.0 license_category: binary diff --git a/pom.xml b/pom.xml index 81cb00bb0cf56..a8021fded26e8 100644 --- a/pom.xml +++ b/pom.xml @@ -229,6 +229,7 @@ extensions-contrib/opentelemetry-emitter extensions-contrib/kubernetes-overlord-extensions extensions-contrib/druid-iceberg-extensions + extensions-contrib/rabbit-stream-indexing-service distribution