mirror of https://github.com/apache/druid.git
Merge branch 'quidem-record' into quidem-msq
This commit is contained in:
commit
090f937d58
|
@ -23,7 +23,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.jackson.AggregatorsModule;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
|
@ -68,7 +68,7 @@ public class GroupByDeserializationBenchmark
|
|||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
AggregatorsModule.registerComplexMetricsAndSerde();
|
||||
}
|
||||
|
||||
|
@ -93,7 +93,7 @@ public class GroupByDeserializationBenchmark
|
|||
public void setup() throws JsonProcessingException
|
||||
{
|
||||
final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper();
|
||||
undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
undecoratedMapper.registerModules(BuiltInTypesModule.getJacksonModulesList());
|
||||
undecoratedMapper.registerModule(new AggregatorsModule());
|
||||
final Pair<GroupByQuery, String> sqlQueryAndResultRow = sqlQueryAndResultRow(
|
||||
numDimensions,
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.druid.frame.processor.FrameProcessorExecutor;
|
|||
import org.apache.druid.frame.read.FrameReader;
|
||||
import org.apache.druid.frame.testutil.FrameSequenceBuilder;
|
||||
import org.apache.druid.frame.write.FrameWriters;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.NonnullPair;
|
||||
|
@ -85,7 +85,7 @@ public class FrameChannelMergerBenchmark
|
|||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private static final String KEY = "key";
|
||||
|
|
|
@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableSet;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
|
@ -90,7 +90,7 @@ public class SqlGroupByBenchmark
|
|||
static {
|
||||
NullHandling.initializeForTests();
|
||||
ExpressionProcessing.initializeForTests();
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private static final Logger log = new Logger(SqlGroupByBenchmark.class);
|
||||
|
@ -332,7 +332,7 @@ public class SqlGroupByBenchmark
|
|||
|
||||
// Hacky and pollutes global namespace, but it is fine since benchmarks are run in isolation. Wasn't able
|
||||
// to work up a cleaner way of doing it by modifying the injector.
|
||||
CalciteTests.getJsonMapper().registerModules(NestedDataModule.getJacksonModulesList());
|
||||
CalciteTests.getJsonMapper().registerModules(BuiltInTypesModule.getJacksonModulesList());
|
||||
|
||||
final DruidSchemaCatalog rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
|
|
|
@ -629,10 +629,21 @@ Retrieves information about the query associated with the given query ID. The re
|
|||
- `sizeInBytes`: the size of the page.
|
||||
- `id`: the page number that you can use to reference a specific page when you get query results.
|
||||
|
||||
If the optional query parameter `detail` is supplied, then the response also includes the following:
|
||||
- A `stages` object that summarizes information about the different stages being used for query execution, such as stage number, phase, start time, duration, input and output information, processing methods, and partitioning.
|
||||
- A `counters` object that provides details on the rows, bytes, and files processed at various stages for each worker across different channels, along with sort progress.
|
||||
- A `warnings` object that provides details about any warnings.
|
||||
|
||||
#### URL
|
||||
|
||||
`GET` `/druid/v2/sql/statements/{queryId}`
|
||||
|
||||
#### Query parameters
|
||||
* `detail` (optional)
|
||||
* Type: Boolean
|
||||
* Default: false
|
||||
* Fetch additional details about the query, which includes the information about different stages, counters for each stage, and any warnings.
|
||||
|
||||
#### Responses
|
||||
|
||||
<Tabs>
|
||||
|
@ -672,7 +683,7 @@ The following example retrieves the status of a query with specified ID `query-9
|
|||
|
||||
|
||||
```shell
|
||||
curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804"
|
||||
curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true"
|
||||
```
|
||||
|
||||
</TabItem>
|
||||
|
@ -680,7 +691,7 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-4
|
|||
|
||||
|
||||
```HTTP
|
||||
GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804 HTTP/1.1
|
||||
GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804?detail=true HTTP/1.1
|
||||
Host: http://ROUTER_IP:ROUTER_PORT
|
||||
```
|
||||
|
||||
|
@ -835,7 +846,422 @@ Host: http://ROUTER_IP:ROUTER_PORT
|
|||
"sizeInBytes": 375
|
||||
}
|
||||
]
|
||||
},
|
||||
"stages": [
|
||||
{
|
||||
"stageNumber": 0,
|
||||
"definition": {
|
||||
"id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_0",
|
||||
"input": [
|
||||
{
|
||||
"type": "table",
|
||||
"dataSource": "wikipedia",
|
||||
"intervals": [
|
||||
"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
|
||||
],
|
||||
"filter": {
|
||||
"type": "equals",
|
||||
"column": "user",
|
||||
"matchValueType": "STRING",
|
||||
"matchValue": "BlueMoon2662"
|
||||
},
|
||||
"filterFields": [
|
||||
"user"
|
||||
]
|
||||
}
|
||||
],
|
||||
"processor": {
|
||||
"type": "scan",
|
||||
"query": {
|
||||
"queryType": "scan",
|
||||
"dataSource": {
|
||||
"type": "inputNumber",
|
||||
"inputNumber": 0
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
"intervals": [
|
||||
"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
|
||||
]
|
||||
},
|
||||
"virtualColumns": [
|
||||
{
|
||||
"type": "expression",
|
||||
"name": "v0",
|
||||
"expression": "'BlueMoon2662'",
|
||||
"outputType": "STRING"
|
||||
}
|
||||
],
|
||||
"resultFormat": "compactedList",
|
||||
"limit": 1001,
|
||||
"filter": {
|
||||
"type": "equals",
|
||||
"column": "user",
|
||||
"matchValueType": "STRING",
|
||||
"matchValue": "BlueMoon2662"
|
||||
},
|
||||
"columns": [
|
||||
"__time",
|
||||
"added",
|
||||
"channel",
|
||||
"cityName",
|
||||
"comment",
|
||||
"commentLength",
|
||||
"countryIsoCode",
|
||||
"countryName",
|
||||
"deleted",
|
||||
"delta",
|
||||
"deltaBucket",
|
||||
"diffUrl",
|
||||
"flags",
|
||||
"isAnonymous",
|
||||
"isMinor",
|
||||
"isNew",
|
||||
"isRobot",
|
||||
"isUnpatrolled",
|
||||
"metroCode",
|
||||
"namespace",
|
||||
"page",
|
||||
"regionIsoCode",
|
||||
"regionName",
|
||||
"v0"
|
||||
],
|
||||
"context": {
|
||||
"__resultFormat": "array",
|
||||
"__user": "allowAll",
|
||||
"enableWindowing": true,
|
||||
"executionMode": "async",
|
||||
"finalize": true,
|
||||
"maxNumTasks": 2,
|
||||
"maxParseExceptions": 0,
|
||||
"queryId": "33b53acb-7533-4880-a81b-51c16c489eab",
|
||||
"scanSignature": "[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"commentLength\",\"type\":\"LONG\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"deltaBucket\",\"type\":\"LONG\"},{\"name\":\"diffUrl\",\"type\":\"STRING\"},{\"name\":\"flags\",\"type\":\"STRING\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"STRING\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"STRING\"}]",
|
||||
"sqlOuterLimit": 1001,
|
||||
"sqlQueryId": "33b53acb-7533-4880-a81b-51c16c489eab",
|
||||
"sqlStringifyArrays": false
|
||||
},
|
||||
"columnTypes": [
|
||||
"LONG",
|
||||
"LONG",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"LONG",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"LONG",
|
||||
"LONG",
|
||||
"LONG",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING",
|
||||
"STRING"
|
||||
],
|
||||
"granularity": {
|
||||
"type": "all"
|
||||
},
|
||||
"legacy": false
|
||||
}
|
||||
},
|
||||
"signature": [
|
||||
{
|
||||
"name": "__boost",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "__time",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "added",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "channel",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "cityName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "comment",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "commentLength",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "countryIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "countryName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "deleted",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "delta",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "deltaBucket",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "diffUrl",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "flags",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isAnonymous",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isMinor",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isNew",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isRobot",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isUnpatrolled",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "metroCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "namespace",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "page",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "v0",
|
||||
"type": "STRING"
|
||||
}
|
||||
],
|
||||
"shuffleSpec": {
|
||||
"type": "mix"
|
||||
},
|
||||
"maxWorkerCount": 1
|
||||
},
|
||||
"phase": "FINISHED",
|
||||
"workerCount": 1,
|
||||
"partitionCount": 1,
|
||||
"shuffle": "mix",
|
||||
"output": "localStorage",
|
||||
"startTime": "2024-07-31T15:20:21.255Z",
|
||||
"duration": 103
|
||||
},
|
||||
{
|
||||
"stageNumber": 1,
|
||||
"definition": {
|
||||
"id": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804_1",
|
||||
"input": [
|
||||
{
|
||||
"type": "stage",
|
||||
"stage": 0
|
||||
}
|
||||
],
|
||||
"processor": {
|
||||
"type": "limit",
|
||||
"limit": 1001
|
||||
},
|
||||
"signature": [
|
||||
{
|
||||
"name": "__boost",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "__time",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "added",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "channel",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "cityName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "comment",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "commentLength",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "countryIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "countryName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "deleted",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "delta",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "deltaBucket",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "diffUrl",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "flags",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isAnonymous",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isMinor",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isNew",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isRobot",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "isUnpatrolled",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "metroCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "namespace",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "page",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "v0",
|
||||
"type": "STRING"
|
||||
}
|
||||
],
|
||||
"shuffleSpec": {
|
||||
"type": "maxCount",
|
||||
"clusterBy": {
|
||||
"columns": [
|
||||
{
|
||||
"columnName": "__boost",
|
||||
"order": "ASCENDING"
|
||||
}
|
||||
]
|
||||
},
|
||||
"partitions": 1
|
||||
},
|
||||
"maxWorkerCount": 1
|
||||
},
|
||||
"phase": "FINISHED",
|
||||
"workerCount": 1,
|
||||
"partitionCount": 1,
|
||||
"shuffle": "globalSort",
|
||||
"output": "localStorage",
|
||||
"startTime": "2024-07-31T15:20:21.355Z",
|
||||
"duration": 10,
|
||||
"sort": true
|
||||
}
|
||||
],
|
||||
"counters": {
|
||||
"0": {
|
||||
"0": {
|
||||
"input0": {
|
||||
"type": "channel",
|
||||
"rows": [
|
||||
24433
|
||||
],
|
||||
"bytes": [
|
||||
7393933
|
||||
],
|
||||
"files": [
|
||||
22
|
||||
],
|
||||
"totalFiles": [
|
||||
22
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"1": {
|
||||
"0": {
|
||||
"sortProgress": {
|
||||
"type": "sortProgress",
|
||||
"totalMergingLevels": -1,
|
||||
"levelToTotalBatches": {},
|
||||
"levelToMergedBatches": {},
|
||||
"totalMergersForUltimateLevel": -1,
|
||||
"triviallyComplete": true,
|
||||
"progressDigest": 1
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"warnings": []
|
||||
}
|
||||
```
|
||||
</details>
|
||||
|
|
|
@ -311,7 +311,7 @@ See [Enabling metrics](../configuration/index.md#enabling-metrics) for more deta
|
|||
|
||||
## Coordination
|
||||
|
||||
These metrics are for the Druid Coordinator and are reset each time the Coordinator runs the coordination logic.
|
||||
These metrics are emitted by the Druid Coordinator in every run of the corresponding coordinator duty.
|
||||
|
||||
|Metric|Description|Dimensions|Normal value|
|
||||
|------|-----------|----------|------------|
|
||||
|
@ -325,6 +325,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|
|||
|`segment/dropSkipped/count`|Number of segments that could not be dropped from any server.|`dataSource`, `tier`, `description`|Varies|
|
||||
|`segment/loadQueue/size`|Size in bytes of segments to load.|`server`|Varies|
|
||||
|`segment/loadQueue/count`|Number of segments to load.|`server`|Varies|
|
||||
|`segment/loading/rateKbps`|Current rate of segment loading on a server in kbps (1000 bits per second). The rate is calculated as a moving average over the last 10 GiB or more of successful segment loads on that server.|`server`|Varies|
|
||||
|`segment/dropQueue/count`|Number of segments to drop.|`server`|Varies|
|
||||
|`segment/loadQueue/assigned`|Number of segments assigned for load or drop to the load queue of a server.|`dataSource`, `server`|Varies|
|
||||
|`segment/loadQueue/success`|Number of segment assignments that completed successfully.|`dataSource`, `server`|Varies|
|
||||
|
|
|
@ -396,67 +396,188 @@ Performs a bitwise XOR operation on all input values.
|
|||
|
||||
## BITWISE_AND
|
||||
|
||||
`BITWISE_AND(expr1, expr2)`
|
||||
Returns the bitwise AND between two expressions: `expr1 & expr2`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_AND(expr1, expr2)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns the bitwise AND between the two expressions, that is, `expr1 & expr2`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise AND operation `12 & 10`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_AND(12, 10) AS "bitwise_and"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_and` |
|
||||
| -- |
|
||||
| 8 |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_COMPLEMENT
|
||||
|
||||
`BITWISE_COMPLEMENT(expr)`
|
||||
Returns the bitwise complement (bitwise not) for the expression: `~expr`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_COMPLEMENT(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns the bitwise NOT for the expression, that is, `~expr`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise complement operation `~12`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_COMPLEMENT(12) AS "bitwise_complement"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_complement` |
|
||||
| -- |
|
||||
| -13 |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_CONVERT_DOUBLE_TO_LONG_BITS
|
||||
|
||||
`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)`
|
||||
Converts the bits of an IEEE 754 floating-point double value to long.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:**`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns the IEEE 754 floating-point double representation of `255` as a long.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(255) AS "ieee_754_double_to_long"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `ieee_754_double_to_long` |
|
||||
| -- |
|
||||
| `4643176031446892544` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
Converts the bits of an IEEE 754 floating-point double value to a long.
|
||||
|
||||
## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE
|
||||
|
||||
`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Converts a long to the IEEE 754 floating-point double specified by the bits stored in the long.
|
||||
|
||||
* **Syntax:**`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns the long representation of `4643176031446892544` as an IEEE 754 floating-point double.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(4643176031446892544) AS "long_to_ieee_754_double"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `long_to_ieee_754_double` |
|
||||
| -- |
|
||||
| `255` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_OR
|
||||
|
||||
`BITWISE_OR(expr1, expr2)`
|
||||
Returns the bitwise OR between the two expressions: `expr1 | expr2`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_OR(expr1, expr2)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns the bitwise OR between the two expressions, that is, `expr1 | expr2`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise OR operation `12 | 10`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_OR(12, 10) AS "bitwise_or"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_or` |
|
||||
| -- |
|
||||
| `14` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_SHIFT_LEFT
|
||||
|
||||
`BITWISE_SHIFT_LEFT(expr1, expr2)`
|
||||
Returns the bitwise left shift by x positions of an expr: `expr << x`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_SHIFT_LEFT(expr, x)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns a bitwise left shift of expr1, that is, `expr1 << expr2`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise SHIFT operation `2 << 3`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_SHIFT_LEFT(2, 3) AS "bitwise_shift_left"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_shift_left` |
|
||||
| -- |
|
||||
| `16` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_SHIFT_RIGHT
|
||||
|
||||
`BITWISE_SHIFT_RIGHT(expr1, expr2)`
|
||||
Returns the bitwise right shift by x positions of an expr: `expr >> x`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_SHIFT_RIGHT(expr, x)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns a bitwise right shift of expr1, that is, `expr1 >> expr2`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise SHIFT operation `16 >> 3`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_SHIFT_RIGHT(16, 3) AS "bitwise_shift_right"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_shift_right` |
|
||||
| -- |
|
||||
| `2` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BITWISE_XOR
|
||||
|
||||
`BITWISE_XOR(expr1, expr2)`
|
||||
Returns the bitwise exclusive OR between the two expressions: `expr1 ^ expr2`.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `BITWISE_XOR(expr1, expr2)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Returns the bitwise exclusive OR between the two expressions, that is, `expr1 ^ expr2`.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example performs the bitwise XOR operation `12 ^ 10`.
|
||||
|
||||
```sql
|
||||
SELECT BITWISE_XOR(12, 10) AS "bitwise_xor"
|
||||
```
|
||||
Returns the following:
|
||||
|
||||
| `bitwise_xor` |
|
||||
| -- |
|
||||
| `6` |
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## BLOOM_FILTER
|
||||
|
||||
|
@ -710,12 +831,37 @@ Returns the rank for a row within a window without gaps. For example, if two row
|
|||
|
||||
## DIV
|
||||
|
||||
`DIV(x, y)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Returns the result of integer division of `x` by `y`.
|
||||
|
||||
* **Syntax:** `DIV(x, y)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<!--
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following calculates integer divisions of `78` by `10`.
|
||||
|
||||
```sql
|
||||
SELECT DIV(78, 10) as "division"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `division` |
|
||||
| -- |
|
||||
| `7` |
|
||||
|
||||
</details>
|
||||
-->
|
||||
|
||||
:::info
|
||||
|
||||
The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md/#safe_divide) instead.
|
||||
|
||||
:::
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## DS_CDF
|
||||
|
||||
`DS_CDF(expr, splitPoint0, splitPoint1, ...)`
|
||||
|
@ -971,28 +1117,78 @@ Returns a union of HLL sketches.
|
|||
|
||||
## HUMAN_READABLE_BINARY_BYTE_FORMAT
|
||||
|
||||
`HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])`
|
||||
Converts an integer byte size into human-readable [IEC](https://en.wikipedia.org/wiki/Binary_prefix) format.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Converts an integer byte size into human-readable IEC format.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts `1000000` into IEC format.
|
||||
|
||||
```sql
|
||||
SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(1000000, 2) AS "iec_format"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `iec_format` |
|
||||
| -- |
|
||||
| `976.56 KiB` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## HUMAN_READABLE_DECIMAL_BYTE_FORMAT
|
||||
|
||||
`HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])`
|
||||
Converts a byte size into human-readable [SI](https://en.wikipedia.org/wiki/Binary_prefix) format.
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
* **Syntax:** `HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
Converts a byte size into human-readable SI format.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts `1000000` into SI format.
|
||||
|
||||
```sql
|
||||
SELECT HUMAN_READABLE_DECIMAL_BYTE_FORMAT(1000000, 2) AS "si_format"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
|`si_format`|
|
||||
|--|
|
||||
|`1.00 MB`|
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## HUMAN_READABLE_DECIMAL_FORMAT
|
||||
|
||||
`HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Converts a byte size into human-readable SI format with single-character units.
|
||||
|
||||
* **Syntax:** `HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts `1000000` into single character SI format.
|
||||
|
||||
```sql
|
||||
SELECT HUMAN_READABLE_DECIMAL_FORMAT(1000000, 2) AS "single_character_si_format"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
|`single_character_si_format`|
|
||||
|--|
|
||||
|`1.00 M`|
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## ICONTAINS_STRING
|
||||
|
||||
`ICONTAINS_STRING(<expr>, str)`
|
||||
|
@ -1619,12 +1815,29 @@ Trims characters from the trailing end of an expression.
|
|||
|
||||
## SAFE_DIVIDE
|
||||
|
||||
`SAFE_DIVIDE(x, y)`
|
||||
|
||||
**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
|
||||
|
||||
Returns `x` divided by `y`, guarded on division by 0.
|
||||
|
||||
* **Syntax:** `SAFE_DIVIDE(x, y)`
|
||||
* **Function type:** Scalar, numeric
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example calculates divisions of integer `78` by integer `10`.
|
||||
|
||||
```sql
|
||||
SELECT SAFE_DIVIDE(78, 10) AS "safe_division"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
|`safe_division`|
|
||||
|--|
|
||||
| `7` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#numeric-functions)
|
||||
|
||||
## SIN
|
||||
|
||||
Calculates the trigonometric sine of an angle expressed in radians.
|
||||
|
|
|
@ -33,8 +33,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
|
|||
* key, and timestamp.
|
||||
* <p>
|
||||
* NOTE: Any records with null values will be skipped, even if they contain non-null keys, or headers
|
||||
* <p>
|
||||
* This functionality is not yet exposed through any built-in InputFormats, but is available for use in extensions.
|
||||
*/
|
||||
public class KafkaRecordEntity extends ByteEntity
|
||||
{
|
||||
|
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.data.input.InputEntity;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Kinesis aware InputFormat. Allows for reading kinesis specific values that are stored in the {@link Record}. At
|
||||
* this time, this input format only supports reading data from the following record components
|
||||
* <p>
|
||||
* - {@link Record#data}
|
||||
* - {@link Record#approximateArrivalTimestamp}
|
||||
* - {@link Record#partitionKey}
|
||||
* <p>
|
||||
* This class can be extended easily to read other fields available in the kinesis record.
|
||||
*/
|
||||
public class KinesisInputFormat implements InputFormat
|
||||
{
|
||||
private static final String DEFAULT_TIMESTAMP_COLUMN_NAME = "kinesis.timestamp";
|
||||
private static final String DEFAULT_PARTITION_KEY_COLUMN_NAME = "kinesis.partitionKey";
|
||||
|
||||
// Since KinesisInputFormat blends data from record properties, and payload, timestamp spec can be pointing to an
|
||||
// attribute within one of these 2 sections. To handle scenarios where there is no timestamp value in the payload, we
|
||||
// induce an artificial timestamp value to avoid unnecessary parser barf out. Users in such situations can use the
|
||||
// inputFormat's kinesis record timestamp as its primary timestamp.
|
||||
public static final String DEFAULT_AUTO_TIMESTAMP_STRING = "__kif_auto_timestamp";
|
||||
private final TimestampSpec dummyTimestampSpec = new TimestampSpec(DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH);
|
||||
|
||||
private final InputFormat valueFormat;
|
||||
private final String timestampColumnName;
|
||||
private final String partitionKeyColumnName;
|
||||
|
||||
public KinesisInputFormat(
|
||||
@JsonProperty("valueFormat") InputFormat valueFormat,
|
||||
@JsonProperty("partitionKeyColumnName") @Nullable String partitionKeyColumnName,
|
||||
@JsonProperty("timestampColumnName") @Nullable String timestampColumnName
|
||||
)
|
||||
{
|
||||
this.valueFormat = Preconditions.checkNotNull(valueFormat, "valueFormat must not be null");
|
||||
Preconditions.checkState(
|
||||
!(timestampColumnName != null && timestampColumnName.equals(partitionKeyColumnName)),
|
||||
"timestampColumnName and partitionKeyColumnName must be different"
|
||||
);
|
||||
this.partitionKeyColumnName = partitionKeyColumnName != null
|
||||
? partitionKeyColumnName
|
||||
: DEFAULT_PARTITION_KEY_COLUMN_NAME;
|
||||
this.timestampColumnName = timestampColumnName != null ? timestampColumnName : DEFAULT_TIMESTAMP_COLUMN_NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSplittable()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
|
||||
{
|
||||
final SettableByteEntity<KinesisRecordEntity> settableByteEntitySource;
|
||||
if (source instanceof SettableByteEntity) {
|
||||
settableByteEntitySource = (SettableByteEntity<KinesisRecordEntity>) source;
|
||||
} else {
|
||||
settableByteEntitySource = new SettableByteEntity<>();
|
||||
settableByteEntitySource.setEntity((KinesisRecordEntity) source);
|
||||
}
|
||||
InputRowSchema newInputRowSchema = new InputRowSchema(
|
||||
dummyTimestampSpec,
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
inputRowSchema.getColumnsFilter(),
|
||||
inputRowSchema.getMetricNames()
|
||||
);
|
||||
return new KinesisInputReader(
|
||||
inputRowSchema,
|
||||
settableByteEntitySource,
|
||||
JsonInputFormat.withLineSplittable(valueFormat, false).createReader(
|
||||
newInputRowSchema,
|
||||
source,
|
||||
temporaryDirectory
|
||||
),
|
||||
partitionKeyColumnName,
|
||||
timestampColumnName
|
||||
);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public InputFormat getValueFormat()
|
||||
{
|
||||
return valueFormat;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public String getTimestampColumnName()
|
||||
{
|
||||
return timestampColumnName;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public String getPartitionKeyColumnName()
|
||||
{
|
||||
return partitionKeyColumnName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
KinesisInputFormat that = (KinesisInputFormat) o;
|
||||
return Objects.equals(valueFormat, that.valueFormat)
|
||||
&& Objects.equals(timestampColumnName, that.timestampColumnName)
|
||||
&& Objects.equals(partitionKeyColumnName, that.partitionKeyColumnName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(valueFormat, timestampColumnName, partitionKeyColumnName);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,256 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.InputRowListPlusRawValues;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.MapInputRowParser;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.CloseableIterators;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class KinesisInputReader implements InputEntityReader
|
||||
{
|
||||
|
||||
private final InputRowSchema inputRowSchema;
|
||||
private final SettableByteEntity<KinesisRecordEntity> source;
|
||||
private final InputEntityReader valueParser;
|
||||
private final String partitionKeyColumnName;
|
||||
private final String timestampColumnName;
|
||||
|
||||
public KinesisInputReader(
|
||||
InputRowSchema inputRowSchema,
|
||||
SettableByteEntity<KinesisRecordEntity> source,
|
||||
InputEntityReader valueParser,
|
||||
String partitionKeyColumnName,
|
||||
String timestampColumnName
|
||||
)
|
||||
{
|
||||
this.inputRowSchema = inputRowSchema;
|
||||
this.source = source;
|
||||
this.valueParser = valueParser;
|
||||
this.partitionKeyColumnName = partitionKeyColumnName;
|
||||
this.timestampColumnName = timestampColumnName;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloseableIterator<InputRow> read() throws IOException
|
||||
{
|
||||
final KinesisRecordEntity record = source.getEntity();
|
||||
final Map<String, Object> mergedHeaderMap = extractHeaders(record);
|
||||
|
||||
if (record.getRecord().getData() != null) {
|
||||
return buildBlendedRows(valueParser, mergedHeaderMap);
|
||||
} else {
|
||||
return CloseableIterators.withEmptyBaggage(buildInputRowsForMap(mergedHeaderMap).iterator());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
|
||||
{
|
||||
final KinesisRecordEntity record = source.getEntity();
|
||||
InputRowListPlusRawValues headers = extractHeaderSample(record);
|
||||
if (record.getRecord().getData() != null) {
|
||||
return buildBlendedRowsSample(valueParser, headers.getRawValues());
|
||||
} else {
|
||||
final List<InputRowListPlusRawValues> rows = Collections.singletonList(headers);
|
||||
return CloseableIterators.withEmptyBaggage(rows.iterator());
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, Object> extractHeaders(KinesisRecordEntity record)
|
||||
{
|
||||
final Map<String, Object> mergedHeaderMap = new HashMap<>();
|
||||
mergedHeaderMap.put(timestampColumnName, record.getRecord().getApproximateArrivalTimestamp().getTime());
|
||||
mergedHeaderMap.put(partitionKeyColumnName, record.getRecord().getPartitionKey());
|
||||
return mergedHeaderMap;
|
||||
}
|
||||
|
||||
private CloseableIterator<InputRow> buildBlendedRows(
|
||||
InputEntityReader valueParser,
|
||||
Map<String, Object> headerKeyList
|
||||
) throws IOException
|
||||
{
|
||||
return valueParser.read().map(
|
||||
r -> {
|
||||
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
|
||||
final Map<String, Object> event = buildBlendedEventMap(r::getRaw, newDimensions, headerKeyList);
|
||||
newDimensions.addAll(headerKeyList.keySet());
|
||||
// Remove the dummy timestamp added in KinesisInputFormat
|
||||
newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
|
||||
|
||||
final DateTime timestamp = MapInputRowParser.parseTimestamp(inputRowSchema.getTimestampSpec(), event);
|
||||
return new MapBasedInputRow(
|
||||
timestamp,
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
newDimensions
|
||||
),
|
||||
event
|
||||
);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private InputRowListPlusRawValues extractHeaderSample(KinesisRecordEntity record)
|
||||
{
|
||||
Map<String, Object> mergedHeaderMap = extractHeaders(record);
|
||||
return InputRowListPlusRawValues.of(buildInputRowsForMap(mergedHeaderMap), mergedHeaderMap);
|
||||
}
|
||||
|
||||
private CloseableIterator<InputRowListPlusRawValues> buildBlendedRowsSample(
|
||||
InputEntityReader valueParser,
|
||||
Map<String, Object> headerKeyList
|
||||
) throws IOException
|
||||
{
|
||||
return valueParser.sample().map(
|
||||
rowAndValues -> {
|
||||
if (rowAndValues.getParseException() != null) {
|
||||
return rowAndValues;
|
||||
}
|
||||
List<InputRow> newInputRows = Lists.newArrayListWithCapacity(rowAndValues.getInputRows().size());
|
||||
List<Map<String, Object>> newRawRows = Lists.newArrayListWithCapacity(rowAndValues.getRawValues().size());
|
||||
|
||||
for (Map<String, Object> raw : rowAndValues.getRawValuesList()) {
|
||||
newRawRows.add(buildBlendedEventMap(raw::get, raw.keySet(), headerKeyList));
|
||||
}
|
||||
for (InputRow r : rowAndValues.getInputRows()) {
|
||||
if (r != null) {
|
||||
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
|
||||
final Map<String, Object> event = buildBlendedEventMap(
|
||||
r::getRaw,
|
||||
newDimensions,
|
||||
headerKeyList
|
||||
);
|
||||
newDimensions.addAll(headerKeyList.keySet());
|
||||
// Remove the dummy timestamp added in KinesisInputFormat
|
||||
newDimensions.remove(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
|
||||
newInputRows.add(
|
||||
new MapBasedInputRow(
|
||||
inputRowSchema.getTimestampSpec().extractTimestamp(event),
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
newDimensions
|
||||
),
|
||||
event
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, null);
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private List<InputRow> buildInputRowsForMap(Map<String, Object> headerKeyList)
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new MapBasedInputRow(
|
||||
inputRowSchema.getTimestampSpec().extractTimestamp(headerKeyList),
|
||||
MapInputRowParser.findDimensions(
|
||||
inputRowSchema.getTimestampSpec(),
|
||||
inputRowSchema.getDimensionsSpec(),
|
||||
headerKeyList.keySet()
|
||||
),
|
||||
headerKeyList
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private Map<String, Object> buildBlendedEventMap(
|
||||
Function<String, Object> getRowValue,
|
||||
Set<String> rowDimensions,
|
||||
Map<String, Object> fallback
|
||||
)
|
||||
{
|
||||
final Set<String> keySet = new HashSet<>(fallback.keySet());
|
||||
keySet.addAll(rowDimensions);
|
||||
|
||||
return new AbstractMap<String, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object get(Object key)
|
||||
{
|
||||
final String skey = (String) key;
|
||||
final Object val = getRowValue.apply(skey);
|
||||
if (val == null) {
|
||||
return fallback.get(skey);
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> keySet()
|
||||
{
|
||||
return keySet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<String, Object>> entrySet()
|
||||
{
|
||||
return keySet().stream()
|
||||
.map(
|
||||
field -> new Entry<String, Object>()
|
||||
{
|
||||
@Override
|
||||
public String getKey()
|
||||
{
|
||||
return field;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getValue()
|
||||
{
|
||||
return get(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object setValue(final Object value)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
)
|
||||
.collect(Collectors.toCollection(LinkedHashSet::new));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.indexing.kinesis.KinesisRecordSupplier;
|
||||
|
||||
/**
|
||||
* A {@link ByteEntity} generated by {@link KinesisRecordSupplier} and fed to any {@link InputFormat} used by kinesis
|
||||
* indexing tasks.
|
||||
* <p>
|
||||
* It can be used as a regular ByteEntity, in which case the kinesis record value is returned, but the {@link #getRecord}
|
||||
* method also allows Kinesis-aware {@link InputFormat} implementations to read the full kinesis record, including
|
||||
* timestamp, encrytion key, patition key, and sequence number
|
||||
* <p>
|
||||
* NOTE: Any records with null values will be returned as records with just only kinesis properties and no data payload
|
||||
*/
|
||||
public class KinesisRecordEntity extends ByteEntity
|
||||
{
|
||||
private final Record record;
|
||||
|
||||
public KinesisRecordEntity(Record record)
|
||||
{
|
||||
super(record.getData());
|
||||
this.record = record;
|
||||
}
|
||||
|
||||
public Record getRecord()
|
||||
{
|
||||
return record;
|
||||
}
|
||||
}
|
|
@ -27,7 +27,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.inject.name.Named;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
|
@ -46,7 +46,7 @@ import java.util.Collections;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, ByteEntity>
|
||||
public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final String TYPE = "index_kinesis";
|
||||
|
||||
|
@ -100,7 +100,7 @@ public class KinesisIndexTask extends SeekableStreamIndexTask<String, String, By
|
|||
}
|
||||
|
||||
@Override
|
||||
protected SeekableStreamIndexTaskRunner<String, String, ByteEntity> createTaskRunner()
|
||||
protected SeekableStreamIndexTaskRunner<String, String, KinesisRecordEntity> createTaskRunner()
|
||||
{
|
||||
//noinspection unchecked
|
||||
return new KinesisIndexTaskRunner(
|
||||
|
|
|
@ -21,8 +21,8 @@ package org.apache.druid.indexing.kinesis;
|
|||
|
||||
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.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.common.LockGranularity;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.seekablestream.SeekableStreamDataSourceMetadata;
|
||||
|
@ -49,7 +49,7 @@ import java.util.Set;
|
|||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String, ByteEntity>
|
||||
public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisIndexTaskRunner.class);
|
||||
private static final long POLL_TIMEOUT = 100;
|
||||
|
@ -81,8 +81,8 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
protected List<OrderedPartitionableRecord<String, String, ByteEntity>> getRecords(
|
||||
RecordSupplier<String, String, ByteEntity> recordSupplier, TaskToolbox toolbox
|
||||
protected List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> getRecords(
|
||||
RecordSupplier<String, String, KinesisRecordEntity> recordSupplier, TaskToolbox toolbox
|
||||
)
|
||||
{
|
||||
return recordSupplier.poll(POLL_TIMEOUT);
|
||||
|
@ -119,7 +119,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner<String
|
|||
@Override
|
||||
protected void possiblyResetDataSourceMetadata(
|
||||
TaskToolbox toolbox,
|
||||
RecordSupplier<String, String, ByteEntity> recordSupplier,
|
||||
RecordSupplier<String, String, KinesisRecordEntity> recordSupplier,
|
||||
Set<StreamPartition<String>> assignment
|
||||
)
|
||||
{
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.inject.Binder;
|
||||
import com.google.inject.name.Names;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.data.input.kinesis.KinesisInputFormat;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig;
|
||||
|
@ -50,7 +51,8 @@ public class KinesisIndexingServiceModule implements DruidModule
|
|||
new NamedType(KinesisIndexTaskIOConfig.class, SCHEME),
|
||||
new NamedType(KinesisSupervisorTuningConfig.class, SCHEME),
|
||||
new NamedType(KinesisSupervisorSpec.class, SCHEME),
|
||||
new NamedType(KinesisSamplerSpec.class, SCHEME)
|
||||
new NamedType(KinesisSamplerSpec.class, SCHEME),
|
||||
new NamedType(KinesisInputFormat.class, SCHEME)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
|
|||
import com.amazonaws.client.builder.AwsClientBuilder;
|
||||
import com.amazonaws.services.kinesis.AmazonKinesis;
|
||||
import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
|
||||
import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
|
||||
import com.amazonaws.services.kinesis.model.DescribeStreamRequest;
|
||||
import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
|
||||
import com.amazonaws.services.kinesis.model.GetRecordsRequest;
|
||||
|
@ -49,7 +50,7 @@ import com.google.common.collect.Maps;
|
|||
import org.apache.druid.common.aws.AWSClientUtil;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.aws.AWSCredentialsUtils;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
|
@ -69,7 +70,6 @@ import javax.annotation.Nullable;
|
|||
import java.lang.invoke.MethodHandle;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.lang.reflect.Method;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
@ -94,7 +94,7 @@ import java.util.stream.Collectors;
|
|||
* This class implements a local buffer for storing fetched Kinesis records. Fetching is done
|
||||
* in background threads.
|
||||
*/
|
||||
public class KinesisRecordSupplier implements RecordSupplier<String, String, ByteEntity>
|
||||
public class KinesisRecordSupplier implements RecordSupplier<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisRecordSupplier.class);
|
||||
private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000;
|
||||
|
@ -210,7 +210,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
// used for retrying on InterruptedException
|
||||
GetRecordsResult recordsResult = null;
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> currRecord;
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> currRecord;
|
||||
long recordBufferOfferWaitMillis;
|
||||
try {
|
||||
|
||||
|
@ -248,7 +248,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
// list will come back empty if there are no records
|
||||
for (Record kinesisRecord : recordsResult.getRecords()) {
|
||||
final List<ByteEntity> data;
|
||||
final List<KinesisRecordEntity> data;
|
||||
|
||||
if (deaggregateHandle == null || getDataHandle == null) {
|
||||
throw new ISE("deaggregateHandle or getDataHandle is null!");
|
||||
|
@ -256,15 +256,15 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
data = new ArrayList<>();
|
||||
|
||||
final List userRecords = (List) deaggregateHandle.invokeExact(
|
||||
final List<UserRecord> userRecords = (List<UserRecord>) deaggregateHandle.invokeExact(
|
||||
Collections.singletonList(kinesisRecord)
|
||||
);
|
||||
|
||||
int recordSize = 0;
|
||||
for (Object userRecord : userRecords) {
|
||||
ByteEntity byteEntity = new ByteEntity((ByteBuffer) getDataHandle.invoke(userRecord));
|
||||
recordSize += byteEntity.getBuffer().array().length;
|
||||
data.add(byteEntity);
|
||||
for (UserRecord userRecord : userRecords) {
|
||||
KinesisRecordEntity kinesisRecordEntity = new KinesisRecordEntity(userRecord);
|
||||
recordSize += kinesisRecordEntity.getBuffer().array().length;
|
||||
data.add(kinesisRecordEntity);
|
||||
}
|
||||
|
||||
|
||||
|
@ -408,7 +408,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
private final ConcurrentMap<StreamPartition<String>, PartitionResource> partitionResources =
|
||||
new ConcurrentHashMap<>();
|
||||
private MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, ByteEntity>> records;
|
||||
private MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records;
|
||||
|
||||
private final boolean backgroundFetchEnabled;
|
||||
private volatile boolean closed = false;
|
||||
|
@ -615,12 +615,12 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
public List<OrderedPartitionableRecord<String, String, ByteEntity>> poll(long timeout)
|
||||
public List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> poll(long timeout)
|
||||
{
|
||||
start();
|
||||
|
||||
try {
|
||||
List<MemoryBoundLinkedBlockingQueue.ObjectContainer<OrderedPartitionableRecord<String, String, ByteEntity>>> polledRecords = new ArrayList<>();
|
||||
List<MemoryBoundLinkedBlockingQueue.ObjectContainer<OrderedPartitionableRecord<String, String, KinesisRecordEntity>>> polledRecords = new ArrayList<>();
|
||||
|
||||
records.drain(
|
||||
polledRecords,
|
||||
|
@ -1040,7 +1040,7 @@ public class KinesisRecordSupplier implements RecordSupplier<String, String, Byt
|
|||
}
|
||||
|
||||
// filter records in buffer and only retain ones whose partition was not seeked
|
||||
MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, ByteEntity>> newQ =
|
||||
MemoryBoundLinkedBlockingQueue<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> newQ =
|
||||
new MemoryBoundLinkedBlockingQueue<>(recordBufferSizeBytes);
|
||||
|
||||
records.stream()
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.utils.IdUtils;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.common.task.Task;
|
||||
import org.apache.druid.indexing.common.task.TaskResource;
|
||||
import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
|
||||
|
@ -74,7 +74,7 @@ import java.util.stream.Collectors;
|
|||
* tasks to satisfy the desired number of replicas. As tasks complete, new tasks are queued to process the next range of
|
||||
* Kinesis sequences.
|
||||
*/
|
||||
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String, ByteEntity>
|
||||
public class KinesisSupervisor extends SeekableStreamSupervisor<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(KinesisSupervisor.class);
|
||||
|
||||
|
@ -150,7 +150,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<SeekableStreamIndexTask<String, String, ByteEntity>> createIndexTasks(
|
||||
protected List<SeekableStreamIndexTask<String, String, KinesisRecordEntity>> createIndexTasks(
|
||||
int replicas,
|
||||
String baseSequenceName,
|
||||
ObjectMapper sortingMapper,
|
||||
|
@ -164,7 +164,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
final Map<String, Object> context = createBaseTaskContexts();
|
||||
context.put(CHECKPOINTS_CTX_KEY, checkpoints);
|
||||
|
||||
List<SeekableStreamIndexTask<String, String, ByteEntity>> taskList = new ArrayList<>();
|
||||
List<SeekableStreamIndexTask<String, String, KinesisRecordEntity>> taskList = new ArrayList<>();
|
||||
for (int i = 0; i < replicas; i++) {
|
||||
String taskId = IdUtils.getRandomIdWithPrefix(baseSequenceName);
|
||||
taskList.add(new KinesisIndexTask(
|
||||
|
@ -183,7 +183,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String,
|
|||
|
||||
|
||||
@Override
|
||||
protected RecordSupplier<String, String, ByteEntity> setupRecordSupplier() throws RuntimeException
|
||||
protected RecordSupplier<String, String, KinesisRecordEntity> setupRecordSupplier() throws RuntimeException
|
||||
{
|
||||
KinesisSupervisorIOConfig ioConfig = spec.getIoConfig();
|
||||
KinesisIndexTaskTuningConfig taskTuningConfig = spec.getTuningConfig();
|
||||
|
|
|
@ -0,0 +1,940 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.ColumnsFilter;
|
||||
import org.apache.druid.data.input.InputEntityReader;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.InputRowListPlusRawValues;
|
||||
import org.apache.druid.data.input.InputRowSchema;
|
||||
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.parsers.CloseableIterator;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathFieldType;
|
||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
|
||||
public class KinesisInputFormatTest
|
||||
{
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
|
||||
private static final String KINESIS_APPROXIMATE_TIME_DATE = "2024-07-29";
|
||||
private static final long KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS = DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis();
|
||||
private static final String DATA_TIMSTAMP_DATE = "2024-07-30";
|
||||
private static final String PARTITION_KEY = "partition_key_1";
|
||||
|
||||
private static final byte[] SIMPLE_JSON_VALUE_BYTES = StringUtils.toUtf8(
|
||||
TestUtils.singleQuoteToStandardJson(
|
||||
"{"
|
||||
+ " 'timestamp': '" + DATA_TIMSTAMP_DATE + "',"
|
||||
+ " 'bar': null,"
|
||||
+ " 'foo': 'x',"
|
||||
+ " 'baz': 4,"
|
||||
+ " 'o': {'mg': 1}"
|
||||
+ "}"
|
||||
)
|
||||
);
|
||||
|
||||
private KinesisInputFormat format;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
format = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerde() throws JsonProcessingException
|
||||
{
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
KinesisInputFormat kif = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
Assert.assertEquals(format, kif);
|
||||
|
||||
final byte[] formatBytes = mapper.writeValueAsBytes(format);
|
||||
final byte[] kifBytes = mapper.writeValueAsBytes(kif);
|
||||
Assert.assertArrayEquals(formatBytes, kifBytes);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRawSample() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec(KinesisInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING, "auto", DateTimes.EPOCH),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRowListPlusRawValues> iterator = reader.sample()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRowListPlusRawValues rawValues = iterator.next();
|
||||
Assert.assertEquals(1, rawValues.getInputRows().size());
|
||||
InputRow row = rawValues.getInputRows().get(0);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProcessesSampleTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRowListPlusRawValues> iterator = reader.sample()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRowListPlusRawValues rawValues = iterator.next();
|
||||
Assert.assertEquals(1, rawValues.getInputRows().size());
|
||||
InputRow row = rawValues.getInputRows().get(0);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS)), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithMultipleMixedRecordsTimestampFromHeader() throws IOException
|
||||
{
|
||||
final byte[][] values = new byte[5][];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = StringUtils.toUtf8(
|
||||
"{\n"
|
||||
+ " \"timestamp\": \"2024-07-2" + i + "\",\n"
|
||||
+ " \"bar\": null,\n"
|
||||
+ " \"foo\": \"x\",\n"
|
||||
+ " \"baz\": 4,\n"
|
||||
+ " \"index\": " + i + ",\n"
|
||||
+ " \"o\": {\n"
|
||||
+ " \"mg\": 1\n"
|
||||
+ " }\n"
|
||||
+ "}"
|
||||
);
|
||||
}
|
||||
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
settableByteEntity,
|
||||
null
|
||||
);
|
||||
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(values[i], DateTimes.of("2024-07-1" + i).getMillis());
|
||||
settableByteEntity.setEntity(inputEntity);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
// Payload verification
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
Assert.assertEquals(DateTimes.of("2024-07-1" + i), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of("2024-07-1" + i).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimestampFromData() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithMultipleMixedRecordsTimestampFromData() throws IOException
|
||||
{
|
||||
final byte[][] values = new byte[5][];
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values[i] = StringUtils.toUtf8(
|
||||
"{\n"
|
||||
+ " \"timestamp\": \"2024-07-2" + i + "\",\n"
|
||||
+ " \"bar\": null,\n"
|
||||
+ " \"foo\": \"x\",\n"
|
||||
+ " \"baz\": 4,\n"
|
||||
+ " \"index\": " + i + ",\n"
|
||||
+ " \"o\": {\n"
|
||||
+ " \"mg\": 1\n"
|
||||
+ " }\n"
|
||||
+ "}"
|
||||
);
|
||||
}
|
||||
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
settableByteEntity,
|
||||
null
|
||||
);
|
||||
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
KinesisRecordEntity inputEntity = makeInputEntity(values[i], KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
settableByteEntity.setEntity(inputEntity);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
// Payload verification
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
Assert.assertEquals(DateTimes.of("2024-07-2" + i), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of("2024-07-29").getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingTimestampThrowsException() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("time", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
while (iterator.hasNext()) {
|
||||
Throwable t = Assert.assertThrows(ParseException.class, iterator::next);
|
||||
Assert.assertTrue(
|
||||
t.getMessage().startsWith("Timestamp[null] is unparseable! Event: {")
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithSchemaDiscoveryKinesisTimestampExcluded() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
DimensionsSpec.builder()
|
||||
.useSchemaDiscovery(true)
|
||||
.setDimensionExclusions(ImmutableList.of("kinesis.newts.timestamp"))
|
||||
.build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"foo",
|
||||
"root_baz",
|
||||
"o",
|
||||
"bar",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2",
|
||||
"kinesis.newts.partitionKey"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithSchemaDiscoveryTimestampFromHeader() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("kinesis.newts.timestamp", "iso", null),
|
||||
DimensionsSpec.builder()
|
||||
.useSchemaDiscovery(true)
|
||||
.build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"foo",
|
||||
"timestamp",
|
||||
"root_baz",
|
||||
"o",
|
||||
"bar",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2",
|
||||
"kinesis.newts.partitionKey"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValueInCsvFormat() throws IOException
|
||||
{
|
||||
format = new KinesisInputFormat(
|
||||
// Value Format
|
||||
new CsvInputFormat(
|
||||
Arrays.asList("foo", "bar", "timestamp", "baz"),
|
||||
null,
|
||||
false,
|
||||
false,
|
||||
0
|
||||
),
|
||||
"kinesis.newts.partitionKey",
|
||||
"kinesis.newts.timestamp"
|
||||
);
|
||||
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(StringUtils.toUtf8("x,,2024-07-30,4"), KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(
|
||||
ImmutableList.of(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey"
|
||||
)
|
||||
)
|
||||
),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
Assert.assertEquals(
|
||||
Arrays.asList(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey"
|
||||
),
|
||||
row.getDimensions()
|
||||
);
|
||||
// Payload verifications
|
||||
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
|
||||
// but test reading them anyway since it isn't technically illegal
|
||||
|
||||
Assert.assertEquals(DateTimes.of("2024-07-30"), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertTrue(row.getDimension("bar").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWithPartialDeclarationSchemaDiscovery() throws IOException
|
||||
{
|
||||
KinesisRecordEntity inputEntity =
|
||||
makeInputEntity(SIMPLE_JSON_VALUE_BYTES, KINESIS_APPROXOIMATE_TIMESTAMP_MILLIS);
|
||||
|
||||
final InputEntityReader reader = format.createReader(
|
||||
new InputRowSchema(
|
||||
new TimestampSpec("timestamp", "iso", null),
|
||||
DimensionsSpec.builder().setDimensions(
|
||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar"))
|
||||
).useSchemaDiscovery(true).build(),
|
||||
ColumnsFilter.all()
|
||||
),
|
||||
newSettableByteEntity(inputEntity),
|
||||
null
|
||||
);
|
||||
|
||||
final int numExpectedIterations = 1;
|
||||
try (CloseableIterator<InputRow> iterator = reader.read()) {
|
||||
int numActualIterations = 0;
|
||||
while (iterator.hasNext()) {
|
||||
|
||||
final InputRow row = iterator.next();
|
||||
|
||||
List<String> expectedDimensions = Arrays.asList(
|
||||
"bar",
|
||||
"foo",
|
||||
"kinesis.newts.timestamp",
|
||||
"kinesis.newts.partitionKey",
|
||||
"root_baz",
|
||||
"o",
|
||||
"path_omg",
|
||||
"jq_omg",
|
||||
"jq_omg2",
|
||||
"baz",
|
||||
"root_baz2",
|
||||
"path_omg2"
|
||||
);
|
||||
Collections.sort(expectedDimensions);
|
||||
Collections.sort(row.getDimensions());
|
||||
Assert.assertEquals(
|
||||
expectedDimensions,
|
||||
row.getDimensions()
|
||||
);
|
||||
|
||||
// Payload verifications
|
||||
Assert.assertEquals(DateTimes.of(DATA_TIMSTAMP_DATE), row.getTimestamp());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(DateTimes.of(KINESIS_APPROXIMATE_TIME_DATE).getMillis()),
|
||||
Iterables.getOnlyElement(row.getDimension("kinesis.newts.timestamp"))
|
||||
);
|
||||
Assert.assertEquals(PARTITION_KEY, Iterables.getOnlyElement(row.getDimension("kinesis.newts.partitionKey")));
|
||||
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
|
||||
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("root_baz")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("path_omg")));
|
||||
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
|
||||
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
|
||||
|
||||
Assert.assertTrue(row.getDimension("root_baz2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("path_omg2").isEmpty());
|
||||
Assert.assertTrue(row.getDimension("jq_omg2").isEmpty());
|
||||
|
||||
numActualIterations++;
|
||||
}
|
||||
|
||||
Assert.assertEquals(numExpectedIterations, numActualIterations);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public void testValidInputFormatConstruction()
|
||||
{
|
||||
InputFormat valueFormat = new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
);
|
||||
// null partitionKeyColumnName and null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, null, null);
|
||||
|
||||
// non-null partitionKeyColumnName and null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, "kinesis.partitionKey", null);
|
||||
|
||||
// null partitionKeyColumnName and non-null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, null, "kinesis.timestamp");
|
||||
|
||||
// non-null partitionKeyColumnName and non-null timestampColumnName is valid
|
||||
new KinesisInputFormat(valueFormat, "kinesis.partitionKey", "kinesis.timestamp");
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public void testInvalidInputFormatConstruction()
|
||||
{
|
||||
// null value format is invalid
|
||||
Assert.assertThrows(
|
||||
"valueFormat must not be null",
|
||||
NullPointerException.class,
|
||||
() -> new KinesisInputFormat(null, null, null)
|
||||
);
|
||||
|
||||
InputFormat valueFormat = new JsonInputFormat(
|
||||
new JSONPathSpec(
|
||||
true,
|
||||
ImmutableList.of(
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz", "baz"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.ROOT, "root_baz2", "baz2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg", "$.o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.PATH, "path_omg2", "$.o.mg2"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg", ".o.mg"),
|
||||
new JSONPathFieldSpec(JSONPathFieldType.JQ, "jq_omg2", ".o.mg2")
|
||||
)
|
||||
),
|
||||
null,
|
||||
null,
|
||||
false,
|
||||
false
|
||||
);
|
||||
|
||||
// partitionKeyColumnName == timestampColumnName is invalid
|
||||
Assert.assertThrows(
|
||||
"timestampColumnName and partitionKeyColumnName must be different",
|
||||
IllegalStateException.class,
|
||||
() -> new KinesisInputFormat(valueFormat, "kinesis.timestamp", "kinesis.timestamp")
|
||||
);
|
||||
}
|
||||
|
||||
private KinesisRecordEntity makeInputEntity(
|
||||
byte[] payload,
|
||||
long kinesisTimestampMillis)
|
||||
{
|
||||
return new KinesisRecordEntity(
|
||||
new Record().withData(ByteBuffer.wrap(payload))
|
||||
.withApproximateArrivalTimestamp(new Date(kinesisTimestampMillis))
|
||||
.withPartitionKey(PARTITION_KEY)
|
||||
);
|
||||
}
|
||||
|
||||
private SettableByteEntity<KinesisRecordEntity> newSettableByteEntity(KinesisRecordEntity kinesisRecordEntity)
|
||||
{
|
||||
SettableByteEntity<KinesisRecordEntity> settableByteEntity = new SettableByteEntity<>();
|
||||
settableByteEntity.setEntity(kinesisRecordEntity);
|
||||
return settableByteEntity;
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
@ -41,6 +42,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec;
|
|||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexer.report.IngestionStatsAndErrors;
|
||||
|
@ -127,39 +129,39 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
private static final String SHARD_ID0 = "0";
|
||||
|
||||
private static final List<KinesisRecord> RECORDS = Arrays.asList(
|
||||
createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", new ByteEntity(StringUtils.toUtf8("unparseable"))),
|
||||
createRecord("1", "7", new ByteEntity(StringUtils.toUtf8(""))),
|
||||
createRecord("1", "8", new ByteEntity(StringUtils.toUtf8("{}"))),
|
||||
createRecord("1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
||||
createRecord("1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")),
|
||||
createRecord("1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")),
|
||||
createRecord("0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")),
|
||||
createRecord("0", "1", jb("2011", "h", "y", "10", "20.0", "1.0"))
|
||||
createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", kjb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("unparseable")).getBuffer()))),
|
||||
createRecord("1", "7", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("")).getBuffer()))),
|
||||
createRecord("1", "8", new KinesisRecordEntity(new Record().withData(new ByteEntity(StringUtils.toUtf8("{}")).getBuffer()))),
|
||||
createRecord("1", "9", kjb("2013", "f", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", kjb("2049", "f", "y", "notanumber", "20.0", "1.0")),
|
||||
createRecord("1", "11", kjb("2049", "f", "y", "10", "notanumber", "1.0")),
|
||||
createRecord("1", "12", kjb("2049", "f", "y", "10", "20.0", "notanumber")),
|
||||
createRecord("0", "0", kjb("2012", "g", "y", "10", "20.0", "1.0")),
|
||||
createRecord("0", "1", kjb("2011", "h", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
|
||||
private static final List<KinesisRecord> SINGLE_PARTITION_RECORDS = Arrays.asList(
|
||||
createRecord("1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "14", jb("2013", "e", "y", "10", "20.0", "1.0"))
|
||||
createRecord("1", "0", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "1", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "2", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "3", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "4", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "5", kjb("2012", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "6", kjb("2013", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "7", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "8", kjb("2011", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "9", kjb("2011", "e", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "10", kjb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "11", kjb("2009", "b", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "12", kjb("2010", "c", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "13", kjb("2012", "d", "y", "10", "20.0", "1.0")),
|
||||
createRecord("1", "14", kjb("2013", "e", "y", "10", "20.0", "1.0"))
|
||||
);
|
||||
|
||||
private static KinesisRecordSupplier recordSupplier;
|
||||
|
@ -272,12 +274,12 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
record.getPartitionId(),
|
||||
record.getSequenceNumber(),
|
||||
record.getData().stream()
|
||||
.map(entity -> new ByteEntity(entity.getBuffer()))
|
||||
.map(entity -> new KinesisRecordEntity(new Record().withData(entity.getBuffer())))
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> clone(
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> clone(
|
||||
List<KinesisRecord> records,
|
||||
int start,
|
||||
int end
|
||||
|
@ -289,14 +291,14 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
/**
|
||||
* Records can only be read once, hence we must use fresh records every time.
|
||||
*/
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> clone(
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> clone(
|
||||
List<KinesisRecord> records
|
||||
)
|
||||
{
|
||||
return records.stream().map(KinesisIndexTaskTest::clone).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private static KinesisRecord createRecord(String partitionId, String sequenceNumber, ByteEntity entity)
|
||||
private static KinesisRecord createRecord(String partitionId, String sequenceNumber, KinesisRecordEntity entity)
|
||||
{
|
||||
return new KinesisRecord(STREAM, partitionId, sequenceNumber, Collections.singletonList(entity));
|
||||
}
|
||||
|
@ -1697,7 +1699,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
maxRowsPerSegment = 2;
|
||||
maxRecordsPerPoll = 1;
|
||||
maxBytesPerPoll = 1_000_000;
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> records =
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records =
|
||||
clone(SINGLE_PARTITION_RECORDS);
|
||||
|
||||
recordSupplier.assign(EasyMock.anyObject());
|
||||
|
@ -2148,7 +2150,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString());
|
||||
EasyMock.expectLastCall().anyTimes();
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> eosRecord = ImmutableList.of(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> eosRecord = ImmutableList.of(
|
||||
new OrderedPartitionableRecord<>(STREAM, SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER, null)
|
||||
);
|
||||
|
||||
|
@ -2454,6 +2456,18 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
return task.getRunner().getStatus() == SeekableStreamIndexTaskRunner.Status.READING;
|
||||
}
|
||||
|
||||
private static KinesisRecordEntity kjb(
|
||||
String timestamp,
|
||||
String dim1,
|
||||
String dim2,
|
||||
String dimLong,
|
||||
String dimFloat,
|
||||
String met1
|
||||
)
|
||||
{
|
||||
return new KinesisRecordEntity(new Record().withData(jb(timestamp, dim1, dim2, dimLong, dimFloat, met1).getBuffer()));
|
||||
}
|
||||
|
||||
@JsonTypeName("index_kinesis")
|
||||
private static class TestableKinesisIndexTask extends KinesisIndexTask
|
||||
{
|
||||
|
@ -2497,15 +2511,15 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
/**
|
||||
* Utility class to keep the test code more readable.
|
||||
*/
|
||||
private static class KinesisRecord extends OrderedPartitionableRecord<String, String, ByteEntity>
|
||||
private static class KinesisRecord extends OrderedPartitionableRecord<String, String, KinesisRecordEntity>
|
||||
{
|
||||
private final List<ByteEntity> data;
|
||||
private final List<KinesisRecordEntity> data;
|
||||
|
||||
public KinesisRecord(
|
||||
String stream,
|
||||
String partitionId,
|
||||
String sequenceNumber,
|
||||
List<ByteEntity> data
|
||||
List<KinesisRecordEntity> data
|
||||
)
|
||||
{
|
||||
super(stream, partitionId, sequenceNumber, data);
|
||||
|
@ -2514,7 +2528,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
|
|||
|
||||
@Nonnull
|
||||
@Override
|
||||
public List<ByteEntity> getData()
|
||||
public List<KinesisRecordEntity> getData()
|
||||
{
|
||||
return data;
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
|
||||
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -99,16 +100,13 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
new Record().withData(jb("2012", "g", "y", "10", "20.0", "1.0")).withSequenceNumber("8"),
|
||||
new Record().withData(jb("2011", "h", "y", "10", "20.0", "1.0")).withSequenceNumber("9")
|
||||
);
|
||||
private static final List<OrderedPartitionableRecord<String, String, ByteEntity>> ALL_RECORDS = ImmutableList.<OrderedPartitionableRecord<String, String, ByteEntity>>builder()
|
||||
private static final List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> ALL_RECORDS = ImmutableList.<OrderedPartitionableRecord<String, String, KinesisRecordEntity>>builder()
|
||||
.addAll(SHARD0_RECORDS.stream()
|
||||
.map(x -> new OrderedPartitionableRecord<>(
|
||||
STREAM,
|
||||
SHARD_ID0,
|
||||
x.getSequenceNumber(),
|
||||
Collections
|
||||
.singletonList(
|
||||
new ByteEntity(
|
||||
x.getData()))
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer())))
|
||||
))
|
||||
.collect(
|
||||
Collectors
|
||||
|
@ -118,14 +116,9 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
STREAM,
|
||||
SHARD_ID1,
|
||||
x.getSequenceNumber(),
|
||||
Collections
|
||||
.singletonList(
|
||||
new ByteEntity(
|
||||
x.getData()))
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(new ByteEntity(x.getData()).getBuffer())))
|
||||
))
|
||||
.collect(
|
||||
Collectors
|
||||
.toList()))
|
||||
.collect(Collectors.toList()))
|
||||
.build();
|
||||
|
||||
|
||||
|
@ -316,7 +309,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
// filter out EOS markers
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> cleanRecords(List<OrderedPartitionableRecord<String, String, ByteEntity>> records)
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> cleanRecords(List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> records)
|
||||
{
|
||||
return records.stream()
|
||||
.filter(x -> !x.getSequenceNumber()
|
||||
|
@ -398,7 +391,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -457,7 +450,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
Assert.assertFalse(recordSupplier.isAnyFetchActive());
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -531,7 +524,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
@ -687,7 +680,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> firstRecord = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
|
||||
Assert.assertEquals(
|
||||
ALL_RECORDS.get(7),
|
||||
|
@ -705,7 +698,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
|
||||
OrderedPartitionableRecord<String, String, ByteEntity> record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
OrderedPartitionableRecord<String, String, KinesisRecordEntity> record2 = recordSupplier.poll(POLL_TIMEOUT_MILLIS).get(0);
|
||||
|
||||
Assert.assertEquals(ALL_RECORDS.get(9), record2);
|
||||
// only one partition in this test. second results come from getRecordsResult0, which has SHARD0_LAG_MILLIS
|
||||
|
@ -776,7 +769,7 @@ public class KinesisRecordSupplierTest extends EasyMockSupport
|
|||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
List<OrderedPartitionableRecord<String, String, ByteEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> polledRecords = cleanRecords(recordSupplier.poll(
|
||||
POLL_TIMEOUT_MILLIS));
|
||||
|
||||
verifyAll();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexing.kinesis;
|
||||
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
@ -27,7 +28,6 @@ import org.apache.druid.client.indexing.SamplerResponse;
|
|||
import org.apache.druid.client.indexing.SamplerSpec;
|
||||
import org.apache.druid.common.aws.AWSCredentialsConfig;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
|
@ -37,6 +37,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema;
|
|||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig;
|
||||
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
|
||||
|
@ -63,6 +64,7 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
|
@ -99,7 +101,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
|
||||
private final KinesisRecordSupplier recordSupplier = mock(KinesisRecordSupplier.class);
|
||||
|
||||
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> generateRecords(String stream)
|
||||
private static List<OrderedPartitionableRecord<String, String, KinesisRecordEntity>> generateRecords(String stream)
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
|
||||
|
@ -115,9 +117,9 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
stream,
|
||||
"1",
|
||||
"6",
|
||||
Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable")))
|
||||
Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("unparseable")))))
|
||||
),
|
||||
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}"))))
|
||||
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(StringUtils.toUtf8("{}"))))))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -428,10 +430,10 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
Assert.assertFalse(it.hasNext());
|
||||
}
|
||||
|
||||
private static List<ByteEntity> jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
private static List<KinesisRecordEntity> jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
{
|
||||
try {
|
||||
return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes(
|
||||
return Collections.singletonList(new KinesisRecordEntity(new Record().withData(ByteBuffer.wrap(new ObjectMapper().writeValueAsBytes(
|
||||
ImmutableMap.builder()
|
||||
.put("timestamp", ts)
|
||||
.put("dim1", dim1)
|
||||
|
@ -440,7 +442,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
|
|||
.put("dimFloat", dimFloat)
|
||||
.put("met1", met1)
|
||||
.build()
|
||||
)));
|
||||
)))));
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
|
|
|
@ -27,12 +27,12 @@ import com.google.common.collect.ImmutableSet;
|
|||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.data.input.InputFormat;
|
||||
import org.apache.druid.data.input.impl.ByteEntity;
|
||||
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.data.input.kinesis.KinesisRecordEntity;
|
||||
import org.apache.druid.indexer.TaskLocation;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskInfoProvider;
|
||||
|
@ -5656,7 +5656,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
|
|||
}
|
||||
|
||||
@Override
|
||||
protected RecordSupplier<String, String, ByteEntity> setupRecordSupplier()
|
||||
protected RecordSupplier<String, String, KinesisRecordEntity> setupRecordSupplier()
|
||||
{
|
||||
return supervisorRecordSupplier;
|
||||
}
|
||||
|
|
|
@ -108,4 +108,14 @@ public class CounterSnapshotsTree
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
synchronized (snapshotsMap) {
|
||||
return "CounterSnapshotsTree{" +
|
||||
"snapshotsMap=" + snapshotsMap +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,9 +117,9 @@ public interface Controller
|
|||
);
|
||||
|
||||
/**
|
||||
* Returns the current list of task ids, ordered by worker number. The Nth task has worker number N.
|
||||
* Returns the current list of worker IDs, ordered by worker number. The Nth worker has worker number N.
|
||||
*/
|
||||
List<String> getTaskIds();
|
||||
List<String> getWorkerIds();
|
||||
|
||||
@Nullable
|
||||
TaskReport.ReportMap liveReports();
|
||||
|
|
|
@ -23,20 +23,25 @@ import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
|||
import org.apache.druid.msq.indexing.error.MSQErrorReport;
|
||||
import org.apache.druid.msq.kernel.StageDefinition;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.PartialKeyStatisticsInformation;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Client for the multi-stage query controller. Used by a Worker task.
|
||||
* Client for the multi-stage query controller. Used by a {@link Worker}. Each instance is specific to a single query,
|
||||
* meaning it communicates with a single controller.
|
||||
*/
|
||||
public interface ControllerClient extends AutoCloseable
|
||||
public interface ControllerClient extends Closeable
|
||||
{
|
||||
/**
|
||||
* Client side method to update the controller with partial key statistics information for a particular stage and worker.
|
||||
* Controller's implementation collates all the information for a stage to fetch key statistics from workers.
|
||||
* Client side method to update the controller with partial key statistics information for a particular stage
|
||||
* and worker. The controller collates all the information for a stage to fetch key statistics from workers.
|
||||
*
|
||||
* Only used when {@link StageDefinition#mustGatherResultKeyStatistics()}.
|
||||
*/
|
||||
void postPartialKeyStatistics(
|
||||
StageId stageId,
|
||||
|
@ -86,11 +91,16 @@ public interface ControllerClient extends AutoCloseable
|
|||
/**
|
||||
* Client side method to inform the controller about the warnings generated by the given worker.
|
||||
*/
|
||||
void postWorkerWarning(
|
||||
List<MSQErrorReport> MSQErrorReports
|
||||
) throws IOException;
|
||||
void postWorkerWarning(List<MSQErrorReport> MSQErrorReports) throws IOException;
|
||||
|
||||
List<String> getTaskList() throws IOException;
|
||||
/**
|
||||
* Client side method for retrieving the list of worker IDs from the controller. These IDs can be passed to
|
||||
* {@link WorkerClient} methods to communicate with other workers. Not necessary when the {@link WorkOrder} has
|
||||
* {@link WorkOrder#getWorkerIds()} set.
|
||||
*
|
||||
* @see Controller#getWorkerIds() for the controller side
|
||||
*/
|
||||
List<String> getWorkerIds() throws IOException;
|
||||
|
||||
/**
|
||||
* Close this client. Idempotent.
|
||||
|
|
|
@ -1171,7 +1171,7 @@ public class ControllerImpl implements Controller
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<String> getTaskIds()
|
||||
public List<String> getWorkerIds()
|
||||
{
|
||||
if (workerManager == null) {
|
||||
return Collections.emptyList();
|
||||
|
@ -1260,7 +1260,7 @@ public class ControllerImpl implements Controller
|
|||
{
|
||||
// Sorted copy of target worker numbers to ensure consistent iteration order.
|
||||
final List<Integer> workersCopy = Ordering.natural().sortedCopy(workers);
|
||||
final List<String> workerIds = getTaskIds();
|
||||
final List<String> workerIds = getWorkerIds();
|
||||
final List<ListenableFuture<Void>> workerFutures = new ArrayList<>(workersCopy.size());
|
||||
|
||||
try {
|
||||
|
@ -1488,7 +1488,7 @@ public class ControllerImpl implements Controller
|
|||
private CounterSnapshotsTree getCountersFromAllTasks()
|
||||
{
|
||||
final CounterSnapshotsTree retVal = new CounterSnapshotsTree();
|
||||
final List<String> taskList = getTaskIds();
|
||||
final List<String> taskList = getWorkerIds();
|
||||
|
||||
final List<ListenableFuture<CounterSnapshotsTree>> futures = new ArrayList<>();
|
||||
|
||||
|
@ -1508,7 +1508,7 @@ public class ControllerImpl implements Controller
|
|||
|
||||
private void postFinishToAllTasks()
|
||||
{
|
||||
final List<String> taskList = getTaskIds();
|
||||
final List<String> taskList = getWorkerIds();
|
||||
|
||||
final List<ListenableFuture<Void>> futures = new ArrayList<>();
|
||||
|
||||
|
@ -2963,7 +2963,7 @@ public class ControllerImpl implements Controller
|
|||
}
|
||||
|
||||
final StageId finalStageId = queryKernel.getStageId(queryDef.getFinalStageDefinition().getStageNumber());
|
||||
final List<String> taskIds = getTaskIds();
|
||||
final List<String> taskIds = getWorkerIds();
|
||||
|
||||
final InputChannelFactory inputChannelFactory;
|
||||
|
||||
|
|
|
@ -91,7 +91,8 @@ public class ControllerMemoryParameters
|
|||
memoryIntrospector.totalMemoryInJvm(),
|
||||
usableMemoryInJvm,
|
||||
numControllersInJvm,
|
||||
memoryIntrospector.numProcessorsInJvm()
|
||||
memoryIntrospector.numProcessorsInJvm(),
|
||||
0
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* 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.msq.exec;
|
||||
|
||||
import org.apache.druid.frame.processor.OutputChannel;
|
||||
import org.apache.druid.frame.processor.OutputChannelFactory;
|
||||
import org.apache.druid.frame.processor.PartitionedOutputChannel;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Decorator for {@link OutputChannelFactory} that notifies a {@link Listener} whenever a channel is opened.
|
||||
*/
|
||||
public class ListeningOutputChannelFactory implements OutputChannelFactory
|
||||
{
|
||||
private final OutputChannelFactory delegate;
|
||||
private final Listener listener;
|
||||
|
||||
public ListeningOutputChannelFactory(final OutputChannelFactory delegate, final Listener listener)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputChannel openChannel(final int partitionNumber) throws IOException
|
||||
{
|
||||
return notifyListener(delegate.openChannel(partitionNumber));
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public OutputChannel openNilChannel(final int partitionNumber)
|
||||
{
|
||||
return notifyListener(delegate.openNilChannel(partitionNumber));
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartitionedOutputChannel openPartitionedChannel(
|
||||
final String name,
|
||||
final boolean deleteAfterRead
|
||||
)
|
||||
{
|
||||
throw new UnsupportedOperationException("Listening to partitioned channels is not supported");
|
||||
}
|
||||
|
||||
private OutputChannel notifyListener(OutputChannel channel)
|
||||
{
|
||||
listener.channelOpened(channel);
|
||||
return channel;
|
||||
}
|
||||
|
||||
public interface Listener
|
||||
{
|
||||
void channelOpened(OutputChannel channel);
|
||||
}
|
||||
}
|
|
@ -32,9 +32,12 @@ import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils;
|
|||
public enum OutputChannelMode
|
||||
{
|
||||
/**
|
||||
* In-memory output channels. Stage shuffle data does not hit disk. This mode requires a consumer stage to run
|
||||
* at the same time as its corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the
|
||||
* logic that determines when we can use in-memory channels.
|
||||
* In-memory output channels. Stage shuffle data does not hit disk. In-memory channels do not fully buffer stage
|
||||
* output. They use a blocking queue; see {@link RunWorkOrder#makeStageOutputChannelFactory()}.
|
||||
*
|
||||
* Because stage output is not fully buffered, this mode requires a consumer stage to run at the same time as its
|
||||
* corresponding producer stage. See {@link ControllerQueryKernelUtils#computeStageGroups} for the logic that
|
||||
* determines when we can use in-memory channels.
|
||||
*/
|
||||
MEMORY("memory"),
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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.msq.exec;
|
||||
|
||||
import org.apache.druid.frame.processor.OutputChannel;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Listener for various things that may happen during execution of {@link RunWorkOrder#start()}. Listener methods are
|
||||
* fired in processing threads, so they must be thread-safe, and it is important that they run quickly.
|
||||
*/
|
||||
public interface RunWorkOrderListener
|
||||
{
|
||||
/**
|
||||
* Called when done reading input. If key statistics were gathered, they are provided.
|
||||
*/
|
||||
void onDoneReadingInput(@Nullable ClusterByStatisticsSnapshot snapshot);
|
||||
|
||||
/**
|
||||
* Called when an output channel becomes available for reading by downstream stages.
|
||||
*/
|
||||
void onOutputChannelAvailable(OutputChannel outputChannel);
|
||||
|
||||
/**
|
||||
* Called when the work order has succeeded.
|
||||
*/
|
||||
void onSuccess(Object resultObject);
|
||||
|
||||
/**
|
||||
* Called when a non-fatal exception is encountered. Work continues after this listener fires.
|
||||
*/
|
||||
void onWarning(Throwable t);
|
||||
|
||||
/**
|
||||
* Called when the work order has failed.
|
||||
*/
|
||||
void onFailure(Throwable t);
|
||||
}
|
|
@ -19,40 +19,44 @@
|
|||
|
||||
package org.apache.druid.msq.exec;
|
||||
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
||||
import org.apache.druid.msq.indexing.MSQWorkerTask;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* Interface for a multi-stage query (MSQ) worker. Workers are long-lived and are able to run multiple {@link WorkOrder}
|
||||
* prior to exiting.
|
||||
*
|
||||
* @see WorkerImpl the production implementation
|
||||
*/
|
||||
public interface Worker
|
||||
{
|
||||
/**
|
||||
* Unique ID for this worker.
|
||||
* Identifier for this worker. Same as {@link WorkerContext#workerId()}.
|
||||
*/
|
||||
String id();
|
||||
|
||||
/**
|
||||
* The task which this worker runs.
|
||||
* Runs the worker in the current thread. Surrounding classes provide the execution thread.
|
||||
*/
|
||||
MSQWorkerTask task();
|
||||
void run();
|
||||
|
||||
/**
|
||||
* Runs the worker in the current thread. Surrounding classes provide
|
||||
* the execution thread.
|
||||
* Terminate the worker upon a cancellation request. Causes a concurrently-running {@link #run()} method in
|
||||
* a separate thread to cancel all outstanding work and exit. Does not block. Use {@link #awaitStop()} if you
|
||||
* would like to wait for {@link #run()} to finish.
|
||||
*/
|
||||
TaskStatus run() throws Exception;
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* Terminate the worker upon a cancellation request.
|
||||
* Wait for {@link #run()} to finish.
|
||||
*/
|
||||
void stopGracefully();
|
||||
void awaitStop();
|
||||
|
||||
/**
|
||||
* Report that the controller has failed. The worker must cease work immediately. Cleanup then exit.
|
||||
|
@ -63,20 +67,20 @@ public interface Worker
|
|||
// Controller-to-worker, and worker-to-worker messages
|
||||
|
||||
/**
|
||||
* Called when the worker chat handler receives a request for a work order. Accepts the work order and schedules it for
|
||||
* execution
|
||||
* Called when the worker receives a new work order. Accepts the work order and schedules it for execution.
|
||||
*/
|
||||
void postWorkOrder(WorkOrder workOrder);
|
||||
|
||||
/**
|
||||
* Returns the statistics snapshot for the given stageId. This is called from {@link WorkerSketchFetcher} under
|
||||
* PARALLEL OR AUTO modes.
|
||||
* {@link ClusterStatisticsMergeMode#PARALLEL} OR {@link ClusterStatisticsMergeMode#AUTO} modes.
|
||||
*/
|
||||
ClusterByStatisticsSnapshot fetchStatisticsSnapshot(StageId stageId);
|
||||
|
||||
/**
|
||||
* Returns the statistics snapshot for the given stageId which contains only the sketch for the specified timeChunk.
|
||||
* This is called from {@link WorkerSketchFetcher} under SEQUENTIAL OR AUTO modes.
|
||||
* This is called from {@link WorkerSketchFetcher} under {@link ClusterStatisticsMergeMode#SEQUENTIAL} or
|
||||
* {@link ClusterStatisticsMergeMode#AUTO} modes.
|
||||
*/
|
||||
ClusterByStatisticsSnapshot fetchStatisticsSnapshotForTimeChunk(StageId stageId, long timeChunk);
|
||||
|
||||
|
@ -84,26 +88,30 @@ public interface Worker
|
|||
* Called when the worker chat handler recieves the result partition boundaries for a particular stageNumber
|
||||
* and queryId
|
||||
*/
|
||||
boolean postResultPartitionBoundaries(
|
||||
ClusterByPartitions stagePartitionBoundaries,
|
||||
String queryId,
|
||||
int stageNumber
|
||||
);
|
||||
boolean postResultPartitionBoundaries(StageId stageId, ClusterByPartitions stagePartitionBoundaries);
|
||||
|
||||
/**
|
||||
* Returns an InputStream of the worker output for a particular queryId, stageNumber and partitionNumber.
|
||||
* Offset indicates the number of bytes to skip the channel data, and is used to prevent re-reading the same data
|
||||
* during retry in case of a connection error
|
||||
* during retry in case of a connection error.
|
||||
*
|
||||
* Returns a null if the workerOutput for a particular queryId, stageNumber, and partitionNumber is not found.
|
||||
* The returned future resolves when at least one byte of data is available, or when the channel is finished.
|
||||
* If the channel is finished, an empty {@link InputStream} is returned.
|
||||
*
|
||||
* @throws IOException when the worker output is found but there is an error while reading it.
|
||||
* With {@link OutputChannelMode#MEMORY}, once this method is called with a certain offset, workers are free to
|
||||
* delete data prior to that offset. (Already-requested offsets will not be re-requested, because
|
||||
* {@link OutputChannelMode#MEMORY} requires a single reader.) In this mode, if an already-requested offset is
|
||||
* re-requested for some reason, an error future is returned.
|
||||
*
|
||||
* The returned future resolves to null if stage output for a particular queryId, stageNumber, and
|
||||
* partitionNumber is not found.
|
||||
*
|
||||
* Throws an exception when worker output is found, but there is an error while reading it.
|
||||
*/
|
||||
@Nullable
|
||||
InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset) throws IOException;
|
||||
ListenableFuture<InputStream> readStageOutput(StageId stageId, int partitionNumber, long offset);
|
||||
|
||||
/**
|
||||
* Returns the snapshot of the worker counters
|
||||
* Returns a snapshot of counters.
|
||||
*/
|
||||
CounterSnapshotsTree getCounters();
|
||||
|
||||
|
@ -115,7 +123,7 @@ public interface Worker
|
|||
void postCleanupStage(StageId stageId);
|
||||
|
||||
/**
|
||||
* Called when the work required for the query has been finished
|
||||
* Called when the worker is no longer needed, and should shut down.
|
||||
*/
|
||||
void postFinish();
|
||||
}
|
||||
|
|
|
@ -21,11 +21,12 @@ package org.apache.druid.msq.exec;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.msq.indexing.MSQWorkerTask;
|
||||
import org.apache.druid.msq.kernel.FrameContext;
|
||||
import org.apache.druid.msq.kernel.FrameProcessorFactory;
|
||||
import org.apache.druid.msq.kernel.QueryDefinition;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
|
||||
import java.io.File;
|
||||
|
@ -33,10 +34,21 @@ import java.io.File;
|
|||
/**
|
||||
* Context used by multi-stage query workers.
|
||||
*
|
||||
* Useful because it allows test fixtures to provide their own implementations.
|
||||
* Each context is scoped to a {@link Worker} and is shared across all {@link WorkOrder} run by that worker.
|
||||
*/
|
||||
public interface WorkerContext
|
||||
{
|
||||
/**
|
||||
* Query ID for this context.
|
||||
*/
|
||||
String queryId();
|
||||
|
||||
/**
|
||||
* Identifier for this worker that enables the controller, and other workers, to find it. For tasks this is the
|
||||
* task ID from {@link MSQWorkerTask#getId()}. For persistent servers, this is the server URI.
|
||||
*/
|
||||
String workerId();
|
||||
|
||||
ObjectMapper jsonMapper();
|
||||
|
||||
// Using an Injector directly because tasks do not have a way to provide their own Guice modules.
|
||||
|
@ -49,9 +61,15 @@ public interface WorkerContext
|
|||
void registerWorker(Worker worker, Closer closer);
|
||||
|
||||
/**
|
||||
* Creates and fetches the controller client for the provided controller ID.
|
||||
* Maximum number of {@link WorkOrder} that a {@link Worker} with this context will be asked to execute
|
||||
* simultaneously.
|
||||
*/
|
||||
ControllerClient makeControllerClient(String controllerId);
|
||||
int maxConcurrentStages();
|
||||
|
||||
/**
|
||||
* Creates a controller client.
|
||||
*/
|
||||
ControllerClient makeControllerClient();
|
||||
|
||||
/**
|
||||
* Creates and fetches a {@link WorkerClient}. It is independent of the workerId because the workerId is passed
|
||||
|
@ -60,24 +78,24 @@ public interface WorkerContext
|
|||
WorkerClient makeWorkerClient();
|
||||
|
||||
/**
|
||||
* Fetch a directory for temporary outputs
|
||||
* Directory for temporary outputs.
|
||||
*/
|
||||
File tempDir();
|
||||
|
||||
FrameContext frameContext(QueryDefinition queryDef, int stageNumber);
|
||||
/**
|
||||
* Create a context with useful objects required by {@link FrameProcessorFactory#makeProcessors}.
|
||||
*/
|
||||
FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode);
|
||||
|
||||
/**
|
||||
* Number of available processing threads.
|
||||
*/
|
||||
int threadCount();
|
||||
|
||||
/**
|
||||
* Fetch node info about self
|
||||
* Fetch node info about self.
|
||||
*/
|
||||
DruidNode selfNode();
|
||||
|
||||
Bouncer processorBouncer();
|
||||
DataServerQueryHandlerFactory dataServerQueryHandlerFactory();
|
||||
|
||||
default File tempDir(int stageNumber, String id)
|
||||
{
|
||||
return new File(StringUtils.format("%s/stage_%02d/%s", tempDir(), stageNumber, id));
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -168,29 +168,14 @@ public class WorkerMemoryParameters
|
|||
this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a production instance for {@link org.apache.druid.msq.indexing.MSQControllerTask}.
|
||||
*/
|
||||
public static WorkerMemoryParameters createProductionInstanceForController(final Injector injector)
|
||||
{
|
||||
long totalLookupFootprint = computeTotalLookupFootprint(injector);
|
||||
return createInstance(
|
||||
Runtime.getRuntime().maxMemory(),
|
||||
computeNumWorkersInJvm(injector),
|
||||
computeNumProcessorsInJvm(injector),
|
||||
0,
|
||||
0,
|
||||
totalLookupFootprint
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a production instance for {@link org.apache.druid.msq.indexing.MSQWorkerTask}.
|
||||
*/
|
||||
public static WorkerMemoryParameters createProductionInstanceForWorker(
|
||||
final Injector injector,
|
||||
final QueryDefinition queryDef,
|
||||
final int stageNumber
|
||||
final int stageNumber,
|
||||
final int maxConcurrentStages
|
||||
)
|
||||
{
|
||||
final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber);
|
||||
|
@ -212,6 +197,7 @@ public class WorkerMemoryParameters
|
|||
Runtime.getRuntime().maxMemory(),
|
||||
computeNumWorkersInJvm(injector),
|
||||
computeNumProcessorsInJvm(injector),
|
||||
maxConcurrentStages,
|
||||
numInputWorkers,
|
||||
numHashOutputPartitions,
|
||||
totalLookupFootprint
|
||||
|
@ -228,6 +214,7 @@ public class WorkerMemoryParameters
|
|||
* @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to
|
||||
* the task capacity.
|
||||
* @param numProcessingThreadsInJvm size of the processing thread pool in the JVM.
|
||||
* @param maxConcurrentStages maximum number of concurrent stages per worker.
|
||||
* @param numInputWorkers total number of workers across all input stages.
|
||||
* @param numHashOutputPartitions total number of output partitions, if using hash partitioning; zero if not using
|
||||
* hash partitioning.
|
||||
|
@ -237,6 +224,7 @@ public class WorkerMemoryParameters
|
|||
final long maxMemoryInJvm,
|
||||
final int numWorkersInJvm,
|
||||
final int numProcessingThreadsInJvm,
|
||||
final int maxConcurrentStages,
|
||||
final int numInputWorkers,
|
||||
final int numHashOutputPartitions,
|
||||
final long totalLookupFootprint
|
||||
|
@ -257,7 +245,8 @@ public class WorkerMemoryParameters
|
|||
);
|
||||
final long usableMemoryInJvm = computeUsableMemoryInJvm(maxMemoryInJvm, totalLookupFootprint);
|
||||
final long workerMemory = memoryPerWorker(usableMemoryInJvm, numWorkersInJvm);
|
||||
final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
|
||||
final long bundleMemory =
|
||||
memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm) / maxConcurrentStages;
|
||||
final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers);
|
||||
final long bundleMemoryForHashPartitioning = memoryNeededForHashPartitioning(numHashOutputPartitions);
|
||||
final long bundleMemoryForProcessing =
|
||||
|
@ -268,6 +257,7 @@ public class WorkerMemoryParameters
|
|||
usableMemoryInJvm,
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm,
|
||||
maxConcurrentStages,
|
||||
numHashOutputPartitions
|
||||
);
|
||||
|
||||
|
@ -281,12 +271,14 @@ public class WorkerMemoryParameters
|
|||
estimateUsableMemory(
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm,
|
||||
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels
|
||||
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels,
|
||||
maxConcurrentStages
|
||||
), totalLookupFootprint),
|
||||
maxMemoryInJvm,
|
||||
usableMemoryInJvm,
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm
|
||||
numProcessingThreadsInJvm,
|
||||
maxConcurrentStages
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -301,14 +293,16 @@ public class WorkerMemoryParameters
|
|||
calculateSuggestedMinMemoryFromUsableMemory(
|
||||
estimateUsableMemory(
|
||||
numWorkersInJvm,
|
||||
(MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE
|
||||
(MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE,
|
||||
maxConcurrentStages
|
||||
),
|
||||
totalLookupFootprint
|
||||
),
|
||||
maxMemoryInJvm,
|
||||
usableMemoryInJvm,
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm
|
||||
numProcessingThreadsInJvm,
|
||||
maxConcurrentStages
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -338,12 +332,14 @@ public class WorkerMemoryParameters
|
|||
estimateUsableMemory(
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm,
|
||||
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels
|
||||
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels,
|
||||
maxConcurrentStages
|
||||
), totalLookupFootprint),
|
||||
maxMemoryInJvm,
|
||||
usableMemoryInJvm,
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm
|
||||
numProcessingThreadsInJvm,
|
||||
maxConcurrentStages
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -352,7 +348,9 @@ public class WorkerMemoryParameters
|
|||
bundleMemoryForProcessing,
|
||||
superSorterMaxActiveProcessors,
|
||||
superSorterMaxChannelsPerProcessor,
|
||||
Ints.checkedCast(workerMemory) // 100% of worker memory is devoted to partition statistics
|
||||
|
||||
// 100% of worker memory is devoted to partition statistics
|
||||
Ints.checkedCast(workerMemory / maxConcurrentStages)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -459,18 +457,19 @@ public class WorkerMemoryParameters
|
|||
final long usableMemoryInJvm,
|
||||
final int numWorkersInJvm,
|
||||
final int numProcessingThreadsInJvm,
|
||||
final int maxConcurrentStages,
|
||||
final int numHashOutputPartitions
|
||||
)
|
||||
{
|
||||
final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
|
||||
|
||||
// Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle, while accounting for
|
||||
// memoryNeededForInputChannels + memoryNeededForHashPartitioning.
|
||||
// Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle per concurrent stage, while
|
||||
// accounting for memoryNeededForInputChannels + memoryNeededForHashPartitioning.
|
||||
final int isHashing = numHashOutputPartitions > 0 ? 1 : 0;
|
||||
return Math.max(
|
||||
0,
|
||||
Ints.checkedCast((bundleMemory - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1)
|
||||
);
|
||||
final long bundleMemoryPerStage = bundleMemory / maxConcurrentStages;
|
||||
final long maxWorkers =
|
||||
(bundleMemoryPerStage - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1;
|
||||
return Math.max(0, Ints.checkedCast(maxWorkers));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -528,7 +527,8 @@ public class WorkerMemoryParameters
|
|||
}
|
||||
|
||||
/**
|
||||
* Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory(int, int, long)}
|
||||
* Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be
|
||||
* done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory}
|
||||
*/
|
||||
private static long memoryPerBundle(
|
||||
final long usableMemoryInJvm,
|
||||
|
@ -536,6 +536,8 @@ public class WorkerMemoryParameters
|
|||
final int numProcessingThreadsInJvm
|
||||
)
|
||||
{
|
||||
// One bundle per worker + one per processor. The worker bundles are used for sorting (SuperSorter) and the
|
||||
// processing bundles are used for reading input and doing per-partition processing.
|
||||
final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
|
||||
|
||||
// Need to subtract memoryForWorkers off the top of usableMemoryInJvm, since this is reserved for
|
||||
|
@ -553,24 +555,28 @@ public class WorkerMemoryParameters
|
|||
private static long estimateUsableMemory(
|
||||
final int numWorkersInJvm,
|
||||
final int numProcessingThreadsInJvm,
|
||||
final long estimatedEachBundleMemory
|
||||
final long estimatedEachBundleMemory,
|
||||
final int maxConcurrentStages
|
||||
)
|
||||
{
|
||||
final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
|
||||
return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount);
|
||||
|
||||
return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount, maxConcurrentStages);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add overheads to the estimated bundle memoery for all the workers. Checkout {@link WorkerMemoryParameters#memoryPerWorker(long, int)}
|
||||
* for the overhead calculation outside the processing bundles.
|
||||
*/
|
||||
private static long estimateUsableMemory(final int numWorkersInJvm, final long estimatedTotalBundleMemory)
|
||||
private static long estimateUsableMemory(
|
||||
final int numWorkersInJvm,
|
||||
final long estimatedTotalBundleMemory,
|
||||
final int maxConcurrentStages
|
||||
)
|
||||
{
|
||||
|
||||
// Currently, we only add the partition stats overhead since it will be the single largest overhead per worker.
|
||||
final long estimateStatOverHeadPerWorker = PARTITION_STATS_MEMORY_MAX_BYTES;
|
||||
return estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm);
|
||||
final long requiredUsableMemory = estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm);
|
||||
return requiredUsableMemory * maxConcurrentStages;
|
||||
}
|
||||
|
||||
private static long memoryNeededForHashPartitioning(final int numOutputPartitions)
|
||||
|
|
|
@ -70,11 +70,13 @@ public class WorkerStorageParameters
|
|||
|
||||
public static WorkerStorageParameters createProductionInstance(
|
||||
final Injector injector,
|
||||
final boolean isIntermediateSuperSorterStorageEnabled
|
||||
final OutputChannelMode outputChannelMode
|
||||
)
|
||||
{
|
||||
long tmpStorageBytesPerTask = injector.getInstance(TaskConfig.class).getTmpStorageBytesPerTask();
|
||||
return createInstance(tmpStorageBytesPerTask, isIntermediateSuperSorterStorageEnabled);
|
||||
|
||||
// If durable storage is enabled, then super sorter intermediate storage should be enabled as well.
|
||||
return createInstance(tmpStorageBytesPerTask, outputChannelMode.isDurable());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
|
|
@ -20,9 +20,13 @@
|
|||
package org.apache.druid.msq.indexing;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.exec.WorkerStorageParameters;
|
||||
import org.apache.druid.msq.kernel.FrameContext;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.querykit.DataSegmentProvider;
|
||||
import org.apache.druid.query.groupby.GroupingEngine;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
|
@ -35,25 +39,31 @@ import java.io.File;
|
|||
|
||||
public class IndexerFrameContext implements FrameContext
|
||||
{
|
||||
private final StageId stageId;
|
||||
private final IndexerWorkerContext context;
|
||||
private final IndexIO indexIO;
|
||||
private final DataSegmentProvider dataSegmentProvider;
|
||||
private final WorkerMemoryParameters memoryParameters;
|
||||
private final WorkerStorageParameters storageParameters;
|
||||
private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory;
|
||||
|
||||
public IndexerFrameContext(
|
||||
StageId stageId,
|
||||
IndexerWorkerContext context,
|
||||
IndexIO indexIO,
|
||||
DataSegmentProvider dataSegmentProvider,
|
||||
DataServerQueryHandlerFactory dataServerQueryHandlerFactory,
|
||||
WorkerMemoryParameters memoryParameters
|
||||
WorkerMemoryParameters memoryParameters,
|
||||
WorkerStorageParameters storageParameters
|
||||
)
|
||||
{
|
||||
this.stageId = stageId;
|
||||
this.context = context;
|
||||
this.indexIO = indexIO;
|
||||
this.dataSegmentProvider = dataSegmentProvider;
|
||||
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
|
||||
this.memoryParameters = memoryParameters;
|
||||
this.storageParameters = storageParameters;
|
||||
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -90,7 +100,8 @@ public class IndexerFrameContext implements FrameContext
|
|||
@Override
|
||||
public File tempDir()
|
||||
{
|
||||
return context.tempDir();
|
||||
// No need to include query ID; each task handles a single query, so there is no ambiguity.
|
||||
return new File(context.tempDir(), StringUtils.format("stage_%06d", stageId.getStageNumber()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -128,4 +139,22 @@ public class IndexerFrameContext implements FrameContext
|
|||
{
|
||||
return memoryParameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bouncer processorBouncer()
|
||||
{
|
||||
return context.injector().getInstance(Bouncer.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public WorkerStorageParameters storageParameters()
|
||||
{
|
||||
return storageParameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
// Nothing to close.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,4 +52,10 @@ public class IndexerResourcePermissionMapper implements ResourcePermissionMapper
|
|||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ResourceAction> getQueryPermissions(String queryId)
|
||||
{
|
||||
return getAdminPermissions();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,9 +24,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.errorprone.annotations.concurrent.GuardedBy;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
import org.apache.druid.guice.annotations.Self;
|
||||
import org.apache.druid.guice.annotations.Smile;
|
||||
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
|
@ -35,16 +33,21 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.ControllerClient;
|
||||
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
|
||||
import org.apache.druid.msq.exec.MemoryIntrospector;
|
||||
import org.apache.druid.msq.exec.OutputChannelMode;
|
||||
import org.apache.druid.msq.exec.TaskDataSegmentProvider;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.exec.WorkerClient;
|
||||
import org.apache.druid.msq.exec.WorkerContext;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.exec.WorkerStorageParameters;
|
||||
import org.apache.druid.msq.indexing.client.IndexerControllerClient;
|
||||
import org.apache.druid.msq.indexing.client.IndexerWorkerClient;
|
||||
import org.apache.druid.msq.indexing.client.WorkerChatHandler;
|
||||
import org.apache.druid.msq.kernel.FrameContext;
|
||||
import org.apache.druid.msq.kernel.QueryDefinition;
|
||||
import org.apache.druid.msq.util.MultiStageQueryContext;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.QueryToolChestWarehouse;
|
||||
import org.apache.druid.rpc.ServiceClientFactory;
|
||||
import org.apache.druid.rpc.ServiceLocations;
|
||||
|
@ -67,37 +70,49 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
private static final long FREQUENCY_CHECK_MILLIS = 1000;
|
||||
private static final long FREQUENCY_CHECK_JITTER = 30;
|
||||
|
||||
private final MSQWorkerTask task;
|
||||
private final TaskToolbox toolbox;
|
||||
private final Injector injector;
|
||||
private final OverlordClient overlordClient;
|
||||
private final IndexIO indexIO;
|
||||
private final TaskDataSegmentProvider dataSegmentProvider;
|
||||
private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory;
|
||||
private final ServiceClientFactory clientFactory;
|
||||
|
||||
@GuardedBy("this")
|
||||
private OverlordClient overlordClient;
|
||||
private final MemoryIntrospector memoryIntrospector;
|
||||
private final int maxConcurrentStages;
|
||||
|
||||
@GuardedBy("this")
|
||||
private ServiceLocator controllerLocator;
|
||||
|
||||
public IndexerWorkerContext(
|
||||
final MSQWorkerTask task,
|
||||
final TaskToolbox toolbox,
|
||||
final Injector injector,
|
||||
final OverlordClient overlordClient,
|
||||
final IndexIO indexIO,
|
||||
final TaskDataSegmentProvider dataSegmentProvider,
|
||||
final DataServerQueryHandlerFactory dataServerQueryHandlerFactory,
|
||||
final ServiceClientFactory clientFactory
|
||||
final ServiceClientFactory clientFactory,
|
||||
final MemoryIntrospector memoryIntrospector,
|
||||
final DataServerQueryHandlerFactory dataServerQueryHandlerFactory
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.toolbox = toolbox;
|
||||
this.injector = injector;
|
||||
this.overlordClient = overlordClient;
|
||||
this.indexIO = indexIO;
|
||||
this.dataSegmentProvider = dataSegmentProvider;
|
||||
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
|
||||
this.clientFactory = clientFactory;
|
||||
this.memoryIntrospector = memoryIntrospector;
|
||||
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
|
||||
this.maxConcurrentStages = MultiStageQueryContext.getMaxConcurrentStages(QueryContext.of(task.getContext()));
|
||||
}
|
||||
|
||||
public static IndexerWorkerContext createProductionInstance(final TaskToolbox toolbox, final Injector injector)
|
||||
public static IndexerWorkerContext createProductionInstance(
|
||||
final MSQWorkerTask task,
|
||||
final TaskToolbox toolbox,
|
||||
final Injector injector
|
||||
)
|
||||
{
|
||||
final IndexIO indexIO = injector.getInstance(IndexIO.class);
|
||||
final SegmentCacheManager segmentCacheManager =
|
||||
|
@ -105,28 +120,42 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
.manufacturate(new File(toolbox.getIndexingTmpDir(), "segment-fetch"));
|
||||
final ServiceClientFactory serviceClientFactory =
|
||||
injector.getInstance(Key.get(ServiceClientFactory.class, EscalatedGlobal.class));
|
||||
final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class);
|
||||
final OverlordClient overlordClient =
|
||||
injector.getInstance(OverlordClient.class).withRetryPolicy(StandardRetryPolicy.unlimited());
|
||||
final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class));
|
||||
final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class);
|
||||
|
||||
return new IndexerWorkerContext(
|
||||
task,
|
||||
toolbox,
|
||||
injector,
|
||||
overlordClient,
|
||||
indexIO,
|
||||
new TaskDataSegmentProvider(
|
||||
toolbox.getCoordinatorClient(),
|
||||
segmentCacheManager,
|
||||
indexIO
|
||||
),
|
||||
new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO),
|
||||
serviceClientFactory,
|
||||
memoryIntrospector,
|
||||
new DataServerQueryHandlerFactory(
|
||||
toolbox.getCoordinatorClient(),
|
||||
serviceClientFactory,
|
||||
smileMapper,
|
||||
warehouse
|
||||
),
|
||||
serviceClientFactory
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String queryId()
|
||||
{
|
||||
return task.getControllerTaskId();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String workerId()
|
||||
{
|
||||
return task.getId();
|
||||
}
|
||||
|
||||
public TaskToolbox toolbox()
|
||||
{
|
||||
return toolbox;
|
||||
|
@ -147,7 +176,8 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
@Override
|
||||
public void registerWorker(Worker worker, Closer closer)
|
||||
{
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
|
||||
final WorkerChatHandler chatHandler =
|
||||
new WorkerChatHandler(worker, toolbox.getAuthorizerMapper(), task.getDataSource());
|
||||
toolbox.getChatHandlerProvider().register(worker.id(), chatHandler, false);
|
||||
closer.register(() -> toolbox.getChatHandlerProvider().unregister(worker.id()));
|
||||
closer.register(() -> {
|
||||
|
@ -161,7 +191,7 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
// Register the periodic controller checker
|
||||
final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s");
|
||||
closer.register(periodicControllerCheckerExec::shutdownNow);
|
||||
final ServiceLocator controllerLocator = makeControllerLocator(worker.task().getControllerTaskId());
|
||||
final ServiceLocator controllerLocator = makeControllerLocator(task.getControllerTaskId());
|
||||
periodicControllerCheckerExec.submit(() -> controllerCheckerRunnable(controllerLocator, worker));
|
||||
}
|
||||
|
||||
|
@ -218,15 +248,21 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
}
|
||||
|
||||
@Override
|
||||
public ControllerClient makeControllerClient(String controllerId)
|
||||
public int maxConcurrentStages()
|
||||
{
|
||||
final ServiceLocator locator = makeControllerLocator(controllerId);
|
||||
return maxConcurrentStages;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerClient makeControllerClient()
|
||||
{
|
||||
final ServiceLocator locator = makeControllerLocator(task.getControllerTaskId());
|
||||
|
||||
return new IndexerControllerClient(
|
||||
clientFactory.makeClient(
|
||||
controllerId,
|
||||
task.getControllerTaskId(),
|
||||
locator,
|
||||
new SpecificTaskRetryPolicy(controllerId, StandardRetryPolicy.unlimited())
|
||||
new SpecificTaskRetryPolicy(task.getControllerTaskId(), StandardRetryPolicy.unlimited())
|
||||
),
|
||||
jsonMapper(),
|
||||
locator
|
||||
|
@ -237,37 +273,33 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
public WorkerClient makeWorkerClient()
|
||||
{
|
||||
// Ignore workerId parameter. The workerId is passed into each method of WorkerClient individually.
|
||||
return new IndexerWorkerClient(clientFactory, makeOverlordClient(), jsonMapper());
|
||||
return new IndexerWorkerClient(clientFactory, overlordClient, jsonMapper());
|
||||
}
|
||||
|
||||
@Override
|
||||
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber)
|
||||
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode)
|
||||
{
|
||||
return new IndexerFrameContext(
|
||||
queryDef.getStageDefinition(stageNumber).getId(),
|
||||
this,
|
||||
indexIO,
|
||||
dataSegmentProvider,
|
||||
dataServerQueryHandlerFactory,
|
||||
WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber)
|
||||
WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber, maxConcurrentStages),
|
||||
WorkerStorageParameters.createProductionInstance(injector, outputChannelMode)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int threadCount()
|
||||
{
|
||||
return processorBouncer().getMaxCount();
|
||||
return memoryIntrospector.numProcessorsInJvm();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidNode selfNode()
|
||||
{
|
||||
return injector.getInstance(Key.get(DruidNode.class, Self.class));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bouncer processorBouncer()
|
||||
{
|
||||
return injector.getInstance(Bouncer.class);
|
||||
return toolbox.getDruidNode();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -276,21 +308,13 @@ public class IndexerWorkerContext implements WorkerContext
|
|||
return dataServerQueryHandlerFactory;
|
||||
}
|
||||
|
||||
private synchronized OverlordClient makeOverlordClient()
|
||||
{
|
||||
if (overlordClient == null) {
|
||||
overlordClient = injector.getInstance(OverlordClient.class)
|
||||
.withRetryPolicy(StandardRetryPolicy.unlimited());
|
||||
}
|
||||
return overlordClient;
|
||||
}
|
||||
|
||||
private synchronized ServiceLocator makeControllerLocator(final String controllerId)
|
||||
{
|
||||
if (controllerLocator == null) {
|
||||
controllerLocator = new SpecificTaskServiceLocator(controllerId, makeOverlordClient());
|
||||
controllerLocator = new SpecificTaskServiceLocator(controllerId, overlordClient);
|
||||
}
|
||||
|
||||
return controllerLocator;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -33,10 +33,13 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
|||
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||
import org.apache.druid.indexing.common.task.AbstractTask;
|
||||
import org.apache.druid.indexing.common.task.Tasks;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.exec.WorkerContext;
|
||||
import org.apache.druid.msq.exec.WorkerImpl;
|
||||
import org.apache.druid.msq.indexing.error.MSQException;
|
||||
import org.apache.druid.msq.indexing.error.MSQFaultUtils;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
@ -48,6 +51,7 @@ import java.util.Set;
|
|||
public class MSQWorkerTask extends AbstractTask
|
||||
{
|
||||
public static final String TYPE = "query_worker";
|
||||
private static final Logger log = new Logger(MSQWorkerTask.class);
|
||||
|
||||
private final String controllerTaskId;
|
||||
private final int workerNumber;
|
||||
|
@ -132,18 +136,25 @@ public class MSQWorkerTask extends AbstractTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus runTask(final TaskToolbox toolbox) throws Exception
|
||||
public TaskStatus runTask(final TaskToolbox toolbox)
|
||||
{
|
||||
final WorkerContext context = IndexerWorkerContext.createProductionInstance(toolbox, injector);
|
||||
final WorkerContext context = IndexerWorkerContext.createProductionInstance(this, toolbox, injector);
|
||||
worker = new WorkerImpl(this, context);
|
||||
return worker.run();
|
||||
|
||||
try {
|
||||
worker.run();
|
||||
return TaskStatus.success(context.workerId());
|
||||
}
|
||||
catch (MSQException e) {
|
||||
return TaskStatus.failure(context.workerId(), MSQFaultUtils.generateMessageWithErrorCode(e.getFault()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
if (worker != null) {
|
||||
worker.stopGracefully();
|
||||
worker.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -152,7 +152,7 @@ public class IndexerControllerClient implements ControllerClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<String> getTaskList() throws IOException
|
||||
public List<String> getWorkerIds() throws IOException
|
||||
{
|
||||
final BytesFullResponseHolder retVal = doRequest(
|
||||
new RequestBuilder(HttpMethod.GET, "/taskList"),
|
||||
|
|
|
@ -19,310 +19,25 @@
|
|||
|
||||
package org.apache.druid.msq.indexing.client;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.commons.lang.mutable.MutableLong;
|
||||
import org.apache.druid.frame.file.FrameFileHttpResponseHandler;
|
||||
import org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.indexing.MSQWorkerTask;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde;
|
||||
import org.apache.druid.msq.indexing.IndexerResourcePermissionMapper;
|
||||
import org.apache.druid.msq.rpc.WorkerResource;
|
||||
import org.apache.druid.segment.realtime.ChatHandler;
|
||||
import org.apache.druid.segment.realtime.ChatHandlers;
|
||||
import org.apache.druid.server.security.Action;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
import org.apache.druid.segment.realtime.ChatHandlerProvider;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
public class WorkerChatHandler implements ChatHandler
|
||||
/**
|
||||
* Subclass of {@link WorkerResource} that implements {@link ChatHandler}, suitable for registration
|
||||
* with a {@link ChatHandlerProvider}.
|
||||
*/
|
||||
public class WorkerChatHandler extends WorkerResource implements ChatHandler
|
||||
{
|
||||
private static final Logger log = new Logger(WorkerChatHandler.class);
|
||||
|
||||
/**
|
||||
* Callers must be able to store an entire chunk in memory. It can't be too large.
|
||||
*/
|
||||
private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000;
|
||||
|
||||
private final Worker worker;
|
||||
private final MSQWorkerTask task;
|
||||
private final TaskToolbox toolbox;
|
||||
|
||||
public WorkerChatHandler(TaskToolbox toolbox, Worker worker)
|
||||
{
|
||||
this.worker = worker;
|
||||
this.task = worker.task();
|
||||
this.toolbox = toolbox;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data.
|
||||
* <p>
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Path("/channels/{queryId}/{stageNumber}/{partitionNumber}")
|
||||
@Produces(MediaType.APPLICATION_OCTET_STREAM)
|
||||
public Response httpGetChannelData(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("partitionNumber") final int partitionNumber,
|
||||
@QueryParam("offset") final long offset,
|
||||
@Context final HttpServletRequest req
|
||||
public WorkerChatHandler(
|
||||
final Worker worker,
|
||||
final AuthorizerMapper authorizerMapper,
|
||||
final String dataSource
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
try {
|
||||
final InputStream inputStream = worker.readChannel(queryId, stageNumber, partitionNumber, offset);
|
||||
if (inputStream == null) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
}
|
||||
|
||||
final Response.ResponseBuilder responseBuilder = Response.ok();
|
||||
|
||||
final byte[] readBuf = new byte[8192];
|
||||
final MutableLong bytesReadTotal = new MutableLong(0L);
|
||||
final int firstRead = inputStream.read(readBuf);
|
||||
|
||||
if (firstRead == -1) {
|
||||
// Empty read means we're at the end of the channel. Set the last fetch header so the client knows this.
|
||||
inputStream.close();
|
||||
return responseBuilder
|
||||
.header(
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME,
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE
|
||||
)
|
||||
.entity(ByteArrays.EMPTY_ARRAY)
|
||||
.build();
|
||||
}
|
||||
|
||||
return Response.ok((StreamingOutput) output -> {
|
||||
try {
|
||||
int bytesReadThisCall = firstRead;
|
||||
do {
|
||||
final int bytesToWrite =
|
||||
(int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal.longValue(), bytesReadThisCall);
|
||||
output.write(readBuf, 0, bytesToWrite);
|
||||
bytesReadTotal.add(bytesReadThisCall);
|
||||
} while (bytesReadTotal.longValue() < CHANNEL_DATA_CHUNK_SIZE
|
||||
&& (bytesReadThisCall = inputStream.read(readBuf)) != -1);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
// Suppress the exception to ensure nothing gets written over the wire once we've sent a 200. The client
|
||||
// will resume from where it left off.
|
||||
log.noStackTrace().warn(
|
||||
e,
|
||||
"Error writing channel for query [%s] stage [%s] partition [%s] offset [%,d] to [%s]",
|
||||
queryId,
|
||||
stageNumber,
|
||||
partitionNumber,
|
||||
offset,
|
||||
req.getRemoteAddr()
|
||||
);
|
||||
}
|
||||
finally {
|
||||
CloseableUtils.closeAll(inputStream, output);
|
||||
}
|
||||
}).build();
|
||||
}
|
||||
catch (IOException e) {
|
||||
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/workOrder")
|
||||
public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postWorkOrder(workOrder);
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/resultPartitionBoundaries/{queryId}/{stageNumber}")
|
||||
public Response httpPostResultPartitionBoundaries(
|
||||
final ClusterByPartitions stagePartitionBoundaries,
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
if (worker.postResultPartitionBoundaries(stagePartitionBoundaries, queryId, stageNumber)) {
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
} else {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/keyStatistics/{queryId}/{stageNumber}")
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM})
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
public Response httpFetchKeyStatistics(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
ClusterByStatisticsSnapshot clusterByStatisticsSnapshot;
|
||||
StageId stageId = new StageId(queryId, stageNumber);
|
||||
try {
|
||||
clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId);
|
||||
if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_OCTET_STREAM)
|
||||
.entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot))
|
||||
.build();
|
||||
} else {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_JSON)
|
||||
.entity(clusterByStatisticsSnapshot)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
String errorMessage = StringUtils.format(
|
||||
"Invalid request for key statistics for query[%s] and stage[%d]",
|
||||
queryId,
|
||||
stageNumber
|
||||
);
|
||||
log.error(e, errorMessage);
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.<String, Object>of("error", errorMessage))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}")
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM})
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
public Response httpFetchKeyStatisticsWithSnapshot(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("timeChunk") final long timeChunk,
|
||||
@QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.READ, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
ClusterByStatisticsSnapshot snapshotForTimeChunk;
|
||||
StageId stageId = new StageId(queryId, stageNumber);
|
||||
try {
|
||||
snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk);
|
||||
if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_OCTET_STREAM)
|
||||
.entity((StreamingOutput) output -> ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk))
|
||||
.build();
|
||||
} else {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_JSON)
|
||||
.entity(snapshotForTimeChunk)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
String errorMessage = StringUtils.format(
|
||||
"Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]",
|
||||
queryId,
|
||||
stageNumber,
|
||||
timeChunk
|
||||
);
|
||||
log.error(e, errorMessage);
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.<String, Object>of("error", errorMessage))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/cleanupStage/{queryId}/{stageNumber}")
|
||||
public Response httpPostCleanupStage(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postCleanupStage(new StageId(queryId, stageNumber));
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/finish")
|
||||
public Response httpPostFinish(@Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
worker.postFinish();
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/counters")
|
||||
public Response httpGetCounters(@Context final HttpServletRequest req)
|
||||
{
|
||||
ChatHandlers.authorizationCheck(req, Action.WRITE, task.getDataSource(), toolbox.getAuthorizerMapper());
|
||||
return Response.status(Response.Status.OK).entity(worker.getCounters()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and
|
||||
* {@link #httpFetchKeyStatisticsWithSnapshot}.
|
||||
*/
|
||||
public enum SketchEncoding
|
||||
{
|
||||
/**
|
||||
* The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}.
|
||||
*/
|
||||
OCTET_STREAM,
|
||||
/**
|
||||
* The key collector is encoded as json
|
||||
*/
|
||||
JSON
|
||||
super(worker, new IndexerResourcePermissionMapper(dataSource), authorizerMapper);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.msq.indexing.error;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
|
||||
|
@ -35,6 +36,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
private final long usableMemory;
|
||||
private final int serverWorkers;
|
||||
private final int serverThreads;
|
||||
private final int maxConcurrentStages;
|
||||
|
||||
@JsonCreator
|
||||
public NotEnoughMemoryFault(
|
||||
|
@ -42,19 +44,23 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
@JsonProperty("serverMemory") final long serverMemory,
|
||||
@JsonProperty("usableMemory") final long usableMemory,
|
||||
@JsonProperty("serverWorkers") final int serverWorkers,
|
||||
@JsonProperty("serverThreads") final int serverThreads
|
||||
@JsonProperty("serverThreads") final int serverThreads,
|
||||
@JsonProperty("maxConcurrentStages") final int maxConcurrentStages
|
||||
)
|
||||
{
|
||||
super(
|
||||
CODE,
|
||||
"Not enough memory. Required at least %,d bytes. (total = %,d bytes; usable = %,d bytes; "
|
||||
+ "worker capacity = %,d; processing threads = %,d). Increase JVM memory with the -Xmx option"
|
||||
+ (serverWorkers > 1 ? " or reduce worker capacity on this server" : ""),
|
||||
+ "worker capacity = %,d; processing threads = %,d; concurrent stages = %,d). "
|
||||
+ "Increase JVM memory with the -Xmx option"
|
||||
+ (serverWorkers > 1 ? ", or reduce worker capacity on this server" : "")
|
||||
+ (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : ""),
|
||||
suggestedServerMemory,
|
||||
serverMemory,
|
||||
usableMemory,
|
||||
serverWorkers,
|
||||
serverThreads
|
||||
serverThreads,
|
||||
maxConcurrentStages
|
||||
);
|
||||
|
||||
this.suggestedServerMemory = suggestedServerMemory;
|
||||
|
@ -62,6 +68,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
this.usableMemory = usableMemory;
|
||||
this.serverWorkers = serverWorkers;
|
||||
this.serverThreads = serverThreads;
|
||||
this.maxConcurrentStages = maxConcurrentStages;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -94,6 +101,13 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
return serverThreads;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public int getMaxConcurrentStages()
|
||||
{
|
||||
return maxConcurrentStages;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -107,12 +121,12 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
return false;
|
||||
}
|
||||
NotEnoughMemoryFault that = (NotEnoughMemoryFault) o;
|
||||
return
|
||||
suggestedServerMemory == that.suggestedServerMemory
|
||||
return suggestedServerMemory == that.suggestedServerMemory
|
||||
&& serverMemory == that.serverMemory
|
||||
&& usableMemory == that.usableMemory
|
||||
&& serverWorkers == that.serverWorkers
|
||||
&& serverThreads == that.serverThreads;
|
||||
&& serverThreads == that.serverThreads
|
||||
&& maxConcurrentStages == that.maxConcurrentStages;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -124,7 +138,8 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
serverMemory,
|
||||
usableMemory,
|
||||
serverWorkers,
|
||||
serverThreads
|
||||
serverThreads,
|
||||
maxConcurrentStages
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -137,6 +152,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
|
|||
" bytes, usableMemory=" + usableMemory +
|
||||
" bytes, serverWorkers=" + serverWorkers +
|
||||
", serverThreads=" + serverThreads +
|
||||
", maxConcurrentStages=" + maxConcurrentStages +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,6 +41,22 @@ public class InputSlices
|
|||
// No instantiation.
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all {@link StageInputSlice} from the provided list of input slices. Ignores other types of input slices.
|
||||
*/
|
||||
public static List<StageInputSlice> allStageSlices(final List<InputSlice> slices)
|
||||
{
|
||||
final List<StageInputSlice> retVal = new ArrayList<>();
|
||||
|
||||
for (final InputSlice slice : slices) {
|
||||
if (slice instanceof StageInputSlice) {
|
||||
retVal.add((StageInputSlice) slice);
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
/**
|
||||
* Combines all {@link StageInputSlice#getPartitions()} from the input slices that are {@link StageInputSlice}.
|
||||
* Ignores other types of input slices.
|
||||
|
@ -49,10 +65,8 @@ public class InputSlices
|
|||
{
|
||||
final List<ReadablePartitions> partitionsList = new ArrayList<>();
|
||||
|
||||
for (final InputSlice slice : slices) {
|
||||
if (slice instanceof StageInputSlice) {
|
||||
partitionsList.add(((StageInputSlice) slice).getPartitions());
|
||||
}
|
||||
for (final StageInputSlice slice : allStageSlices(slices)) {
|
||||
partitionsList.add(slice.getPartitions());
|
||||
}
|
||||
|
||||
return ReadablePartitions.combine(partitionsList);
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec;
|
|||
import org.apache.druid.data.input.impl.InlineInputSource;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.msq.counters.ChannelCounters;
|
||||
import org.apache.druid.msq.counters.CounterNames;
|
||||
import org.apache.druid.msq.counters.CounterTracker;
|
||||
|
@ -53,6 +53,7 @@ import org.apache.druid.segment.incremental.SimpleRowIngestionMeters;
|
|||
import org.apache.druid.timeline.SegmentId;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.function.Consumer;
|
||||
|
@ -94,7 +95,7 @@ public class ExternalInputSliceReader implements InputSliceReader
|
|||
externalInputSlice.getInputSources(),
|
||||
externalInputSlice.getInputFormat(),
|
||||
externalInputSlice.getSignature(),
|
||||
temporaryDirectory,
|
||||
new File(temporaryDirectory, String.valueOf(inputNumber)),
|
||||
counters.channel(CounterNames.inputChannel(inputNumber)).setTotalFiles(slice.fileCount()),
|
||||
counters.warnings(),
|
||||
warningPublisher
|
||||
|
@ -128,9 +129,13 @@ public class ExternalInputSliceReader implements InputSliceReader
|
|||
ColumnsFilter.all()
|
||||
);
|
||||
|
||||
if (!temporaryDirectory.exists() && !temporaryDirectory.mkdir()) {
|
||||
throw new ISE("Cannot create temporary directory at [%s]", temporaryDirectory);
|
||||
try {
|
||||
FileUtils.mkdirp(temporaryDirectory);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
return Iterators.transform(
|
||||
inputSources.iterator(),
|
||||
inputSource -> {
|
||||
|
|
|
@ -20,8 +20,11 @@
|
|||
package org.apache.druid.msq.kernel;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
|
||||
import org.apache.druid.msq.exec.OutputChannelMode;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.exec.WorkerStorageParameters;
|
||||
import org.apache.druid.msq.querykit.DataSegmentProvider;
|
||||
import org.apache.druid.query.groupby.GroupingEngine;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
|
@ -30,12 +33,16 @@ import org.apache.druid.segment.SegmentWrangler;
|
|||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
|
||||
/**
|
||||
* Provides services and objects for the functioning of the frame processors
|
||||
* Provides services and objects for the functioning of the frame processors. Scoped to a specific stage of a
|
||||
* specific query, i.e., one {@link WorkOrder}.
|
||||
*
|
||||
* Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(QueryDefinition, int, OutputChannelMode)}.
|
||||
*/
|
||||
public interface FrameContext
|
||||
public interface FrameContext extends Closeable
|
||||
{
|
||||
SegmentWrangler segmentWrangler();
|
||||
|
||||
|
@ -59,5 +66,14 @@ public interface FrameContext
|
|||
|
||||
IndexMergerV9 indexMerger();
|
||||
|
||||
Bouncer processorBouncer();
|
||||
|
||||
WorkerMemoryParameters memoryParameters();
|
||||
|
||||
WorkerStorageParameters storageParameters();
|
||||
|
||||
default File tempDir(String name)
|
||||
{
|
||||
return new File(tempDir(), name);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -109,7 +109,7 @@ public class WorkOrder
|
|||
|
||||
/**
|
||||
* Worker IDs for this query, if known in advance (at the time the work order is created). May be null, in which
|
||||
* case workers use {@link ControllerClient#getTaskList()} to find worker IDs.
|
||||
* case workers use {@link ControllerClient#getWorkerIds()} to find worker IDs.
|
||||
*/
|
||||
@Nullable
|
||||
@JsonProperty("workers")
|
||||
|
|
|
@ -42,6 +42,8 @@ import java.util.Set;
|
|||
* This separation of decision-making from the "real world" allows the decision-making to live in one,
|
||||
* easy-to-follow place.
|
||||
*
|
||||
* Not thread-safe.
|
||||
*
|
||||
* @see org.apache.druid.msq.kernel.controller.ControllerQueryKernel state machine on the controller side
|
||||
*/
|
||||
public class WorkerStageKernel
|
||||
|
@ -51,9 +53,10 @@ public class WorkerStageKernel
|
|||
|
||||
private WorkerStagePhase phase = WorkerStagePhase.NEW;
|
||||
|
||||
// We read this variable in the main thread and the netty threads
|
||||
@Nullable
|
||||
private volatile ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot;
|
||||
private ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot;
|
||||
|
||||
private boolean doneReadingInput;
|
||||
|
||||
@Nullable
|
||||
private ClusterByPartitions resultPartitionBoundaries;
|
||||
|
@ -107,25 +110,25 @@ public class WorkerStageKernel
|
|||
|
||||
public void startPreshuffleWaitingForResultPartitionBoundaries()
|
||||
{
|
||||
assertPreshuffleStatisticsNeeded();
|
||||
assertPreshuffleStatisticsNeeded(true);
|
||||
transitionTo(WorkerStagePhase.PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES);
|
||||
}
|
||||
|
||||
public void startPreshuffleWritingOutput()
|
||||
{
|
||||
assertPreshuffleStatisticsNeeded();
|
||||
transitionTo(WorkerStagePhase.PRESHUFFLE_WRITING_OUTPUT);
|
||||
}
|
||||
|
||||
public void setResultKeyStatisticsSnapshot(final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot)
|
||||
public void setResultKeyStatisticsSnapshot(@Nullable final ClusterByStatisticsSnapshot resultKeyStatisticsSnapshot)
|
||||
{
|
||||
assertPreshuffleStatisticsNeeded();
|
||||
assertPreshuffleStatisticsNeeded(resultKeyStatisticsSnapshot != null);
|
||||
this.resultKeyStatisticsSnapshot = resultKeyStatisticsSnapshot;
|
||||
this.doneReadingInput = true;
|
||||
}
|
||||
|
||||
public void setResultPartitionBoundaries(final ClusterByPartitions resultPartitionBoundaries)
|
||||
{
|
||||
assertPreshuffleStatisticsNeeded();
|
||||
assertPreshuffleStatisticsNeeded(true);
|
||||
this.resultPartitionBoundaries = resultPartitionBoundaries;
|
||||
}
|
||||
|
||||
|
@ -134,6 +137,11 @@ public class WorkerStageKernel
|
|||
return resultKeyStatisticsSnapshot != null;
|
||||
}
|
||||
|
||||
public boolean isDoneReadingInput()
|
||||
{
|
||||
return doneReadingInput;
|
||||
}
|
||||
|
||||
public boolean hasResultPartitionBoundaries()
|
||||
{
|
||||
return resultPartitionBoundaries != null;
|
||||
|
@ -152,10 +160,10 @@ public class WorkerStageKernel
|
|||
@Nullable
|
||||
public Object getResultObject()
|
||||
{
|
||||
if (phase == WorkerStagePhase.RESULTS_READY || phase == WorkerStagePhase.FINISHED) {
|
||||
if (phase == WorkerStagePhase.RESULTS_COMPLETE) {
|
||||
return resultObject;
|
||||
} else {
|
||||
throw new ISE("Results are not ready yet");
|
||||
throw new ISE("Results are not ready in phase[%s]", phase);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -174,7 +182,7 @@ public class WorkerStageKernel
|
|||
throw new NullPointerException("resultObject must not be null");
|
||||
}
|
||||
|
||||
transitionTo(WorkerStagePhase.RESULTS_READY);
|
||||
transitionTo(WorkerStagePhase.RESULTS_COMPLETE);
|
||||
this.resultObject = resultObject;
|
||||
}
|
||||
|
||||
|
@ -196,16 +204,18 @@ public class WorkerStageKernel
|
|||
}
|
||||
}
|
||||
|
||||
public boolean addPostedResultsComplete(Pair<StageId, Integer> stageIdAndWorkerNumber)
|
||||
public boolean addPostedResultsComplete(StageId stageId, int workerNumber)
|
||||
{
|
||||
return postedResultsComplete.add(stageIdAndWorkerNumber);
|
||||
return postedResultsComplete.add(Pair.of(stageId, workerNumber));
|
||||
}
|
||||
|
||||
private void assertPreshuffleStatisticsNeeded()
|
||||
private void assertPreshuffleStatisticsNeeded(final boolean delivered)
|
||||
{
|
||||
if (!workOrder.getStageDefinition().mustGatherResultKeyStatistics()) {
|
||||
if (delivered != workOrder.getStageDefinition().mustGatherResultKeyStatistics()) {
|
||||
throw new ISE(
|
||||
"Result partitioning is not necessary for stage [%s]",
|
||||
"Result key statistics %s, but %s, for stage[%s]",
|
||||
delivered ? "delivered" : "not delivered",
|
||||
workOrder.getStageDefinition().mustGatherResultKeyStatistics() ? "expected" : "not expected",
|
||||
workOrder.getStageDefinition().getId()
|
||||
);
|
||||
}
|
||||
|
@ -222,7 +232,12 @@ public class WorkerStageKernel
|
|||
);
|
||||
phase = newPhase;
|
||||
} else {
|
||||
throw new IAE("Cannot transition from [%s] to [%s]", phase, newPhase);
|
||||
throw new IAE(
|
||||
"Cannot transition stage[%s] from[%s] to[%s]",
|
||||
workOrder.getStageDefinition().getId(),
|
||||
phase,
|
||||
newPhase
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,11 +54,12 @@ public enum WorkerStagePhase
|
|||
@Override
|
||||
public boolean canTransitionFrom(final WorkerStagePhase priorPhase)
|
||||
{
|
||||
return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES;
|
||||
return priorPhase == PRESHUFFLE_WAITING_FOR_RESULT_PARTITION_BOUNDARIES /* if globally sorting */
|
||||
|| priorPhase == READING_INPUT /* if locally sorting */;
|
||||
}
|
||||
},
|
||||
|
||||
RESULTS_READY {
|
||||
RESULTS_COMPLETE {
|
||||
@Override
|
||||
public boolean canTransitionFrom(final WorkerStagePhase priorPhase)
|
||||
{
|
||||
|
@ -70,7 +71,9 @@ public enum WorkerStagePhase
|
|||
@Override
|
||||
public boolean canTransitionFrom(final WorkerStagePhase priorPhase)
|
||||
{
|
||||
return priorPhase == RESULTS_READY;
|
||||
// Stages can transition to FINISHED even if they haven't generated all output yet. For example, this is
|
||||
// possible if the downstream stage is applying a limit.
|
||||
return priorPhase.compareTo(FINISHED) < 0;
|
||||
}
|
||||
},
|
||||
|
||||
|
@ -84,4 +87,24 @@ public enum WorkerStagePhase
|
|||
};
|
||||
|
||||
public abstract boolean canTransitionFrom(WorkerStagePhase priorPhase);
|
||||
|
||||
/**
|
||||
* Whether this phase indicates that the stage is no longer running.
|
||||
*/
|
||||
public boolean isTerminal()
|
||||
{
|
||||
return this == FINISHED || this == FAILED;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether this phase indicates a stage is running and consuming its full complement of resources.
|
||||
*
|
||||
* There are still some resources that can be consumed by stages that are not running. For example, in the
|
||||
* {@link #FINISHED} state, stages can still have data on disk that has not been cleaned-up yet, some pointers
|
||||
* to that data that still reside in memory, and some counters in memory available for collection by the controller.
|
||||
*/
|
||||
public boolean isRunning()
|
||||
{
|
||||
return this != NEW && this != RESULTS_COMPLETE && this != FINISHED && this != FAILED;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -97,7 +97,7 @@ public abstract class BaseWorkerClientImpl implements WorkerClient
|
|||
"/keyStatistics/%s/%d?sketchEncoding=%s",
|
||||
StringUtils.urlEncode(stageId.getQueryId()),
|
||||
stageId.getStageNumber(),
|
||||
WorkerChatHandler.SketchEncoding.OCTET_STREAM
|
||||
WorkerResource.SketchEncoding.OCTET_STREAM
|
||||
);
|
||||
|
||||
return getClient(workerId).asyncRequest(
|
||||
|
@ -118,7 +118,7 @@ public abstract class BaseWorkerClientImpl implements WorkerClient
|
|||
StringUtils.urlEncode(stageId.getQueryId()),
|
||||
stageId.getStageNumber(),
|
||||
timeChunk,
|
||||
WorkerChatHandler.SketchEncoding.OCTET_STREAM
|
||||
WorkerResource.SketchEncoding.OCTET_STREAM
|
||||
);
|
||||
|
||||
return getClient(workerId).asyncRequest(
|
||||
|
|
|
@ -82,6 +82,27 @@ public class ControllerResource
|
|||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by subtasks to inform the controller that they are done reading their input, in cases where they would
|
||||
* not be calling {@link #httpPostPartialKeyStatistics(Object, String, int, int, HttpServletRequest)}.
|
||||
*
|
||||
* See {@link ControllerClient#postDoneReadingInput(StageId, int)} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/doneReadingInput/{queryId}/{stageNumber}/{workerNumber}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
public Response httpPostDoneReadingInput(
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("workerNumber") final int workerNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req);
|
||||
controller.doneReadingInput(stageNumber, workerNumber);
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Used by subtasks to post system errors. Note that the errors are organized by taskId, not by query/stage/worker,
|
||||
* because system errors are associated with a task rather than a specific query/stage/worker execution context.
|
||||
|
@ -166,7 +187,7 @@ public class ControllerResource
|
|||
}
|
||||
|
||||
/**
|
||||
* See {@link ControllerClient#getTaskList()} for the client-side code that calls this API.
|
||||
* See {@link ControllerClient#getWorkerIds} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Path("/taskList")
|
||||
|
@ -174,7 +195,7 @@ public class ControllerResource
|
|||
public Response httpGetTaskList(@Context final HttpServletRequest req)
|
||||
{
|
||||
MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req);
|
||||
return Response.ok(new MSQTaskList(controller.getTaskIds())).build();
|
||||
return Response.ok(new MSQTaskList(controller.getWorkerIds())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -47,4 +47,20 @@ public class MSQResourceUtils
|
|||
throw new ForbiddenException(access.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public static void authorizeQueryRequest(
|
||||
final ResourcePermissionMapper permissionMapper,
|
||||
final AuthorizerMapper authorizerMapper,
|
||||
final HttpServletRequest request,
|
||||
final String queryId
|
||||
)
|
||||
{
|
||||
final List<ResourceAction> resourceActions = permissionMapper.getQueryPermissions(queryId);
|
||||
|
||||
Access access = AuthorizationUtils.authorizeAllResourceActions(request, resourceActions, authorizerMapper);
|
||||
|
||||
if (!access.isAllowed()) {
|
||||
throw new ForbiddenException(access.toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,11 +23,9 @@ import org.apache.druid.server.security.ResourceAction;
|
|||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Provides HTTP resources such as {@link ControllerResource} with information about which permissions are needed
|
||||
* for requests.
|
||||
*/
|
||||
public interface ResourcePermissionMapper
|
||||
{
|
||||
List<ResourceAction> getAdminPermissions();
|
||||
|
||||
List<ResourceAction> getQueryPermissions(String queryId);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,391 @@
|
|||
/*
|
||||
* 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.msq.rpc;
|
||||
|
||||
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.frame.file.FrameFileHttpResponseHandler;
|
||||
import org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.servlet.AsyncContext;
|
||||
import javax.servlet.AsyncEvent;
|
||||
import javax.servlet.AsyncListener;
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
public class WorkerResource
|
||||
{
|
||||
private static final Logger log = new Logger(WorkerResource.class);
|
||||
|
||||
/**
|
||||
* Callers must be able to store an entire chunk in memory. It can't be too large.
|
||||
*/
|
||||
private static final long CHANNEL_DATA_CHUNK_SIZE = 1_000_000;
|
||||
private static final long GET_CHANNEL_DATA_TIMEOUT = 30_000L;
|
||||
|
||||
protected final Worker worker;
|
||||
protected final ResourcePermissionMapper permissionMapper;
|
||||
protected final AuthorizerMapper authorizerMapper;
|
||||
|
||||
public WorkerResource(
|
||||
final Worker worker,
|
||||
final ResourcePermissionMapper permissionMapper,
|
||||
final AuthorizerMapper authorizerMapper
|
||||
)
|
||||
{
|
||||
this.worker = worker;
|
||||
this.permissionMapper = permissionMapper;
|
||||
this.authorizerMapper = authorizerMapper;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns up to {@link #CHANNEL_DATA_CHUNK_SIZE} bytes of stage output data.
|
||||
* <p>
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#fetchChannelData} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Path("/channels/{queryId}/{stageNumber}/{partitionNumber}")
|
||||
@Produces(MediaType.APPLICATION_OCTET_STREAM)
|
||||
public Response httpGetChannelData(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("partitionNumber") final int partitionNumber,
|
||||
@QueryParam("offset") final long offset,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
|
||||
final ListenableFuture<InputStream> dataFuture =
|
||||
worker.readStageOutput(new StageId(queryId, stageNumber), partitionNumber, offset);
|
||||
|
||||
final AsyncContext asyncContext = req.startAsync();
|
||||
asyncContext.setTimeout(GET_CHANNEL_DATA_TIMEOUT);
|
||||
asyncContext.addListener(
|
||||
new AsyncListener()
|
||||
{
|
||||
@Override
|
||||
public void onComplete(AsyncEvent event)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTimeout(AsyncEvent event)
|
||||
{
|
||||
HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse();
|
||||
response.setStatus(HttpServletResponse.SC_OK);
|
||||
event.getAsyncContext().complete();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onError(AsyncEvent event)
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onStartAsync(AsyncEvent event)
|
||||
{
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
// Save these items, since "req" becomes inaccessible in future exception handlers.
|
||||
final String remoteAddr = req.getRemoteAddr();
|
||||
final String requestURI = req.getRequestURI();
|
||||
|
||||
Futures.addCallback(
|
||||
dataFuture,
|
||||
new FutureCallback<InputStream>()
|
||||
{
|
||||
@Override
|
||||
public void onSuccess(final InputStream inputStream)
|
||||
{
|
||||
HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse();
|
||||
|
||||
try (final OutputStream outputStream = response.getOutputStream()) {
|
||||
if (inputStream == null) {
|
||||
response.setStatus(HttpServletResponse.SC_NOT_FOUND);
|
||||
} else {
|
||||
response.setStatus(HttpServletResponse.SC_OK);
|
||||
response.setContentType(MediaType.APPLICATION_OCTET_STREAM);
|
||||
|
||||
final byte[] readBuf = new byte[8192];
|
||||
final int firstRead = inputStream.read(readBuf);
|
||||
|
||||
if (firstRead == -1) {
|
||||
// Empty read means we're at the end of the channel.
|
||||
// Set the last fetch header so the client knows this.
|
||||
response.setHeader(
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_NAME,
|
||||
FrameFileHttpResponseHandler.HEADER_LAST_FETCH_VALUE
|
||||
);
|
||||
} else {
|
||||
long bytesReadTotal = 0;
|
||||
int bytesReadThisCall = firstRead;
|
||||
do {
|
||||
final int bytesToWrite =
|
||||
(int) Math.min(CHANNEL_DATA_CHUNK_SIZE - bytesReadTotal, bytesReadThisCall);
|
||||
outputStream.write(readBuf, 0, bytesToWrite);
|
||||
bytesReadTotal += bytesReadThisCall;
|
||||
} while (bytesReadTotal < CHANNEL_DATA_CHUNK_SIZE
|
||||
&& (bytesReadThisCall = inputStream.read(readBuf)) != -1);
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.noStackTrace().warn(e, "Could not respond to request from[%s] to[%s]", remoteAddr, requestURI);
|
||||
}
|
||||
finally {
|
||||
CloseableUtils.closeAndSuppressExceptions(inputStream, e -> log.warn("Failed to close output channel"));
|
||||
asyncContext.complete();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e)
|
||||
{
|
||||
if (!dataFuture.isCancelled()) {
|
||||
try {
|
||||
HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse();
|
||||
response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
|
||||
asyncContext.complete();
|
||||
}
|
||||
catch (Exception e2) {
|
||||
e.addSuppressed(e2);
|
||||
}
|
||||
|
||||
log.noStackTrace().warn(e, "Request failed from[%s] to[%s]", remoteAddr, requestURI);
|
||||
}
|
||||
}
|
||||
},
|
||||
Execs.directExecutor()
|
||||
);
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postWorkOrder} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
||||
@Path("/workOrder")
|
||||
public Response httpPostWorkOrder(final WorkOrder workOrder, @Context final HttpServletRequest req)
|
||||
{
|
||||
final String queryId = workOrder.getQueryDefinition().getQueryId();
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
worker.postWorkOrder(workOrder);
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postResultPartitionBoundaries} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
||||
@Path("/resultPartitionBoundaries/{queryId}/{stageNumber}")
|
||||
public Response httpPostResultPartitionBoundaries(
|
||||
final ClusterByPartitions stagePartitionBoundaries,
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
if (worker.postResultPartitionBoundaries(new StageId(queryId, stageNumber), stagePartitionBoundaries)) {
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
} else {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/keyStatistics/{queryId}/{stageNumber}")
|
||||
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM})
|
||||
public Response httpFetchKeyStatistics(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
ClusterByStatisticsSnapshot clusterByStatisticsSnapshot;
|
||||
StageId stageId = new StageId(queryId, stageNumber);
|
||||
try {
|
||||
clusterByStatisticsSnapshot = worker.fetchStatisticsSnapshot(stageId);
|
||||
if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_OCTET_STREAM)
|
||||
.entity(
|
||||
(StreamingOutput) output ->
|
||||
ClusterByStatisticsSnapshotSerde.serialize(output, clusterByStatisticsSnapshot)
|
||||
)
|
||||
.build();
|
||||
} else {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_JSON)
|
||||
.entity(clusterByStatisticsSnapshot)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
String errorMessage = StringUtils.format(
|
||||
"Invalid request for key statistics for query[%s] and stage[%d]",
|
||||
queryId,
|
||||
stageNumber
|
||||
);
|
||||
log.error(e, errorMessage);
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.<String, Object>of("error", errorMessage))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/keyStatisticsForTimeChunk/{queryId}/{stageNumber}/{timeChunk}")
|
||||
@Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_OCTET_STREAM})
|
||||
public Response httpFetchKeyStatisticsWithSnapshot(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@PathParam("timeChunk") final long timeChunk,
|
||||
@QueryParam("sketchEncoding") @Nullable final SketchEncoding sketchEncoding,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
ClusterByStatisticsSnapshot snapshotForTimeChunk;
|
||||
StageId stageId = new StageId(queryId, stageNumber);
|
||||
try {
|
||||
snapshotForTimeChunk = worker.fetchStatisticsSnapshotForTimeChunk(stageId, timeChunk);
|
||||
if (SketchEncoding.OCTET_STREAM.equals(sketchEncoding)) {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_OCTET_STREAM)
|
||||
.entity(
|
||||
(StreamingOutput) output ->
|
||||
ClusterByStatisticsSnapshotSerde.serialize(output, snapshotForTimeChunk)
|
||||
)
|
||||
.build();
|
||||
} else {
|
||||
return Response.status(Response.Status.ACCEPTED)
|
||||
.type(MediaType.APPLICATION_JSON)
|
||||
.entity(snapshotForTimeChunk)
|
||||
.build();
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
String errorMessage = StringUtils.format(
|
||||
"Invalid request for key statistics for query[%s], stage[%d] and timeChunk[%d]",
|
||||
queryId,
|
||||
stageNumber,
|
||||
timeChunk
|
||||
);
|
||||
log.error(e, errorMessage);
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(ImmutableMap.<String, Object>of("error", errorMessage))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postCleanupStage} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/cleanupStage/{queryId}/{stageNumber}")
|
||||
public Response httpPostCleanupStage(
|
||||
@PathParam("queryId") final String queryId,
|
||||
@PathParam("stageNumber") final int stageNumber,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
MSQResourceUtils.authorizeQueryRequest(permissionMapper, authorizerMapper, req, queryId);
|
||||
worker.postCleanupStage(new StageId(queryId, stageNumber));
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#postFinish} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/finish")
|
||||
public Response httpPostFinish(@Context final HttpServletRequest req)
|
||||
{
|
||||
MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req);
|
||||
worker.postFinish();
|
||||
return Response.status(Response.Status.ACCEPTED).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link org.apache.druid.msq.exec.WorkerClient#getCounters} for the client-side code that calls this API.
|
||||
*/
|
||||
@GET
|
||||
@Produces({MediaType.APPLICATION_JSON + "; qs=0.9", SmileMediaTypes.APPLICATION_JACKSON_SMILE + "; qs=0.1"})
|
||||
@Path("/counters")
|
||||
public Response httpGetCounters(@Context final HttpServletRequest req)
|
||||
{
|
||||
MSQResourceUtils.authorizeAdminRequest(permissionMapper, authorizerMapper, req);
|
||||
return Response.status(Response.Status.OK).entity(worker.getCounters()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and
|
||||
* {@link #httpFetchKeyStatisticsWithSnapshot}.
|
||||
*/
|
||||
public enum SketchEncoding
|
||||
{
|
||||
/**
|
||||
* The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}.
|
||||
*/
|
||||
OCTET_STREAM,
|
||||
/**
|
||||
* The key collector is encoded as json
|
||||
*/
|
||||
JSON
|
||||
}
|
||||
}
|
|
@ -0,0 +1,115 @@
|
|||
/*
|
||||
* 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.msq.shuffle.input;
|
||||
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.exec.OutputChannelMode;
|
||||
import org.apache.druid.msq.indexing.InputChannelFactory;
|
||||
import org.apache.druid.msq.input.stage.StageInputSlice;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Meta-factory that wraps {@link #inputChannelFactoryProvider}, and can create various other kinds of factories.
|
||||
*/
|
||||
public class MetaInputChannelFactory implements InputChannelFactory
|
||||
{
|
||||
private final Int2ObjectMap<OutputChannelMode> stageOutputModeMap;
|
||||
private final Function<OutputChannelMode, InputChannelFactory> inputChannelFactoryProvider;
|
||||
private final Map<OutputChannelMode, InputChannelFactory> inputChannelFactoryMap = new HashMap<>();
|
||||
|
||||
public MetaInputChannelFactory(
|
||||
final Int2ObjectMap<OutputChannelMode> stageOutputModeMap,
|
||||
final Function<OutputChannelMode, InputChannelFactory> inputChannelFactoryProvider
|
||||
)
|
||||
{
|
||||
this.stageOutputModeMap = stageOutputModeMap;
|
||||
this.inputChannelFactoryProvider = inputChannelFactoryProvider;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a meta-factory.
|
||||
*
|
||||
* @param slices stage slices from {@link WorkOrder#getInputs()}
|
||||
* @param defaultOutputChannelMode mode to use when {@link StageInputSlice#getOutputChannelMode()} is null; i.e.,
|
||||
* when running with an older controller
|
||||
* @param inputChannelFactoryProvider provider of {@link InputChannelFactory} for various {@link OutputChannelMode}
|
||||
*/
|
||||
public static MetaInputChannelFactory create(
|
||||
final List<StageInputSlice> slices,
|
||||
final OutputChannelMode defaultOutputChannelMode,
|
||||
final Function<OutputChannelMode, InputChannelFactory> inputChannelFactoryProvider
|
||||
)
|
||||
{
|
||||
final Int2ObjectMap<OutputChannelMode> stageOutputModeMap = new Int2ObjectOpenHashMap<>();
|
||||
|
||||
for (final StageInputSlice slice : slices) {
|
||||
final OutputChannelMode newMode;
|
||||
|
||||
if (slice.getOutputChannelMode() != null) {
|
||||
newMode = slice.getOutputChannelMode();
|
||||
} else {
|
||||
newMode = defaultOutputChannelMode;
|
||||
}
|
||||
|
||||
final OutputChannelMode prevMode = stageOutputModeMap.putIfAbsent(
|
||||
slice.getStageNumber(),
|
||||
newMode
|
||||
);
|
||||
|
||||
if (prevMode != null && prevMode != newMode) {
|
||||
throw new ISE(
|
||||
"Inconsistent output modes for stage[%s], got[%s] and[%s]",
|
||||
slice.getStageNumber(),
|
||||
prevMode,
|
||||
newMode
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
return new MetaInputChannelFactory(stageOutputModeMap, inputChannelFactoryProvider);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableFrameChannel openChannel(
|
||||
final StageId stageId,
|
||||
final int workerNumber,
|
||||
final int partitionNumber
|
||||
) throws IOException
|
||||
{
|
||||
final OutputChannelMode outputChannelMode = stageOutputModeMap.get(stageId.getStageNumber());
|
||||
|
||||
if (outputChannelMode == null) {
|
||||
throw new ISE("No output mode for stageNumber[%s]", stageId.getStageNumber());
|
||||
}
|
||||
|
||||
return inputChannelFactoryMap.computeIfAbsent(outputChannelMode, inputChannelFactoryProvider)
|
||||
.openChannel(stageId, workerNumber, partitionNumber);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* 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.msq.shuffle.input;
|
||||
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.msq.indexing.InputChannelFactory;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.shuffle.output.StageOutputHolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* An {@link InputChannelFactory} that loads data locally when possible, and otherwise connects directly to other
|
||||
* workers. Used when durable shuffle storage is off.
|
||||
*/
|
||||
public class WorkerOrLocalInputChannelFactory implements InputChannelFactory
|
||||
{
|
||||
private final String myId;
|
||||
private final Supplier<List<String>> workerIdsSupplier;
|
||||
private final InputChannelFactory workerInputChannelFactory;
|
||||
private final StageOutputHolderProvider stageOutputHolderProvider;
|
||||
|
||||
public WorkerOrLocalInputChannelFactory(
|
||||
final String myId,
|
||||
final Supplier<List<String>> workerIdsSupplier,
|
||||
final InputChannelFactory workerInputChannelFactory,
|
||||
final StageOutputHolderProvider stageOutputHolderProvider
|
||||
)
|
||||
{
|
||||
this.myId = myId;
|
||||
this.workerIdsSupplier = workerIdsSupplier;
|
||||
this.workerInputChannelFactory = workerInputChannelFactory;
|
||||
this.stageOutputHolderProvider = stageOutputHolderProvider;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException
|
||||
{
|
||||
final String taskId = workerIdsSupplier.get().get(workerNumber);
|
||||
if (taskId.equals(myId)) {
|
||||
return stageOutputHolderProvider.getHolder(stageId, partitionNumber).readLocally();
|
||||
} else {
|
||||
return workerInputChannelFactory.openChannel(stageId, workerNumber, partitionNumber);
|
||||
}
|
||||
}
|
||||
|
||||
public interface StageOutputHolderProvider
|
||||
{
|
||||
StageOutputHolder getHolder(StageId stageId, int partitionNumber);
|
||||
}
|
||||
}
|
|
@ -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.msq.shuffle.output;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Input stream based on a list of byte arrays.
|
||||
*/
|
||||
public class ByteChunksInputStream extends InputStream
|
||||
{
|
||||
private final List<byte[]> chunks;
|
||||
private int chunkNum;
|
||||
private int positionWithinChunk;
|
||||
|
||||
/**
|
||||
* Create a new stream wrapping a list of chunks.
|
||||
*
|
||||
* @param chunks byte arrays
|
||||
* @param positionWithinFirstChunk starting position within the first byte array
|
||||
*/
|
||||
public ByteChunksInputStream(final List<byte[]> chunks, final int positionWithinFirstChunk)
|
||||
{
|
||||
this.chunks = chunks;
|
||||
this.positionWithinChunk = positionWithinFirstChunk;
|
||||
this.chunkNum = -1;
|
||||
advanceChunk();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read()
|
||||
{
|
||||
if (chunkNum >= chunks.size()) {
|
||||
return -1;
|
||||
} else {
|
||||
final byte[] currentChunk = chunks.get(chunkNum);
|
||||
final byte b = currentChunk[positionWithinChunk++];
|
||||
|
||||
if (positionWithinChunk == currentChunk.length) {
|
||||
chunkNum++;
|
||||
positionWithinChunk = 0;
|
||||
}
|
||||
|
||||
return b & 0xFF;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b)
|
||||
{
|
||||
return read(b, 0, b.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len)
|
||||
{
|
||||
if (len == 0) {
|
||||
return 0;
|
||||
} else if (chunkNum >= chunks.size()) {
|
||||
return -1;
|
||||
} else {
|
||||
int r = 0;
|
||||
|
||||
while (r < len && chunkNum < chunks.size()) {
|
||||
final byte[] currentChunk = chunks.get(chunkNum);
|
||||
int toReadFromCurrentChunk = Math.min(len - r, currentChunk.length - positionWithinChunk);
|
||||
System.arraycopy(currentChunk, positionWithinChunk, b, off + r, toReadFromCurrentChunk);
|
||||
r += toReadFromCurrentChunk;
|
||||
positionWithinChunk += toReadFromCurrentChunk;
|
||||
if (positionWithinChunk == currentChunk.length) {
|
||||
chunkNum++;
|
||||
positionWithinChunk = 0;
|
||||
}
|
||||
}
|
||||
|
||||
return r;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
chunkNum = chunks.size();
|
||||
positionWithinChunk = 0;
|
||||
}
|
||||
|
||||
private void advanceChunk()
|
||||
{
|
||||
chunkNum++;
|
||||
|
||||
// Verify nonempty
|
||||
if (chunkNum < chunks.size() && chunks.get(chunkNum).length == 0) {
|
||||
throw DruidException.defensive("Empty chunk not allowed");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,274 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.errorprone.annotations.concurrent.GuardedBy;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.channel.ByteTracker;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFileWriter;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.exec.OutputChannelMode;
|
||||
import org.apache.druid.msq.kernel.controller.ControllerQueryKernelUtils;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Deque;
|
||||
|
||||
/**
|
||||
* Reader for the case where stage output is a generic {@link ReadableFrameChannel}.
|
||||
*
|
||||
* Because this reader returns an underlying channel directly, it must only be used when it is certain that
|
||||
* only a single consumer exists, i.e., when using output mode {@link OutputChannelMode#MEMORY}. See
|
||||
* {@link ControllerQueryKernelUtils#canUseMemoryOutput} for the code that ensures that there is only a single
|
||||
* consumer in the in-memory case.
|
||||
*/
|
||||
public class ChannelStageOutputReader implements StageOutputReader
|
||||
{
|
||||
enum State
|
||||
{
|
||||
INIT,
|
||||
LOCAL,
|
||||
REMOTE,
|
||||
CLOSED
|
||||
}
|
||||
|
||||
private final ReadableFrameChannel channel;
|
||||
private final FrameFileWriter writer;
|
||||
|
||||
/**
|
||||
* Pair of chunk size + chunk InputStream.
|
||||
*/
|
||||
private final Deque<byte[]> chunks = new ArrayDeque<>();
|
||||
|
||||
/**
|
||||
* State of this reader.
|
||||
*/
|
||||
@GuardedBy("this")
|
||||
private State state = State.INIT;
|
||||
|
||||
/**
|
||||
* Position of {@link #positionWithinFirstChunk} in the first chunk of {@link #chunks}, within the overall stream.
|
||||
*/
|
||||
@GuardedBy("this")
|
||||
private long cursor;
|
||||
|
||||
/**
|
||||
* Offset of the first chunk in {@link #chunks} which corresponds to {@link #cursor}.
|
||||
*/
|
||||
@GuardedBy("this")
|
||||
private int positionWithinFirstChunk;
|
||||
|
||||
/**
|
||||
* Whether {@link FrameFileWriter#close()} is called on {@link #writer}.
|
||||
*/
|
||||
@GuardedBy("this")
|
||||
private boolean didCloseWriter;
|
||||
|
||||
public ChannelStageOutputReader(final ReadableFrameChannel channel)
|
||||
{
|
||||
this.channel = channel;
|
||||
this.writer = FrameFileWriter.open(new ChunkAcceptor(), null, ByteTracker.unboundedTracker());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an input stream starting at the provided offset.
|
||||
*
|
||||
* The returned {@link InputStream} is non-blocking, and is slightly buffered (up to one frame). It does not
|
||||
* necessarily contain the complete remaining dataset; this means that multiple calls to this method are necessary
|
||||
* to fetch the complete dataset.
|
||||
*
|
||||
* The provided offset must be greater than, or equal to, the offset provided to the prior call.
|
||||
*
|
||||
* This class supports either remote or local reads, but not both. Calling both this method and {@link #readLocally()}
|
||||
* on the same instance of this class is an error.
|
||||
*
|
||||
* @param offset offset into the stage output stream
|
||||
*/
|
||||
@Override
|
||||
public synchronized ListenableFuture<InputStream> readRemotelyFrom(final long offset)
|
||||
{
|
||||
if (state == State.INIT) {
|
||||
state = State.REMOTE;
|
||||
} else if (state == State.LOCAL) {
|
||||
throw new ISE("Cannot read both remotely and locally");
|
||||
} else if (state == State.CLOSED) {
|
||||
throw new ISE("Closed");
|
||||
}
|
||||
|
||||
if (offset < cursor) {
|
||||
return Futures.immediateFailedFuture(
|
||||
new ISE("Offset[%,d] no longer available, current cursor is[%,d]", offset, cursor));
|
||||
}
|
||||
|
||||
while (chunks.isEmpty() || offset > cursor) {
|
||||
// Fetch additional chunks if needed.
|
||||
if (chunks.isEmpty()) {
|
||||
if (didCloseWriter) {
|
||||
if (offset == cursor) {
|
||||
return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY));
|
||||
} else {
|
||||
throw DruidException.defensive(
|
||||
"Channel finished but cursor[%,d] does not match requested offset[%,d]",
|
||||
cursor,
|
||||
offset
|
||||
);
|
||||
}
|
||||
} else if (channel.isFinished()) {
|
||||
try {
|
||||
writer.close();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
didCloseWriter = true;
|
||||
continue;
|
||||
} else if (channel.canRead()) {
|
||||
try {
|
||||
writer.writeFrame(channel.read(), FrameFileWriter.NO_PARTITION);
|
||||
}
|
||||
catch (Exception e) {
|
||||
try {
|
||||
writer.abort();
|
||||
}
|
||||
catch (IOException e2) {
|
||||
e.addSuppressed(e2);
|
||||
}
|
||||
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
return FutureUtils.transformAsync(channel.readabilityFuture(), ignored -> readRemotelyFrom(offset));
|
||||
}
|
||||
}
|
||||
|
||||
// Advance cursor to the provided offset, or the end of the current chunk, whichever is earlier.
|
||||
final byte[] chunk = chunks.peek();
|
||||
final long amountToAdvance = Math.min(offset - cursor, chunk.length - positionWithinFirstChunk);
|
||||
cursor += amountToAdvance;
|
||||
positionWithinFirstChunk += Ints.checkedCast(amountToAdvance);
|
||||
|
||||
// Remove first chunk if it is no longer needed. (i.e., if the cursor is at the end of it.)
|
||||
if (positionWithinFirstChunk == chunk.length) {
|
||||
chunks.poll();
|
||||
positionWithinFirstChunk = 0;
|
||||
}
|
||||
}
|
||||
|
||||
if (chunks.isEmpty() || offset != cursor) {
|
||||
throw DruidException.defensive(
|
||||
"Expected cursor[%,d] to be caught up to offset[%,d] by this point, and to have nonzero chunks",
|
||||
cursor,
|
||||
offset
|
||||
);
|
||||
}
|
||||
|
||||
return Futures.immediateFuture(new ByteChunksInputStream(ImmutableList.copyOf(chunks), positionWithinFirstChunk));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link ReadableFrameChannel} that backs this reader.
|
||||
*
|
||||
* Callers are responsible for closing the returned channel. Once this method is called, the caller becomes the
|
||||
* owner of the channel, and this class's {@link #close()} method will no longer close the channel.
|
||||
*
|
||||
* Only a single reader is supported. Once this method is called, it cannot be called again.
|
||||
*
|
||||
* This class supports either remote or local reads, but not both. Calling both this method and
|
||||
* {@link #readRemotelyFrom(long)} on the same instance of this class is an error.
|
||||
*/
|
||||
@Override
|
||||
public synchronized ReadableFrameChannel readLocally()
|
||||
{
|
||||
if (state == State.INIT) {
|
||||
state = State.LOCAL;
|
||||
return channel;
|
||||
} else if (state == State.REMOTE) {
|
||||
throw new ISE("Cannot read both remotely and locally");
|
||||
} else if (state == State.LOCAL) {
|
||||
throw new ISE("Cannot read channel multiple times");
|
||||
} else {
|
||||
assert state == State.CLOSED;
|
||||
throw new ISE("Closed");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the {@link ReadableFrameChannel} backing this reader, unless {@link #readLocally()} has been called.
|
||||
* In that case, the caller of {@link #readLocally()} is responsible for closing the channel.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void close()
|
||||
{
|
||||
// Call channel.close() unless readLocally() has been called. In that case, we expect the caller to close it.
|
||||
if (state != State.LOCAL) {
|
||||
state = State.CLOSED;
|
||||
channel.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Input stream that can have bytes appended to it, and that can have bytes acknowledged.
|
||||
*/
|
||||
private class ChunkAcceptor implements WritableByteChannel
|
||||
{
|
||||
private boolean open = true;
|
||||
|
||||
@Override
|
||||
public int write(final ByteBuffer src) throws IOException
|
||||
{
|
||||
if (!open) {
|
||||
throw new IOException("Closed");
|
||||
}
|
||||
|
||||
final int len = src.remaining();
|
||||
if (len > 0) {
|
||||
final byte[] bytes = new byte[len];
|
||||
src.get(bytes);
|
||||
chunks.add(bytes);
|
||||
}
|
||||
|
||||
return len;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isOpen()
|
||||
{
|
||||
return open;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
open = false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,99 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.druid.frame.channel.ReadableFileFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFile;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.channels.Channels;
|
||||
|
||||
/**
|
||||
* Reader for the case where stage output is stored in a {@link FrameFile} on disk.
|
||||
*/
|
||||
public class FileStageOutputReader implements StageOutputReader
|
||||
{
|
||||
private final FrameFile frameFile;
|
||||
|
||||
public FileStageOutputReader(FrameFile frameFile)
|
||||
{
|
||||
this.frameFile = frameFile;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an input stream starting at the provided offset. The file is opened and seeked in-line with this method
|
||||
* call, so the returned future is always immediately resolved. Callers are responsible for closing the returned
|
||||
* input stream.
|
||||
*
|
||||
* This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when
|
||||
* broadcasting the output of a stage.
|
||||
*
|
||||
* @param offset offset into the stage output file
|
||||
*/
|
||||
@Override
|
||||
public ListenableFuture<InputStream> readRemotelyFrom(long offset)
|
||||
{
|
||||
try {
|
||||
final RandomAccessFile randomAccessFile = new RandomAccessFile(frameFile.file(), "r");
|
||||
|
||||
if (offset >= randomAccessFile.length()) {
|
||||
randomAccessFile.close();
|
||||
return Futures.immediateFuture(new ByteArrayInputStream(ByteArrays.EMPTY_ARRAY));
|
||||
} else {
|
||||
randomAccessFile.seek(offset);
|
||||
return Futures.immediateFuture(Channels.newInputStream(randomAccessFile.getChannel()));
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
return Futures.immediateFailedFuture(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a channel pointing to a fresh {@link FrameFile#newReference()} of the underlying frame file. Callers are
|
||||
* responsible for closing the returned channel.
|
||||
*
|
||||
* This class supports remote and local reads from the same {@link FrameFile}, which, for example, is useful when
|
||||
* broadcasting the output of a stage.
|
||||
*/
|
||||
@Override
|
||||
public ReadableFrameChannel readLocally()
|
||||
{
|
||||
return new ReadableFileFrameChannel(frameFile.newReference());
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes the initial reference to the underlying {@link FrameFile}. Does not close additional references created by
|
||||
* calls to {@link #readLocally()}; those references are closed when the channel(s) returned by {@link #readLocally()}
|
||||
* are closed.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
frameFile.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,124 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* Channel that wraps a {@link ListenableFuture} of a {@link ReadableFrameChannel}, but acts like a regular (non-future)
|
||||
* {@link ReadableFrameChannel}.
|
||||
*/
|
||||
public class FutureReadableFrameChannel implements ReadableFrameChannel
|
||||
{
|
||||
private static final Logger log = new Logger(FutureReadableFrameChannel.class);
|
||||
|
||||
private final ListenableFuture<ReadableFrameChannel> channelFuture;
|
||||
private ReadableFrameChannel channel;
|
||||
|
||||
public FutureReadableFrameChannel(final ListenableFuture<ReadableFrameChannel> channelFuture)
|
||||
{
|
||||
this.channelFuture = channelFuture;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinished()
|
||||
{
|
||||
if (populateChannel()) {
|
||||
return channel.isFinished();
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canRead()
|
||||
{
|
||||
if (populateChannel()) {
|
||||
return channel.canRead();
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Frame read()
|
||||
{
|
||||
if (populateChannel()) {
|
||||
return channel.read();
|
||||
} else {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<?> readabilityFuture()
|
||||
{
|
||||
if (populateChannel()) {
|
||||
return channel.readabilityFuture();
|
||||
} else {
|
||||
return FutureUtils.transformAsync(channelFuture, ignored -> readabilityFuture());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
if (populateChannel()) {
|
||||
channel.close();
|
||||
} else {
|
||||
channelFuture.cancel(true);
|
||||
|
||||
// In case of a race where channelFuture resolved between populateChannel() and here, the cancel call above would
|
||||
// have no effect. Guard against this case by checking if the channelFuture has resolved, and if so, close the
|
||||
// channel here.
|
||||
try {
|
||||
final ReadableFrameChannel theChannel = FutureUtils.getUncheckedImmediately(channelFuture);
|
||||
|
||||
try {
|
||||
theChannel.close();
|
||||
}
|
||||
catch (Throwable t) {
|
||||
log.noStackTrace().warn(t, "Failed to close channel");
|
||||
}
|
||||
}
|
||||
catch (Throwable ignored) {
|
||||
// Suppress.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean populateChannel()
|
||||
{
|
||||
if (channel != null) {
|
||||
return true;
|
||||
} else if (channelFuture.isDone()) {
|
||||
channel = FutureUtils.getUncheckedImmediately(channelFuture);
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.frame.channel.ByteTracker;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableNilFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFileWriter;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.channels.Channels;
|
||||
|
||||
/**
|
||||
* Reader for the case where stage output is known to be empty.
|
||||
*/
|
||||
public class NilStageOutputReader implements StageOutputReader
|
||||
{
|
||||
public static final NilStageOutputReader INSTANCE = new NilStageOutputReader();
|
||||
|
||||
private static final byte[] EMPTY_FRAME_FILE;
|
||||
|
||||
static {
|
||||
try {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close();
|
||||
EMPTY_FRAME_FILE = baos.toByteArray();
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<InputStream> readRemotelyFrom(final long offset)
|
||||
{
|
||||
final ByteArrayInputStream in = new ByteArrayInputStream(EMPTY_FRAME_FILE);
|
||||
|
||||
//noinspection ResultOfMethodCallIgnored: OK to ignore since "skip" always works for ByteArrayInputStream.
|
||||
in.skip(offset);
|
||||
|
||||
return Futures.immediateFuture(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableFrameChannel readLocally()
|
||||
{
|
||||
return ReadableNilFrameChannel.INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
// Nothing to do.
|
||||
}
|
||||
}
|
|
@ -0,0 +1,141 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.SettableFuture;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.frame.channel.ReadableFileFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableNilFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFile;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.rpc.WorkerResource;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import java.io.Closeable;
|
||||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* Container for a {@link StageOutputReader}, which is used to read the output of a stage.
|
||||
*/
|
||||
public class StageOutputHolder implements Closeable
|
||||
{
|
||||
private final SettableFuture<ReadableFrameChannel> channelFuture;
|
||||
private final ListenableFuture<StageOutputReader> readerFuture;
|
||||
|
||||
public StageOutputHolder()
|
||||
{
|
||||
this.channelFuture = SettableFuture.create();
|
||||
this.readerFuture = FutureUtils.transform(channelFuture, StageOutputHolder::createReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method for remote reads.
|
||||
*
|
||||
* Provides the implementation for {@link Worker#readStageOutput(StageId, int, long)}, which is in turn used by
|
||||
* {@link WorkerResource#httpGetChannelData(String, int, int, long, HttpServletRequest)}.
|
||||
*
|
||||
* @see StageOutputReader#readRemotelyFrom(long) for details on behavior
|
||||
*/
|
||||
public ListenableFuture<InputStream> readRemotelyFrom(final long offset)
|
||||
{
|
||||
return FutureUtils.transformAsync(readerFuture, reader -> reader.readRemotelyFrom(offset));
|
||||
}
|
||||
|
||||
/**
|
||||
* Method for local reads.
|
||||
*
|
||||
* Used instead of {@link #readRemotelyFrom(long)} when a worker is reading a channel from itself, to avoid needless
|
||||
* HTTP calls to itself.
|
||||
*
|
||||
* @see StageOutputReader#readLocally() for details on behavior
|
||||
*/
|
||||
public ReadableFrameChannel readLocally()
|
||||
{
|
||||
return new FutureReadableFrameChannel(FutureUtils.transform(readerFuture, StageOutputReader::readLocally));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the channel that backs {@link #readLocally()} and {@link #readRemotelyFrom(long)}.
|
||||
*/
|
||||
public void setChannel(final ReadableFrameChannel channel)
|
||||
{
|
||||
if (!channelFuture.set(channel)) {
|
||||
if (FutureUtils.getUncheckedImmediately(channelFuture) == null) {
|
||||
throw new ISE("Closed");
|
||||
} else {
|
||||
throw new ISE("Channel already set");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
channelFuture.set(null);
|
||||
|
||||
final StageOutputReader reader;
|
||||
|
||||
try {
|
||||
reader = FutureUtils.getUnchecked(readerFuture, true);
|
||||
}
|
||||
catch (Throwable e) {
|
||||
// Error creating the reader, nothing to close. Suppress.
|
||||
return;
|
||||
}
|
||||
|
||||
if (reader != null) {
|
||||
CloseableUtils.closeAndWrapExceptions(reader);
|
||||
}
|
||||
}
|
||||
|
||||
private static StageOutputReader createReader(final ReadableFrameChannel channel)
|
||||
{
|
||||
if (channel == null) {
|
||||
// Happens if close() was called before the channel resolved.
|
||||
throw new ISE("Closed");
|
||||
}
|
||||
|
||||
if (channel instanceof ReadableNilFrameChannel) {
|
||||
return NilStageOutputReader.INSTANCE;
|
||||
}
|
||||
|
||||
if (channel instanceof ReadableFileFrameChannel) {
|
||||
// Optimized implementation when reading an entire file.
|
||||
final ReadableFileFrameChannel fileChannel = (ReadableFileFrameChannel) channel;
|
||||
|
||||
if (fileChannel.isEntireFile()) {
|
||||
final FrameFile frameFile = fileChannel.newFrameFileReference();
|
||||
|
||||
// Close original channel, so we don't leak a frame file reference.
|
||||
channel.close();
|
||||
|
||||
return new FileStageOutputReader(frameFile);
|
||||
}
|
||||
}
|
||||
|
||||
// Generic implementation for any other type of channel.
|
||||
return new ChannelStageOutputReader(channel);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,77 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFile;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.InputStream;
|
||||
|
||||
/**
|
||||
* Interface for reading output channels for a particular stage. Each instance of this interface represents a
|
||||
* stream from a single {@link org.apache.druid.msq.kernel.StagePartition} in {@link FrameFile} format.
|
||||
*
|
||||
* @see FileStageOutputReader implementation backed by {@link FrameFile}
|
||||
* @see ChannelStageOutputReader implementation backed by {@link ReadableFrameChannel}
|
||||
* @see NilStageOutputReader implementation for an empty channel
|
||||
*/
|
||||
public interface StageOutputReader extends Closeable
|
||||
{
|
||||
/**
|
||||
* Method for remote reads.
|
||||
*
|
||||
* This method ultimately backs {@link Worker#readStageOutput(StageId, int, long)}. Refer to that method's
|
||||
* documentation for details about behavior of the returned future.
|
||||
*
|
||||
* Callers are responsible for closing the returned {@link InputStream}. This input stream may encapsulate
|
||||
* resources that are not closed by this class's {@link #close()} method.
|
||||
*
|
||||
* It is implementation-dependent whether calls to this method must have monotonically increasing offsets.
|
||||
* In particular, {@link ChannelStageOutputReader} requires monotonically increasing offsets, but
|
||||
* {@link FileStageOutputReader} and {@link NilStageOutputReader} do not.
|
||||
*
|
||||
* @param offset offset into the stage output file
|
||||
*
|
||||
* @see StageOutputHolder#readRemotelyFrom(long) which uses this method
|
||||
* @see Worker#readStageOutput(StageId, int, long) for documentation on behavior of the returned future
|
||||
*/
|
||||
ListenableFuture<InputStream> readRemotelyFrom(long offset);
|
||||
|
||||
/**
|
||||
* Method for local reads.
|
||||
*
|
||||
* Depending on implementation, this method may or may not be able to be called multiple times, and may or may not
|
||||
* be able to be mixed with {@link #readRemotelyFrom(long)}. Refer to the specific implementation for more details.
|
||||
*
|
||||
* Callers are responsible for closing the returned channel. The returned channel may encapsulate resources that
|
||||
* are not closed by this class's {@link #close()} method.
|
||||
*
|
||||
* It is implementation-dependent whether this method can be called multiple times. In particular,
|
||||
* {@link ChannelStageOutputReader#readLocally()} can only be called one time, but the implementations in
|
||||
* {@link FileStageOutputReader} and {@link NilStageOutputReader} can be called multiple times.
|
||||
*
|
||||
* @see StageOutputHolder#readLocally() which uses this method
|
||||
*/
|
||||
ReadableFrameChannel readLocally();
|
||||
}
|
|
@ -23,12 +23,14 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.error.ErrorResponse;
|
||||
import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
||||
import org.apache.druid.msq.indexing.error.MSQErrorReport;
|
||||
import org.apache.druid.msq.indexing.report.MSQStagesReport;
|
||||
import org.apache.druid.msq.sql.SqlStatementState;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class SqlStatementResult
|
||||
{
|
||||
|
@ -51,6 +53,27 @@ public class SqlStatementResult
|
|||
@Nullable
|
||||
private final ErrorResponse errorResponse;
|
||||
|
||||
@Nullable
|
||||
private final MSQStagesReport stages;
|
||||
|
||||
@Nullable
|
||||
private final CounterSnapshotsTree counters;
|
||||
|
||||
@Nullable
|
||||
private final List<MSQErrorReport> warnings;
|
||||
|
||||
public SqlStatementResult(
|
||||
String queryId,
|
||||
SqlStatementState state,
|
||||
DateTime createdAt,
|
||||
List<ColumnNameAndTypes> sqlRowSignature,
|
||||
Long durationMs,
|
||||
ResultSetInformation resultSetInformation,
|
||||
ErrorResponse errorResponse
|
||||
)
|
||||
{
|
||||
this(queryId, state, createdAt, sqlRowSignature, durationMs, resultSetInformation, errorResponse, null, null, null);
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public SqlStatementResult(
|
||||
|
@ -67,8 +90,13 @@ public class SqlStatementResult
|
|||
@Nullable @JsonProperty("result")
|
||||
ResultSetInformation resultSetInformation,
|
||||
@Nullable @JsonProperty("errorDetails")
|
||||
ErrorResponse errorResponse
|
||||
|
||||
ErrorResponse errorResponse,
|
||||
@Nullable @JsonProperty("stages")
|
||||
MSQStagesReport stages,
|
||||
@Nullable @JsonProperty("counters")
|
||||
CounterSnapshotsTree counters,
|
||||
@Nullable @JsonProperty("warnings")
|
||||
List<MSQErrorReport> warnings
|
||||
)
|
||||
{
|
||||
this.queryId = queryId;
|
||||
|
@ -78,6 +106,9 @@ public class SqlStatementResult
|
|||
this.durationMs = durationMs;
|
||||
this.resultSetInformation = resultSetInformation;
|
||||
this.errorResponse = errorResponse;
|
||||
this.stages = stages;
|
||||
this.counters = counters;
|
||||
this.warnings = warnings;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -130,41 +161,28 @@ public class SqlStatementResult
|
|||
return errorResponse;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
@JsonProperty("stages")
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public MSQStagesReport getStages()
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
SqlStatementResult that = (SqlStatementResult) o;
|
||||
return Objects.equals(queryId, that.queryId) && state == that.state && Objects.equals(
|
||||
createdAt,
|
||||
that.createdAt
|
||||
) && Objects.equals(sqlRowSignature, that.sqlRowSignature) && Objects.equals(
|
||||
durationMs,
|
||||
that.durationMs
|
||||
) && Objects.equals(resultSetInformation, that.resultSetInformation) && Objects.equals(
|
||||
errorResponse == null ? null : errorResponse.getAsMap(),
|
||||
that.errorResponse == null ? null : that.errorResponse.getAsMap()
|
||||
);
|
||||
return stages;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
@JsonProperty("counters")
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public CounterSnapshotsTree getCounters()
|
||||
{
|
||||
return Objects.hash(
|
||||
queryId,
|
||||
state,
|
||||
createdAt,
|
||||
sqlRowSignature,
|
||||
durationMs,
|
||||
resultSetInformation,
|
||||
errorResponse == null ? null : errorResponse.getAsMap()
|
||||
);
|
||||
return counters;
|
||||
}
|
||||
|
||||
@JsonProperty("warnings")
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public List<MSQErrorReport> getWarnings()
|
||||
{
|
||||
return warnings;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -180,6 +198,9 @@ public class SqlStatementResult
|
|||
", errorResponse=" + (errorResponse == null
|
||||
? "{}"
|
||||
: errorResponse.getAsMap().toString()) +
|
||||
", stages=" + stages +
|
||||
", counters=" + counters +
|
||||
", warnings=" + warnings +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -113,6 +113,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
|
@ -231,7 +232,9 @@ public class SqlStatementResource
|
|||
@Path("/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response doGetStatus(
|
||||
@PathParam("id") final String queryId, @Context final HttpServletRequest req
|
||||
@PathParam("id") final String queryId,
|
||||
@QueryParam("detail") boolean detail,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
try {
|
||||
|
@ -242,7 +245,8 @@ public class SqlStatementResource
|
|||
queryId,
|
||||
authenticationResult,
|
||||
true,
|
||||
Action.READ
|
||||
Action.READ,
|
||||
detail
|
||||
);
|
||||
|
||||
if (sqlStatementResult.isPresent()) {
|
||||
|
@ -369,7 +373,8 @@ public class SqlStatementResource
|
|||
queryId,
|
||||
authenticationResult,
|
||||
false,
|
||||
Action.WRITE
|
||||
Action.WRITE,
|
||||
false
|
||||
);
|
||||
if (sqlStatementResult.isPresent()) {
|
||||
switch (sqlStatementResult.get().getState()) {
|
||||
|
@ -479,7 +484,7 @@ public class SqlStatementResource
|
|||
}
|
||||
String taskId = String.valueOf(firstRow[0]);
|
||||
|
||||
Optional<SqlStatementResult> statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ);
|
||||
Optional<SqlStatementResult> statementResult = getStatementStatus(taskId, authenticationResult, true, Action.READ, false);
|
||||
|
||||
if (statementResult.isPresent()) {
|
||||
return Response.status(Response.Status.OK).entity(statementResult.get()).build();
|
||||
|
@ -565,7 +570,8 @@ public class SqlStatementResource
|
|||
String queryId,
|
||||
AuthenticationResult authenticationResult,
|
||||
boolean withResults,
|
||||
Action forAction
|
||||
Action forAction,
|
||||
boolean detail
|
||||
) throws DruidException
|
||||
{
|
||||
TaskStatusResponse taskResponse = contactOverlord(overlordClient.taskStatus(queryId), queryId);
|
||||
|
@ -582,14 +588,29 @@ public class SqlStatementResource
|
|||
MSQControllerTask msqControllerTask = getMSQControllerTaskAndCheckPermission(queryId, authenticationResult, forAction);
|
||||
SqlStatementState sqlStatementState = SqlStatementResourceHelper.getSqlStatementState(statusPlus);
|
||||
|
||||
Supplier<Optional<MSQTaskReportPayload>> msqTaskReportPayloadSupplier = () -> {
|
||||
try {
|
||||
return Optional.ofNullable(SqlStatementResourceHelper.getPayload(
|
||||
contactOverlord(overlordClient.taskReportAsMap(queryId), queryId)
|
||||
));
|
||||
}
|
||||
catch (DruidException e) {
|
||||
if (e.getErrorCode().equals("notFound") || e.getMessage().contains("Unable to contact overlord")) {
|
||||
return Optional.empty();
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
};
|
||||
|
||||
if (SqlStatementState.FAILED == sqlStatementState) {
|
||||
return SqlStatementResourceHelper.getExceptionPayload(
|
||||
queryId,
|
||||
taskResponse,
|
||||
statusPlus,
|
||||
sqlStatementState,
|
||||
contactOverlord(overlordClient.taskReportAsMap(queryId), queryId),
|
||||
jsonMapper
|
||||
msqTaskReportPayloadSupplier.get().orElse(null),
|
||||
jsonMapper,
|
||||
detail
|
||||
);
|
||||
} else {
|
||||
Optional<List<ColumnNameAndTypes>> signature = SqlStatementResourceHelper.getSignature(msqControllerTask);
|
||||
|
@ -605,7 +626,10 @@ public class SqlStatementResource
|
|||
sqlStatementState,
|
||||
msqControllerTask.getQuerySpec().getDestination()
|
||||
).orElse(null) : null,
|
||||
null
|
||||
null,
|
||||
detail ? SqlStatementResourceHelper.getQueryStagesReport(msqTaskReportPayloadSupplier.get().orElse(null)) : null,
|
||||
detail ? SqlStatementResourceHelper.getQueryCounters(msqTaskReportPayloadSupplier.get().orElse(null)) : null,
|
||||
detail ? SqlStatementResourceHelper.getQueryWarningDetails(msqTaskReportPayloadSupplier.get().orElse(null)) : null
|
||||
));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -250,11 +250,12 @@ public class SqlStatementResourceHelper
|
|||
TaskStatusResponse taskResponse,
|
||||
TaskStatusPlus statusPlus,
|
||||
SqlStatementState sqlStatementState,
|
||||
TaskReport.ReportMap msqPayload,
|
||||
ObjectMapper jsonMapper
|
||||
MSQTaskReportPayload msqTaskReportPayload,
|
||||
ObjectMapper jsonMapper,
|
||||
boolean detail
|
||||
)
|
||||
{
|
||||
final MSQErrorReport exceptionDetails = getQueryExceptionDetails(getPayload(msqPayload));
|
||||
final MSQErrorReport exceptionDetails = getQueryExceptionDetails(msqTaskReportPayload);
|
||||
final MSQFault fault = exceptionDetails == null ? null : exceptionDetails.getFault();
|
||||
if (exceptionDetails == null || fault == null) {
|
||||
return Optional.of(new SqlStatementResult(
|
||||
|
@ -267,7 +268,10 @@ public class SqlStatementResourceHelper
|
|||
DruidException.forPersona(DruidException.Persona.DEVELOPER)
|
||||
.ofCategory(DruidException.Category.UNCATEGORIZED)
|
||||
.build("%s", taskResponse.getStatus().getErrorMsg())
|
||||
.toErrorResponse()
|
||||
.toErrorResponse(),
|
||||
detail ? getQueryStagesReport(msqTaskReportPayload) : null,
|
||||
detail ? getQueryCounters(msqTaskReportPayload) : null,
|
||||
detail ? getQueryWarningDetails(msqTaskReportPayload) : null
|
||||
));
|
||||
}
|
||||
|
||||
|
@ -293,7 +297,10 @@ public class SqlStatementResourceHelper
|
|||
ex.withContext(exceptionContext);
|
||||
return ex;
|
||||
}
|
||||
}).toErrorResponse()
|
||||
}).toErrorResponse(),
|
||||
detail ? getQueryStagesReport(msqTaskReportPayload) : null,
|
||||
detail ? getQueryCounters(msqTaskReportPayload) : null,
|
||||
detail ? getQueryWarningDetails(msqTaskReportPayload) : null
|
||||
));
|
||||
}
|
||||
|
||||
|
@ -353,7 +360,7 @@ public class SqlStatementResourceHelper
|
|||
}
|
||||
|
||||
@Nullable
|
||||
public static MSQStagesReport.Stage getFinalStage(MSQTaskReportPayload msqTaskReportPayload)
|
||||
public static MSQStagesReport.Stage getFinalStage(@Nullable MSQTaskReportPayload msqTaskReportPayload)
|
||||
{
|
||||
if (msqTaskReportPayload == null || msqTaskReportPayload.getStages().getStages() == null) {
|
||||
return null;
|
||||
|
@ -369,11 +376,29 @@ public class SqlStatementResourceHelper
|
|||
}
|
||||
|
||||
@Nullable
|
||||
private static MSQErrorReport getQueryExceptionDetails(MSQTaskReportPayload payload)
|
||||
private static MSQErrorReport getQueryExceptionDetails(@Nullable MSQTaskReportPayload payload)
|
||||
{
|
||||
return payload == null ? null : payload.getStatus().getErrorReport();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static List<MSQErrorReport> getQueryWarningDetails(@Nullable MSQTaskReportPayload payload)
|
||||
{
|
||||
return payload == null ? null : new ArrayList<>(payload.getStatus().getWarningReports());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static MSQStagesReport getQueryStagesReport(@Nullable MSQTaskReportPayload payload)
|
||||
{
|
||||
return payload == null ? null : payload.getStages();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static CounterSnapshotsTree getQueryCounters(@Nullable MSQTaskReportPayload payload)
|
||||
{
|
||||
return payload == null ? null : payload.getCounters();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static MSQTaskReportPayload getPayload(TaskReport.ReportMap reportMap)
|
||||
{
|
||||
|
|
|
@ -230,7 +230,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
.setQueryContext(adjustedContext)
|
||||
.setExpectedDataSource("foo")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
NullHandling.sqlCompatible()
|
||||
? ImmutableList.of(
|
||||
|
@ -278,7 +278,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
.setQueryContext(adjustedContext)
|
||||
.setExpectedDataSource("foo")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, null},
|
||||
|
@ -325,7 +325,7 @@ public class MSQArraysTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.verifyResults();
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableSet;
|
|||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||
import org.apache.druid.data.input.impl.systemfield.SystemFields;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.msq.indexing.MSQSpec;
|
||||
import org.apache.druid.msq.indexing.MSQTuningConfig;
|
||||
|
@ -64,7 +64,7 @@ import java.util.Map;
|
|||
public class MSQComplexGroupByTest extends MSQTestBase
|
||||
{
|
||||
static {
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private String dataFileNameJsonString;
|
||||
|
@ -109,7 +109,7 @@ public class MSQComplexGroupByTest extends MSQTestBase
|
|||
dataFileSignature
|
||||
);
|
||||
|
||||
objectMapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList());
|
||||
}
|
||||
|
||||
@MethodSource("data")
|
||||
|
@ -129,7 +129,7 @@ public class MSQComplexGroupByTest extends MSQTestBase
|
|||
+ " GROUP BY 1\n"
|
||||
+ " PARTITIONED BY ALL")
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
|
@ -257,7 +257,7 @@ public class MSQComplexGroupByTest extends MSQTestBase
|
|||
+ " GROUP BY 1\n"
|
||||
+ " PARTITIONED BY ALL")
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.msq.exec.MSQDrillWindowQueryTest.DrillWindowQueryMSQComp
|
|||
import org.apache.druid.msq.sql.MSQTaskSqlEngine;
|
||||
import org.apache.druid.msq.test.CalciteMSQTestsHelper;
|
||||
import org.apache.druid.msq.test.ExtractResultsFactory;
|
||||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
import org.apache.druid.msq.test.MSQTestOverlordServiceClient;
|
||||
import org.apache.druid.msq.test.VerifyMSQSupportedNativeQueriesPredicate;
|
||||
import org.apache.druid.query.groupby.TestGroupByBuffers;
|
||||
|
|
|
@ -57,7 +57,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -88,7 +88,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -127,7 +127,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(queryContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -168,7 +168,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -219,7 +219,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -258,7 +258,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(DEFAULT_MSQ_CONTEXT)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -335,7 +335,7 @@ public class MSQExportTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(queryContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of())
|
||||
.setExpectedSegments(ImmutableSet.of())
|
||||
.setExpectedResultRows(ImmutableList.of())
|
||||
.verifyResults();
|
||||
|
||||
|
|
|
@ -113,7 +113,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(context)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedMSQSegmentReport(
|
||||
new MSQSegmentReport(
|
||||
|
@ -310,7 +310,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27/P1D"),
|
||||
"test",
|
||||
|
@ -368,7 +368,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(context)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
|
@ -432,7 +432,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(QueryContexts.override(context, ROLLUP_CONTEXT_PARAMS))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedRollUp(true)
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
|
@ -457,7 +457,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(context)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
|
@ -507,7 +507,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedFooRows())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -530,7 +530,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo1", Intervals.of("1970-01-01/P1D"), "test", 0)
|
||||
)
|
||||
|
@ -565,7 +565,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(localContext)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo1", Intervals.of("1970-01-01/P1D"), "test", 0)
|
||||
)
|
||||
|
@ -597,7 +597,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo1", Intervals.of("1999-12-31T/P1D"), "test", 0)
|
||||
)
|
||||
|
@ -637,7 +637,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
|
@ -686,7 +686,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedMultiValueFooRows())
|
||||
.verifyResults();
|
||||
}
|
||||
|
@ -704,7 +704,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedMultiValueFooRows())
|
||||
.verifyResults();
|
||||
}
|
||||
|
@ -722,7 +722,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedMultiValueFooRows())
|
||||
.verifyResults();
|
||||
}
|
||||
|
@ -740,7 +740,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(expectedMultiValueFooRowsGroupBy())
|
||||
.verifyResults();
|
||||
}
|
||||
|
@ -776,7 +776,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
NullHandling.replaceWithDefault() ?
|
||||
ImmutableList.of(
|
||||
|
@ -812,7 +812,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
NullHandling.replaceWithDefault() ?
|
||||
ImmutableList.of(
|
||||
|
@ -848,7 +848,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(adjustedContext)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
NullHandling.replaceWithDefault() ?
|
||||
ImmutableList.of(
|
||||
|
@ -911,7 +911,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedRollUp(true)
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
|
@ -969,7 +969,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedQueryGranularity(Granularities.DAY)
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
|
@ -1043,7 +1043,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedQueryGranularity(Granularities.DAY)
|
||||
.addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -1068,7 +1068,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedRollUp(true)
|
||||
.addExpectedAggregatorFactory(new HyperUniquesAggregatorFactory("cnt", "cnt", false, true))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedFooRowsWithAggregatedComplexColumn())
|
||||
.verifyResults();
|
||||
|
||||
|
@ -1102,7 +1102,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27/P1D"),
|
||||
"test",
|
||||
|
@ -1171,7 +1171,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.addExpectedAggregatorFactory(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27/P1D"),
|
||||
"test",
|
||||
|
@ -1419,6 +1419,13 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setQueryContext(context)
|
||||
.setExpectedDataSource("foo")
|
||||
.setExpectedRowSignature(expectedRowSignature)
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo", Intervals.of("2023-01-01/P1D"), "test", 0),
|
||||
SegmentId.of("foo", Intervals.of("2023-01-01/P1D"), "test", 1),
|
||||
SegmentId.of("foo", Intervals.of("2023-02-01/P1D"), "test", 0)
|
||||
)
|
||||
)
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
new Object[]{1672531200000L, "day1_1"},
|
||||
|
@ -1491,7 +1498,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(queryContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedMSQSegmentReport(
|
||||
new MSQSegmentReport(
|
||||
|
@ -1538,7 +1545,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(localContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27/P1D"),
|
||||
"test",
|
||||
|
@ -1590,7 +1597,7 @@ public class MSQInsertTest extends MSQTestBase
|
|||
.setExpectedDataSource("foo1")
|
||||
.setQueryContext(localContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27/P1D"),
|
||||
"test",
|
||||
|
|
|
@ -142,7 +142,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo", Intervals.of("2000-01-01T/P1D"), "test", 0),
|
||||
SegmentId.of("foo", Intervals.of("2000-01-02T/P1D"), "test", 0),
|
||||
|
@ -214,7 +214,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
"2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z")))
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-02T/P1D"),
|
||||
"test",
|
||||
|
@ -284,7 +284,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
"Cannot use RangeShardSpec, RangedShardSpec only supports string CLUSTER BY keys. Using NumberedShardSpec instead."
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(
|
||||
.setExpectedSegments(ImmutableSet.of(
|
||||
SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27T00:00:00.000Z/2016-06-27T01:00:00.000Z"),
|
||||
|
@ -374,7 +374,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(ImmutableList.of(Intervals.of(
|
||||
"2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z")))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo1",
|
||||
Intervals.of("2016-06-27T01:00:00.000Z/2016-06-27T02:00:00.000Z"),
|
||||
"test",
|
||||
|
@ -453,7 +453,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -469,7 +469,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
.with().totalFiles(1),
|
||||
|
@ -542,7 +542,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -558,7 +558,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(
|
||||
.setExpectedSegments(ImmutableSet.of(
|
||||
SegmentId.of("foo", Intervals.of("2000-01-01T/P1M"), "test", 0),
|
||||
SegmentId.of("foo", Intervals.of("2001-01-01T/P1M"), "test", 0)
|
||||
)
|
||||
|
@ -628,7 +628,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T")))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -640,7 +640,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{946771200000L, 2.0f}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -720,7 +720,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T")))
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2001-01-01T/2001-02-01T")))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -816,6 +816,11 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setQueryContext(context)
|
||||
.setExpectedDataSource("foo")
|
||||
.setExpectedRowSignature(expectedRowSignature)
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo", Intervals.ETERNITY, "test", 0)
|
||||
)
|
||||
)
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
new Object[]{1672531200000L, "day1_1"},
|
||||
|
@ -893,7 +898,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2000-03-01T")))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -950,7 +955,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setQueryContext(queryContext)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedShardSpec(DimensionRangeShardSpec.class)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0), SegmentId.of("foo1", Intervals.ETERNITY, "test", 1)))
|
||||
.setExpectedResultRows(expectedRows)
|
||||
.setExpectedMSQSegmentReport(
|
||||
new MSQSegmentReport(
|
||||
|
@ -1009,7 +1014,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Collections.singletonList(Intervals.of("2000-01-01T/2002-01-01T")))
|
||||
.setExpectedTombstoneIntervals(ImmutableSet.of(Intervals.of("2001-01-01T/2001-02-01T")))
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -1071,7 +1076,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
Intervals.of("%s/%s", "2000-02-01", Intervals.ETERNITY.getEnd())
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -1119,7 +1124,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(expectedFooSegments())
|
||||
.setExpectedSegments(expectedFooSegments())
|
||||
.setExpectedResultRows(expectedFooRows())
|
||||
.setExpectedLastCompactionState(
|
||||
expectedCompactionState(
|
||||
|
@ -1161,7 +1166,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedShardSpec(DimensionRangeShardSpec.class)
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, NullHandling.sqlCompatible() ? "" : null, 1.0f, 1L},
|
||||
|
@ -1206,7 +1211,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foobar", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
new Object[]{946684800000L, 1.0f},
|
||||
|
@ -1248,7 +1253,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of(
|
||||
"foobar",
|
||||
|
@ -1324,7 +1329,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -1348,7 +1353,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{978480000000L, null}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
.with().totalFiles(1),
|
||||
|
@ -1399,7 +1404,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
.setExpectedRowSignature(rowSignature)
|
||||
.setQueryContext(context)
|
||||
.setExpectedDestinationIntervals(Intervals.ONLY_ETERNITY)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of(
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
|
||||
"foo",
|
||||
Intervals.of("2000-01-01T/P1M"),
|
||||
"test",
|
||||
|
@ -1421,7 +1426,7 @@ public class MSQReplaceTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedCountersForStageWorkerChannel(
|
||||
CounterSnapshotMatcher
|
||||
.with().totalFiles(1),
|
||||
|
|
|
@ -1584,7 +1584,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 6.0, null}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1615,7 +1615,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 6.0}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1646,7 +1646,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 21.0}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1677,7 +1677,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 21.0}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1709,7 +1709,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 21.0, 21.0}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
|
@ -1741,7 +1741,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{978480000000L, 6.0f, 6.0, 6.0}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(
|
||||
.setExpectedSegments(
|
||||
ImmutableSet.of(
|
||||
SegmentId.of("foo1", Intervals.of("2000-01-01T/P1D"), "test", 0),
|
||||
SegmentId.of("foo1", Intervals.of("2000-01-02T/P1D"), "test", 0),
|
||||
|
@ -2045,7 +2045,7 @@ public class MSQWindowTest extends MSQTestBase
|
|||
new Object[]{0L, 129L, "Albuquerque", 140L}
|
||||
)
|
||||
)
|
||||
.setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.setExpectedSegments(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0)))
|
||||
.verifyResults();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,54 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.exec;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.indexing.MSQWorkerTask;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
|
||||
@RunWith(MockitoJUnitRunner.class)
|
||||
public class WorkerImplTest
|
||||
{
|
||||
@Mock
|
||||
WorkerContext workerContext;
|
||||
|
||||
@Test
|
||||
public void testFetchStatsThrows()
|
||||
{
|
||||
WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE));
|
||||
Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshot(new StageId("xx", 1)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchStatsWithTimeChunkThrows()
|
||||
{
|
||||
WorkerImpl worker = new WorkerImpl(new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0), workerContext, WorkerStorageParameters.createInstanceForTests(Long.MAX_VALUE));
|
||||
Assert.assertThrows(ISE.class, () -> worker.fetchStatisticsSnapshotForTimeChunk(new StageId("xx", 1), 1L));
|
||||
}
|
||||
|
||||
}
|
|
@ -32,34 +32,54 @@ public class WorkerMemoryParametersTest
|
|||
@Test
|
||||
public void test_oneWorkerInJvm_alone()
|
||||
{
|
||||
Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 0, 0));
|
||||
Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 0, 0));
|
||||
Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 0, 0));
|
||||
Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 0, 0));
|
||||
Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 1, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(1_000_000_000, 1, 32, 1, 0, 0)
|
||||
() -> create(1_000_000_000, 1, 32, 1, 1, 0, 0)
|
||||
);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault());
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault());
|
||||
|
||||
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0))
|
||||
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0))
|
||||
.getFault();
|
||||
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_oneWorkerInJvm_alone_twoConcurrentStages()
|
||||
{
|
||||
Assert.assertEquals(params(166_750_000, 1, 20, 37_500_000), create(1_000_000_000, 1, 1, 2, 1, 0, 0));
|
||||
Assert.assertEquals(params(110_500_000, 2, 6, 37_500_000), create(1_000_000_000, 1, 2, 2, 1, 0, 0));
|
||||
Assert.assertEquals(params(65_500_000, 2, 3, 37_500_000), create(1_000_000_000, 1, 4, 2, 1, 0, 0));
|
||||
Assert.assertEquals(params(35_500_000, 1, 3, 37_500_000), create(1_000_000_000, 1, 8, 2, 1, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(1_000_000_000, 1, 12, 2, 1, 0, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(1_736_034_666, 1_000_000_000, 750_000_000, 1, 12, 2), e.getFault());
|
||||
|
||||
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 2, 1, 0, 0))
|
||||
.getFault();
|
||||
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(4_048_090_666L, 1_000_000_000, 750_000_000, 2, 32, 2), fault);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_oneWorkerInJvm_twoHundredWorkersInCluster()
|
||||
{
|
||||
Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 200, 0, 0));
|
||||
Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 1, 200, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(1_000_000_000, 1, 4, 200, 0, 0)
|
||||
() -> create(1_000_000_000, 1, 4, 1, 200, 0, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault());
|
||||
|
@ -68,76 +88,102 @@ public class WorkerMemoryParametersTest
|
|||
@Test
|
||||
public void test_fourWorkersInJvm_twoHundredWorkersInCluster()
|
||||
{
|
||||
Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 0, 0));
|
||||
Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 0, 0));
|
||||
Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 0, 0));
|
||||
Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 200, 0, 0));
|
||||
Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 0, 0));
|
||||
Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 200, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(8_000_000_000L, 4, 32, 200, 0, 0)
|
||||
() -> create(8_000_000_000L, 4, 32, 1, 200, 0, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault());
|
||||
|
||||
// Make sure 124 actually works, and 125 doesn't. (Verify the error message above.)
|
||||
Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 124, 0, 0));
|
||||
Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 1, 124, 0, 0));
|
||||
|
||||
final MSQException e2 = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(8_000_000_000L, 4, 32, 125, 0, 0)
|
||||
() -> create(8_000_000_000L, 4, 32, 1, 125, 0, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(125, 124), e2.getFault());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_oneWorkerInJvm_smallWorkerCapacity()
|
||||
public void test_fourWorkersInJvm_twoHundredWorkersInCluster_twoConcurrentStages()
|
||||
{
|
||||
// Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels
|
||||
Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 0, 0));
|
||||
Assert.assertEquals(params(406_500_000, 1, 74, 84_375_000), create(9_000_000_000L, 4, 1, 2, 200, 0, 0));
|
||||
Assert.assertEquals(params(305_250_000, 2, 30, 84_375_000), create(9_000_000_000L, 4, 2, 2, 200, 0, 0));
|
||||
Assert.assertEquals(params(178_687_500, 4, 10, 84_375_000), create(9_000_000_000L, 4, 4, 2, 200, 0, 0));
|
||||
Assert.assertEquals(params(52_125_000, 4, 6, 84_375_000), create(9_000_000_000L, 4, 8, 2, 200, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(1_000_000_000, 1, 32, 1, 0, 0)
|
||||
() -> create(8_000_000_000L, 4, 16, 2, 200, 0, 0)
|
||||
);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32), e.getFault());
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault());
|
||||
|
||||
// Make sure 109 actually works, and 110 doesn't. (Verify the error message above.)
|
||||
Assert.assertEquals(params(25_000_000, 4, 3, 75_000_000), create(8_000_000_000L, 4, 16, 2, 109, 0, 0));
|
||||
|
||||
final MSQException e2 = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(128_000_000, 1, 4, 1, 0, 0)
|
||||
() -> create(8_000_000_000L, 4, 16, 2, 110, 0, 0)
|
||||
);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4), e2.getFault());
|
||||
|
||||
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 0, 0))
|
||||
Assert.assertEquals(new TooManyWorkersFault(110, 109), e2.getFault());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_oneWorkerInJvm_smallWorkerCapacity()
|
||||
{
|
||||
// Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels
|
||||
Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 1, 0, 0));
|
||||
Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 1, 0, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(1_000_000_000, 1, 32, 1, 1, 0, 0)
|
||||
);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault());
|
||||
|
||||
final MSQException e2 = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(128_000_000, 1, 4, 1, 1, 0, 0)
|
||||
);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4, 1), e2.getFault());
|
||||
|
||||
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0))
|
||||
.getFault();
|
||||
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32), fault);
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions()
|
||||
{
|
||||
Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 200, 200, 0));
|
||||
Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 200, 200, 0));
|
||||
Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 200, 200, 0));
|
||||
Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 200, 200, 0));
|
||||
Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 200, 0));
|
||||
Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 200, 0));
|
||||
Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 200, 0));
|
||||
Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 200, 0));
|
||||
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(9_000_000_000L, 4, 16, 200, 200, 0)
|
||||
() -> create(9_000_000_000L, 4, 16, 1, 200, 200, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault());
|
||||
|
||||
// Make sure 138 actually works, and 139 doesn't. (Verify the error message above.)
|
||||
Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 138, 138, 0));
|
||||
Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 138, 138, 0));
|
||||
|
||||
final MSQException e2 = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> create(9_000_000_000L, 4, 16, 139, 139, 0)
|
||||
() -> create(9_000_000_000L, 4, 16, 1, 139, 139, 0)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new TooManyWorkersFault(139, 138), e2.getFault());
|
||||
|
@ -148,10 +194,10 @@ public class WorkerMemoryParametersTest
|
|||
{
|
||||
final MSQException e = Assert.assertThrows(
|
||||
MSQException.class,
|
||||
() -> WorkerMemoryParameters.createInstance(1, 1, 1, 32, 1, 1)
|
||||
() -> WorkerMemoryParameters.createInstance(1, 1, 1, 1, 32, 1, 1)
|
||||
);
|
||||
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1), e.getFault());
|
||||
Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1, 1), e.getFault());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -179,6 +225,7 @@ public class WorkerMemoryParametersTest
|
|||
final long maxMemoryInJvm,
|
||||
final int numWorkersInJvm,
|
||||
final int numProcessingThreadsInJvm,
|
||||
final int maxConcurrentStages,
|
||||
final int numInputWorkers,
|
||||
final int numHashOutputPartitions,
|
||||
final int totalLookUpFootprint
|
||||
|
@ -188,6 +235,7 @@ public class WorkerMemoryParametersTest
|
|||
maxMemoryInJvm,
|
||||
numWorkersInJvm,
|
||||
numProcessingThreadsInJvm,
|
||||
maxConcurrentStages,
|
||||
numInputWorkers,
|
||||
numHashOutputPartitions,
|
||||
totalLookUpFootprint
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.msq.indexing;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
|
||||
|
@ -30,6 +31,7 @@ import org.apache.druid.rpc.ServiceLocator;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.quality.Strictness;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
|
@ -44,12 +46,19 @@ public class IndexerWorkerContextTest
|
|||
Mockito.when(injectorMock.getInstance(SegmentCacheManagerFactory.class))
|
||||
.thenReturn(Mockito.mock(SegmentCacheManagerFactory.class));
|
||||
|
||||
final MSQWorkerTask task =
|
||||
Mockito.mock(MSQWorkerTask.class, Mockito.withSettings().strictness(Strictness.STRICT_STUBS));
|
||||
Mockito.when(task.getContext()).thenReturn(ImmutableMap.of());
|
||||
|
||||
indexerWorkerContext = new IndexerWorkerContext(
|
||||
task,
|
||||
Mockito.mock(TaskToolbox.class),
|
||||
injectorMock,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
|
@ -19,12 +19,8 @@
|
|||
|
||||
package org.apache.druid.msq.indexing;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.druid.frame.key.ClusterByPartitions;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
|
@ -32,12 +28,9 @@ import org.apache.druid.msq.indexing.client.WorkerChatHandler;
|
|||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.msq.kernel.WorkOrder;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthConfig;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -51,15 +44,16 @@ import javax.annotation.Nullable;
|
|||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.io.InputStream;
|
||||
import java.util.HashMap;
|
||||
|
||||
public class WorkerChatHandlerTest
|
||||
{
|
||||
private static final StageId TEST_STAGE = new StageId("123", 0);
|
||||
private static final String DATASOURCE = "foo";
|
||||
|
||||
@Mock
|
||||
private HttpServletRequest req;
|
||||
|
||||
private TaskToolbox toolbox;
|
||||
private AuthorizerMapper authorizerMapper;
|
||||
private AutoCloseable mocks;
|
||||
|
||||
private final TestWorker worker = new TestWorker();
|
||||
|
@ -67,29 +61,16 @@ public class WorkerChatHandlerTest
|
|||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
mapper,
|
||||
indexIO,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance()
|
||||
);
|
||||
|
||||
authorizerMapper = CalciteTests.TEST_AUTHORIZER_MAPPER;
|
||||
mocks = MockitoAnnotations.openMocks(this);
|
||||
Mockito.when(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT))
|
||||
.thenReturn(new AuthenticationResult("druid", "druid", null, null));
|
||||
TaskToolbox.Builder builder = new TaskToolbox.Builder();
|
||||
toolbox = builder.authorizerMapper(CalciteTests.TEST_AUTHORIZER_MAPPER)
|
||||
.indexIO(indexIO)
|
||||
.indexMergerV9(indexMerger)
|
||||
.taskReportFileWriter(new NoopTestTaskReportFileWriter())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchSnapshot()
|
||||
{
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE);
|
||||
Assert.assertEquals(
|
||||
ClusterByStatisticsSnapshot.empty(),
|
||||
chatHandler.httpFetchKeyStatistics(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), null, req)
|
||||
|
@ -100,7 +81,7 @@ public class WorkerChatHandlerTest
|
|||
@Test
|
||||
public void testFetchSnapshot404()
|
||||
{
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE);
|
||||
Assert.assertEquals(
|
||||
Response.Status.BAD_REQUEST.getStatusCode(),
|
||||
chatHandler.httpFetchKeyStatistics("123", 2, null, req)
|
||||
|
@ -111,7 +92,7 @@ public class WorkerChatHandlerTest
|
|||
@Test
|
||||
public void testFetchSnapshotWithTimeChunk()
|
||||
{
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE);
|
||||
Assert.assertEquals(
|
||||
ClusterByStatisticsSnapshot.empty(),
|
||||
chatHandler.httpFetchKeyStatisticsWithSnapshot(TEST_STAGE.getQueryId(), TEST_STAGE.getStageNumber(), 1, null, req)
|
||||
|
@ -122,7 +103,7 @@ public class WorkerChatHandlerTest
|
|||
@Test
|
||||
public void testFetchSnapshotWithTimeChunk404()
|
||||
{
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(toolbox, worker);
|
||||
WorkerChatHandler chatHandler = new WorkerChatHandler(worker, authorizerMapper, DATASOURCE);
|
||||
Assert.assertEquals(
|
||||
Response.Status.BAD_REQUEST.getStatusCode(),
|
||||
chatHandler.httpFetchKeyStatisticsWithSnapshot("123", 2, 1, null, req)
|
||||
|
@ -133,7 +114,6 @@ public class WorkerChatHandlerTest
|
|||
|
||||
private static class TestWorker implements Worker
|
||||
{
|
||||
|
||||
@Override
|
||||
public String id()
|
||||
{
|
||||
|
@ -141,19 +121,13 @@ public class WorkerChatHandlerTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public MSQWorkerTask task()
|
||||
public void run()
|
||||
{
|
||||
return new MSQWorkerTask("controller", "ds", 1, new HashMap<>(), 0);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stopGracefully()
|
||||
public void stop()
|
||||
{
|
||||
|
||||
}
|
||||
|
@ -164,6 +138,12 @@ public class WorkerChatHandlerTest
|
|||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void awaitStop()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postWorkOrder(WorkOrder workOrder)
|
||||
{
|
||||
|
@ -192,9 +172,8 @@ public class WorkerChatHandlerTest
|
|||
|
||||
@Override
|
||||
public boolean postResultPartitionBoundaries(
|
||||
ClusterByPartitions stagePartitionBoundaries,
|
||||
String queryId,
|
||||
int stageNumber
|
||||
StageId stageId,
|
||||
ClusterByPartitions stagePartitionBoundaries
|
||||
)
|
||||
{
|
||||
return false;
|
||||
|
@ -202,7 +181,7 @@ public class WorkerChatHandlerTest
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public InputStream readChannel(String queryId, int stageNumber, int partitionNumber, long offset)
|
||||
public ListenableFuture<InputStream> readStageOutput(StageId stageId, int partitionNumber, long offset)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -74,7 +74,7 @@ public class MSQFaultSerdeTest
|
|||
));
|
||||
assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2));
|
||||
assertFaultSerde(new InvalidFieldFault("the source", "the column", 1, "the error", "the log msg"));
|
||||
assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2));
|
||||
assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2, 2));
|
||||
assertFaultSerde(QueryNotSupportedFault.INSTANCE);
|
||||
assertFaultSerde(new QueryRuntimeFault("new error", "base error"));
|
||||
assertFaultSerde(new QueryRuntimeFault("new error", null));
|
||||
|
|
|
@ -0,0 +1,149 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
|
||||
public class ByteChunksInputStreamTest
|
||||
{
|
||||
private final List<byte[]> chunks = ImmutableList.of(
|
||||
new byte[]{-128, -127, -1, 0, 1, 126, 127},
|
||||
new byte[]{0},
|
||||
new byte[]{3, 4, 5}
|
||||
);
|
||||
|
||||
@Test
|
||||
public void test_read_fromStart() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 0)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
||||
int c;
|
||||
while ((c = in.read()) != -1) {
|
||||
MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0));
|
||||
baos.write(c);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_read_fromSecondByte() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 1)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
||||
int c;
|
||||
while ((c = in.read()) != -1) {
|
||||
MatcherAssert.assertThat("InputStream#read contract", c, Matchers.greaterThanOrEqualTo(0));
|
||||
baos.write(c);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_read_array1_fromStart() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 0)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final byte[] buf = new byte[2];
|
||||
|
||||
int r;
|
||||
while ((r = in.read(buf, 1, 1)) != -1) {
|
||||
Assert.assertEquals("InputStream#read bytes read", 1, r);
|
||||
baos.write(buf, 1, 1);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_read_array1_fromSecondByte() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 1)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final byte[] buf = new byte[2];
|
||||
|
||||
int r;
|
||||
while ((r = in.read(buf, 1, 1)) != -1) {
|
||||
Assert.assertEquals("InputStream#read bytes read", 1, r);
|
||||
baos.write(buf, 1, 1);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_read_array3_fromStart() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 0)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final byte[] buf = new byte[5];
|
||||
|
||||
int r;
|
||||
while ((r = in.read(buf, 2, 3)) != -1) {
|
||||
baos.write(buf, 2, r);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(0), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_read_array3_fromSecondByte() throws IOException
|
||||
{
|
||||
try (final InputStream in = new ByteChunksInputStream(chunks, 1)) {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final byte[] buf = new byte[6];
|
||||
|
||||
int r;
|
||||
while ((r = in.read(buf, 2, 3)) != -1) {
|
||||
baos.write(buf, 2, r);
|
||||
}
|
||||
|
||||
Assert.assertArrayEquals(chunksSubset(1), baos.toByteArray());
|
||||
}
|
||||
}
|
||||
|
||||
private byte[] chunksSubset(final int positionInFirstChunk)
|
||||
{
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
||||
for (int chunk = 0, p = positionInFirstChunk; chunk < chunks.size(); chunk++, p = 0) {
|
||||
baos.write(chunks.get(chunk), p, chunks.get(chunk).length - p);
|
||||
}
|
||||
|
||||
return baos.toByteArray();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,255 @@
|
|||
/*
|
||||
* 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.msq.shuffle.output;
|
||||
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.math.IntMath;
|
||||
import org.apache.druid.common.guava.FutureUtils;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.FrameType;
|
||||
import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableFileFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFile;
|
||||
import org.apache.druid.frame.read.FrameReader;
|
||||
import org.apache.druid.frame.testutil.FrameSequenceBuilder;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.MatcherAssert;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableCauseMatcher;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.math.RoundingMode;
|
||||
import java.util.List;
|
||||
|
||||
public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final int MAX_FRAMES = 10;
|
||||
private static final int EXPECTED_NUM_ROWS = 1209;
|
||||
|
||||
private final BlockingQueueFrameChannel channel = new BlockingQueueFrameChannel(MAX_FRAMES);
|
||||
private final ChannelStageOutputReader reader = new ChannelStageOutputReader(channel.readable());
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
private FrameReader frameReader;
|
||||
private List<Frame> frameList;
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
final IncrementalIndex index = TestIndex.getIncrementalTestIndex();
|
||||
final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter(index);
|
||||
frameReader = FrameReader.create(adapter.getRowSignature());
|
||||
frameList = FrameSequenceBuilder.fromAdapter(adapter)
|
||||
.frameType(FrameType.ROW_BASED)
|
||||
.maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING))
|
||||
.frames()
|
||||
.toList();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown()
|
||||
{
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readLocally() throws IOException
|
||||
{
|
||||
writeAllFramesToChannel();
|
||||
|
||||
Assert.assertSame(channel.readable(), reader.readLocally());
|
||||
reader.close(); // Won't close the channel, because it's already been returned by readLocally
|
||||
|
||||
final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size();
|
||||
Assert.assertEquals(EXPECTED_NUM_ROWS, numRows);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readLocally_closePriorToRead() throws IOException
|
||||
{
|
||||
writeAllFramesToChannel();
|
||||
|
||||
reader.close();
|
||||
|
||||
// Can't read the channel after closing the reader
|
||||
Assert.assertThrows(
|
||||
IllegalStateException.class,
|
||||
reader::readLocally
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readLocally_thenReadRemotely() throws IOException
|
||||
{
|
||||
writeAllFramesToChannel();
|
||||
|
||||
Assert.assertSame(channel.readable(), reader.readLocally());
|
||||
|
||||
// Can't read remotely after reading locally
|
||||
Assert.assertThrows(
|
||||
IllegalStateException.class,
|
||||
() -> reader.readRemotelyFrom(0)
|
||||
);
|
||||
|
||||
// Can still read locally after this error
|
||||
final int numRows = FrameTestUtil.readRowsFromFrameChannel(channel.readable(), frameReader).toList().size();
|
||||
Assert.assertEquals(EXPECTED_NUM_ROWS, numRows);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readRemotely_strideBasedOnReturnedChunk() throws IOException
|
||||
{
|
||||
// Test that reads entire chunks from readRemotelyFrom. This is a typical usage pattern.
|
||||
|
||||
writeAllFramesToChannel();
|
||||
|
||||
final File tmpFile = temporaryFolder.newFile();
|
||||
|
||||
try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) {
|
||||
int numReads = 0;
|
||||
long offset = 0;
|
||||
|
||||
while (true) {
|
||||
try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) {
|
||||
numReads++;
|
||||
final long bytesWritten = ByteStreams.copy(in, tmpOut);
|
||||
offset += bytesWritten;
|
||||
|
||||
if (bytesWritten == 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
MatcherAssert.assertThat(numReads, Matchers.greaterThan(1));
|
||||
}
|
||||
|
||||
final FrameFile frameFile = FrameFile.open(tmpFile, null);
|
||||
final int numRows =
|
||||
FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size();
|
||||
|
||||
Assert.assertEquals(EXPECTED_NUM_ROWS, numRows);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readRemotely_strideOneByte() throws IOException
|
||||
{
|
||||
// Test that reads one byte at a time from readRemotelyFrom. This helps ensure that there are no edge cases
|
||||
// in the chunk-reading logic.
|
||||
|
||||
writeAllFramesToChannel();
|
||||
|
||||
final File tmpFile = temporaryFolder.newFile();
|
||||
|
||||
try (final FileOutputStream tmpOut = new FileOutputStream(tmpFile)) {
|
||||
int numReads = 0;
|
||||
long offset = 0;
|
||||
|
||||
while (true) {
|
||||
try (final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(offset), true)) {
|
||||
numReads++;
|
||||
final int nextByte = in.read();
|
||||
|
||||
if (nextByte < 0) {
|
||||
break;
|
||||
}
|
||||
|
||||
tmpOut.write(nextByte);
|
||||
offset++;
|
||||
}
|
||||
}
|
||||
|
||||
Assert.assertEquals(numReads, offset + 1);
|
||||
}
|
||||
|
||||
final FrameFile frameFile = FrameFile.open(tmpFile, null);
|
||||
final int numRows =
|
||||
FrameTestUtil.readRowsFromFrameChannel(new ReadableFileFrameChannel(frameFile), frameReader).toList().size();
|
||||
|
||||
Assert.assertEquals(EXPECTED_NUM_ROWS, numRows);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readRemotely_thenLocally() throws IOException
|
||||
{
|
||||
writeAllFramesToChannel();
|
||||
|
||||
// Read remotely
|
||||
FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true);
|
||||
|
||||
// Then read locally
|
||||
Assert.assertThrows(
|
||||
IllegalStateException.class,
|
||||
reader::readLocally
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_readRemotely_cannotReverse() throws IOException
|
||||
{
|
||||
writeAllFramesToChannel();
|
||||
|
||||
// Read remotely from offset = 1.
|
||||
final InputStream in = FutureUtils.getUnchecked(reader.readRemotelyFrom(1), true);
|
||||
final int offset = ByteStreams.toByteArray(in).length;
|
||||
MatcherAssert.assertThat(offset, Matchers.greaterThan(0));
|
||||
|
||||
// Then read again from offset = 0; should get an error.
|
||||
final RuntimeException e = Assert.assertThrows(
|
||||
RuntimeException.class,
|
||||
() -> FutureUtils.getUnchecked(reader.readRemotelyFrom(0), true)
|
||||
);
|
||||
|
||||
MatcherAssert.assertThat(
|
||||
e,
|
||||
ThrowableCauseMatcher.hasCause(
|
||||
Matchers.allOf(
|
||||
CoreMatchers.instanceOf(IllegalStateException.class),
|
||||
ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Offset[0] no longer available"))
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private void writeAllFramesToChannel() throws IOException
|
||||
{
|
||||
for (Frame frame : frameList) {
|
||||
channel.writable().write(frame);
|
||||
}
|
||||
channel.writable().close();
|
||||
}
|
||||
}
|
|
@ -71,14 +71,6 @@ public class SqlStatementResultTest
|
|||
{
|
||||
|
||||
Assert.assertEquals(JSON_STRING, MAPPER.writeValueAsString(SQL_STATEMENT_RESULT));
|
||||
Assert.assertEquals(
|
||||
SQL_STATEMENT_RESULT,
|
||||
MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
SQL_STATEMENT_RESULT.hashCode(),
|
||||
MAPPER.readValue(MAPPER.writeValueAsString(SQL_STATEMENT_RESULT), SqlStatementResult.class).hashCode()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"SqlStatementResult{"
|
||||
+ "queryId='q1',"
|
||||
|
@ -87,7 +79,10 @@ public class SqlStatementResultTest
|
|||
+ " sqlRowSignature=[ColumnNameAndTypes{colName='_time', sqlTypeName='TIMESTAMP', nativeTypeName='LONG'}, ColumnNameAndTypes{colName='alias', sqlTypeName='VARCHAR', nativeTypeName='STRING'}, ColumnNameAndTypes{colName='market', sqlTypeName='VARCHAR', nativeTypeName='STRING'}],"
|
||||
+ " durationInMs=100,"
|
||||
+ " resultSetInformation=ResultSetInformation{numTotalRows=1, totalSizeInBytes=1, resultFormat=object, records=null, dataSource='ds', pages=[PageInformation{id=0, numRows=null, sizeInBytes=1, worker=null, partition=null}]},"
|
||||
+ " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}}}",
|
||||
+ " errorResponse={error=druidException, errorCode=QueryNotSupported, persona=USER, category=UNCATEGORIZED, errorMessage=QueryNotSupported, context={}},"
|
||||
+ " stages=null,"
|
||||
+ " counters=null,"
|
||||
+ " warnings=null}",
|
||||
SQL_STATEMENT_RESULT.toString()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -206,7 +206,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
new ResultSetInformation(0L, 0L, null, "foo1", null, null),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(expected, actual);
|
||||
assertSqlStatementResult(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -236,7 +236,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
new ResultSetInformation(0L, 0L, null, "foo1", null, null),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(expected, actual);
|
||||
assertSqlStatementResult(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -282,7 +282,7 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
}
|
||||
}).toErrorResponse()
|
||||
);
|
||||
Assert.assertEquals(expected, actual);
|
||||
assertSqlStatementResult(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -687,11 +687,11 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(expected, actual);
|
||||
assertSqlStatementResult(expected, actual);
|
||||
|
||||
Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest());
|
||||
Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus());
|
||||
Assert.assertEquals(expected, getResponse.getEntity());
|
||||
assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity());
|
||||
|
||||
Response resultsResponse = resource.doGetResults(
|
||||
actual.getQueryId(),
|
||||
|
@ -730,11 +730,11 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
new ResultSetInformation(NullHandling.sqlCompatible() ? 6L : 5L, 0L, null, "foo1", null, null),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(expected, actual);
|
||||
assertSqlStatementResult(expected, actual);
|
||||
|
||||
Response getResponse = resource.doGetStatus(actual.getQueryId(), SqlStatementResourceTest.makeOkRequest());
|
||||
Response getResponse = resource.doGetStatus(actual.getQueryId(), false, SqlStatementResourceTest.makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), getResponse.getStatus());
|
||||
Assert.assertEquals(expected, getResponse.getEntity());
|
||||
assertSqlStatementResult(expected, (SqlStatementResult) getResponse.getEntity());
|
||||
|
||||
Response resultsResponse = resource.doGetResults(
|
||||
actual.getQueryId(),
|
||||
|
@ -754,4 +754,27 @@ public class SqlMSQStatementResourcePostTest extends MSQTestBase
|
|||
return context;
|
||||
}
|
||||
|
||||
private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual)
|
||||
{
|
||||
Assert.assertEquals(expected.getQueryId(), actual.getQueryId());
|
||||
Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt());
|
||||
Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature());
|
||||
Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs());
|
||||
Assert.assertEquals(expected.getStages(), actual.getStages());
|
||||
Assert.assertEquals(expected.getState(), actual.getState());
|
||||
Assert.assertEquals(expected.getWarnings(), actual.getWarnings());
|
||||
Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation());
|
||||
|
||||
if (actual.getCounters() == null || expected.getCounters() == null) {
|
||||
Assert.assertEquals(expected.getCounters(), actual.getCounters());
|
||||
} else {
|
||||
Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString());
|
||||
}
|
||||
|
||||
if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) {
|
||||
Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse());
|
||||
} else {
|
||||
Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -97,6 +97,7 @@ import javax.ws.rs.core.Response;
|
|||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -449,7 +450,11 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
)));
|
||||
|
||||
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(FINISHED_SELECT_MSQ_QUERY))
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_SELECT_MSQ_QUERY)))
|
||||
.thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get())));
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_SELECT_MSQ_QUERY)))
|
||||
.thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get())));
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_SELECT_MSQ_QUERY)))
|
||||
.thenAnswer(inv -> Futures.immediateFuture(TaskReport.buildTaskReports(selectTaskReport.get())));
|
||||
|
||||
Mockito.when(indexingServiceClient.taskStatus(ArgumentMatchers.eq(ERRORED_SELECT_MSQ_QUERY)))
|
||||
|
@ -584,6 +589,10 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(FINISHED_INSERT_MSQ_QUERY)))
|
||||
.thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT)));
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(ACCEPTED_INSERT_MSQ_TASK)))
|
||||
.thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT)));
|
||||
Mockito.when(indexingServiceClient.taskReportAsMap(ArgumentMatchers.eq(RUNNING_INSERT_MSQ_QUERY)))
|
||||
.thenReturn(Futures.immediateFuture(TaskReport.buildTaskReports(MSQ_INSERT_TASK_REPORT)));
|
||||
|
||||
Mockito.when(indexingServiceClient.taskPayload(FINISHED_INSERT_MSQ_QUERY))
|
||||
.thenReturn(Futures.immediateFuture(new TaskPayloadResponse(
|
||||
|
@ -690,9 +699,9 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
@Test
|
||||
public void testMSQSelectAcceptedQuery()
|
||||
{
|
||||
Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, makeOkRequest());
|
||||
Response response = resource.doGetStatus(ACCEPTED_SELECT_MSQ_QUERY, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(
|
||||
assertSqlStatementResult(
|
||||
new SqlStatementResult(
|
||||
ACCEPTED_SELECT_MSQ_QUERY,
|
||||
SqlStatementState.ACCEPTED,
|
||||
|
@ -702,7 +711,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
response.getEntity()
|
||||
(SqlStatementResult) response.getEntity()
|
||||
);
|
||||
|
||||
assertExceptionMessage(
|
||||
|
@ -724,9 +733,9 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
public void testMSQSelectRunningQuery()
|
||||
{
|
||||
|
||||
Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, makeOkRequest());
|
||||
Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(
|
||||
assertSqlStatementResult(
|
||||
new SqlStatementResult(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
SqlStatementState.RUNNING,
|
||||
|
@ -736,7 +745,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
response.getEntity()
|
||||
(SqlStatementResult) response.getEntity()
|
||||
);
|
||||
|
||||
assertExceptionMessage(
|
||||
|
@ -754,10 +763,40 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQSelectRunningQueryWithDetail()
|
||||
{
|
||||
Response response = resource.doGetStatus(RUNNING_SELECT_MSQ_QUERY, true, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
|
||||
SqlStatementResult expectedSqlStatementResult = new SqlStatementResult(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
SqlStatementState.RUNNING,
|
||||
CREATED_TIME,
|
||||
COL_NAME_AND_TYPES,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
selectTaskReport.get().getPayload().getStages(),
|
||||
selectTaskReport.get().getPayload().getCounters(),
|
||||
new ArrayList<>(selectTaskReport.get().getPayload().getStatus().getWarningReports())
|
||||
);
|
||||
|
||||
assertSqlStatementResult(
|
||||
expectedSqlStatementResult,
|
||||
(SqlStatementResult) response.getEntity()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
Response.Status.ACCEPTED.getStatusCode(),
|
||||
resource.deleteQuery(RUNNING_SELECT_MSQ_QUERY, makeOkRequest()).getStatus()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFinishedSelectMSQQuery() throws Exception
|
||||
{
|
||||
Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, makeOkRequest());
|
||||
Response response = resource.doGetStatus(FINISHED_SELECT_MSQ_QUERY, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(objectMapper.writeValueAsString(new SqlStatementResult(
|
||||
FINISHED_SELECT_MSQ_QUERY,
|
||||
|
@ -825,7 +864,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
public void testFailedMSQQuery()
|
||||
{
|
||||
for (String queryID : ImmutableList.of(ERRORED_SELECT_MSQ_QUERY, ERRORED_INSERT_MSQ_QUERY)) {
|
||||
assertExceptionMessage(resource.doGetStatus(queryID, makeOkRequest()), FAILURE_MSG, Response.Status.OK);
|
||||
assertExceptionMessage(resource.doGetStatus(queryID, false, makeOkRequest()), FAILURE_MSG, Response.Status.OK);
|
||||
assertExceptionMessage(
|
||||
resource.doGetResults(queryID, 0L, null, makeOkRequest()),
|
||||
StringUtils.format(
|
||||
|
@ -845,9 +884,9 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
@Test
|
||||
public void testFinishedInsertMSQQuery()
|
||||
{
|
||||
Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, makeOkRequest());
|
||||
Response response = resource.doGetStatus(FINISHED_INSERT_MSQ_QUERY, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(new SqlStatementResult(
|
||||
assertSqlStatementResult(new SqlStatementResult(
|
||||
FINISHED_INSERT_MSQ_QUERY,
|
||||
SqlStatementState.SUCCESS,
|
||||
CREATED_TIME,
|
||||
|
@ -855,7 +894,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
100L,
|
||||
new ResultSetInformation(null, null, null, "test", null, null),
|
||||
null
|
||||
), response.getEntity());
|
||||
), (SqlStatementResult) response.getEntity());
|
||||
|
||||
Assert.assertEquals(
|
||||
Response.Status.OK.getStatusCode(),
|
||||
|
@ -876,7 +915,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
public void testNonMSQTasks()
|
||||
{
|
||||
for (String queryID : ImmutableList.of(RUNNING_NON_MSQ_TASK, FAILED_NON_MSQ_TASK, FINISHED_NON_MSQ_TASK)) {
|
||||
assertNotFound(resource.doGetStatus(queryID, makeOkRequest()), queryID);
|
||||
assertNotFound(resource.doGetStatus(queryID, false, makeOkRequest()), queryID);
|
||||
assertNotFound(resource.doGetResults(queryID, 0L, null, makeOkRequest()), queryID);
|
||||
assertNotFound(resource.deleteQuery(queryID, makeOkRequest()), queryID);
|
||||
}
|
||||
|
@ -885,9 +924,9 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
@Test
|
||||
public void testMSQInsertAcceptedQuery()
|
||||
{
|
||||
Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, makeOkRequest());
|
||||
Response response = resource.doGetStatus(ACCEPTED_INSERT_MSQ_TASK, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(
|
||||
assertSqlStatementResult(
|
||||
new SqlStatementResult(
|
||||
ACCEPTED_INSERT_MSQ_TASK,
|
||||
SqlStatementState.ACCEPTED,
|
||||
|
@ -897,7 +936,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
response.getEntity()
|
||||
(SqlStatementResult) response.getEntity()
|
||||
);
|
||||
|
||||
assertExceptionMessage(
|
||||
|
@ -918,9 +957,9 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
@Test
|
||||
public void testMSQInsertRunningQuery()
|
||||
{
|
||||
Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, makeOkRequest());
|
||||
Response response = resource.doGetStatus(RUNNING_INSERT_MSQ_QUERY, false, makeOkRequest());
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
|
||||
Assert.assertEquals(
|
||||
assertSqlStatementResult(
|
||||
new SqlStatementResult(
|
||||
RUNNING_INSERT_MSQ_QUERY,
|
||||
SqlStatementState.RUNNING,
|
||||
|
@ -930,7 +969,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
null,
|
||||
null
|
||||
),
|
||||
response.getEntity()
|
||||
(SqlStatementResult) response.getEntity()
|
||||
);
|
||||
|
||||
assertExceptionMessage(
|
||||
|
@ -955,6 +994,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
Response.Status.OK.getStatusCode(),
|
||||
resource.doGetStatus(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
false,
|
||||
makeExpectedReq(makeAuthResultForUser(SUPERUSER))
|
||||
).getStatus()
|
||||
);
|
||||
|
@ -984,6 +1024,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
Response.Status.FORBIDDEN.getStatusCode(),
|
||||
resource.doGetStatus(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
false,
|
||||
makeExpectedReq(differentUserAuthResult)
|
||||
).getStatus()
|
||||
);
|
||||
|
@ -1013,6 +1054,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
Response.Status.OK.getStatusCode(),
|
||||
resource.doGetStatus(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
false,
|
||||
makeExpectedReq(differentUserAuthResult)
|
||||
).getStatus()
|
||||
);
|
||||
|
@ -1042,6 +1084,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
Response.Status.FORBIDDEN.getStatusCode(),
|
||||
resource.doGetStatus(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
false,
|
||||
makeExpectedReq(differentUserAuthResult)
|
||||
).getStatus()
|
||||
);
|
||||
|
@ -1071,6 +1114,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
Response.Status.OK.getStatusCode(),
|
||||
resource.doGetStatus(
|
||||
RUNNING_SELECT_MSQ_QUERY,
|
||||
false,
|
||||
makeExpectedReq(differentUserAuthResult)
|
||||
).getStatus()
|
||||
);
|
||||
|
@ -1107,7 +1151,7 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
|
||||
Assert.assertEquals(
|
||||
Response.Status.NOT_FOUND.getStatusCode(),
|
||||
resource.doGetStatus(taskIdNotFound, makeOkRequest()).getStatus()
|
||||
resource.doGetStatus(taskIdNotFound, false, makeOkRequest()).getStatus()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
Response.Status.NOT_FOUND.getStatusCode(),
|
||||
|
@ -1124,4 +1168,28 @@ public class SqlStatementResourceTest extends MSQTestBase
|
|||
{
|
||||
Assert.assertEquals(Response.Status.OK.getStatusCode(), resource.isEnabled(makeOkRequest()).getStatus());
|
||||
}
|
||||
|
||||
private void assertSqlStatementResult(SqlStatementResult expected, SqlStatementResult actual)
|
||||
{
|
||||
Assert.assertEquals(expected.getQueryId(), actual.getQueryId());
|
||||
Assert.assertEquals(expected.getCreatedAt(), actual.getCreatedAt());
|
||||
Assert.assertEquals(expected.getSqlRowSignature(), actual.getSqlRowSignature());
|
||||
Assert.assertEquals(expected.getDurationMs(), actual.getDurationMs());
|
||||
Assert.assertEquals(expected.getStages(), actual.getStages());
|
||||
Assert.assertEquals(expected.getState(), actual.getState());
|
||||
Assert.assertEquals(expected.getWarnings(), actual.getWarnings());
|
||||
Assert.assertEquals(expected.getResultSetInformation(), actual.getResultSetInformation());
|
||||
|
||||
if (actual.getCounters() == null || expected.getCounters() == null) {
|
||||
Assert.assertEquals(expected.getCounters(), actual.getCounters());
|
||||
} else {
|
||||
Assert.assertEquals(expected.getCounters().toString(), actual.getCounters().toString());
|
||||
}
|
||||
|
||||
if (actual.getErrorResponse() == null || expected.getErrorResponse() == null) {
|
||||
Assert.assertEquals(expected.getErrorResponse(), actual.getErrorResponse());
|
||||
} else {
|
||||
Assert.assertEquals(expected.getErrorResponse().getAsMap(), actual.getErrorResponse().getAsMap());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,15 +64,7 @@ public class CalciteArraysQueryMSQTest extends CalciteArraysQueryTest
|
|||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters();
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
|
|
|
@ -67,15 +67,7 @@ public class CalciteNestedDataQueryMSQTest extends CalciteNestedDataQueryTest
|
|||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters();
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
|
|
|
@ -136,15 +136,7 @@ public class CalciteSelectJoinQueryMSQTest
|
|||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters();
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
|
|
|
@ -73,15 +73,7 @@ public class CalciteSelectQueryMSQTest extends CalciteQueryTest
|
|||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters();
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
|
|
|
@ -79,15 +79,7 @@ public class CalciteUnionQueryMSQTest extends CalciteUnionQueryTest
|
|||
Injector injector
|
||||
)
|
||||
{
|
||||
final WorkerMemoryParameters workerMemoryParameters =
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
0,
|
||||
0
|
||||
);
|
||||
final WorkerMemoryParameters workerMemoryParameters = MSQTestBase.makeTestWorkerMemoryParameters();
|
||||
final MSQTestOverlordServiceClient indexingServiceClient = new MSQTestOverlordServiceClient(
|
||||
queryJsonMapper,
|
||||
injector,
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.druid.discovery.NodeRole;
|
|||
import org.apache.druid.frame.channel.FrameChannelSequence;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.guice.DruidInjectorBuilder;
|
||||
import org.apache.druid.guice.DruidSecondaryModule;
|
||||
import org.apache.druid.guice.ExpressionModule;
|
||||
|
@ -55,7 +56,6 @@ import org.apache.druid.guice.GuiceInjectors;
|
|||
import org.apache.druid.guice.IndexingServiceTuningConfigModule;
|
||||
import org.apache.druid.guice.JoinableFactoryModule;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.SegmentWranglerModule;
|
||||
import org.apache.druid.guice.StartupInjectorBuilder;
|
||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||
|
@ -334,16 +334,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
private SegmentCacheManager segmentCacheManager;
|
||||
|
||||
private TestGroupByBuffers groupByBuffers;
|
||||
protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(
|
||||
WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
2,
|
||||
1,
|
||||
0
|
||||
)
|
||||
);
|
||||
protected final WorkerMemoryParameters workerMemoryParameters = Mockito.spy(makeTestWorkerMemoryParameters());
|
||||
|
||||
protected static class MSQBaseComponentSupplier extends StandardComponentSupplier
|
||||
{
|
||||
|
@ -367,8 +358,8 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
{
|
||||
// We want this module to bring InputSourceModule along for the ride.
|
||||
binder.install(new InputSourceModule());
|
||||
binder.install(new NestedDataModule());
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
binder.install(new BuiltInTypesModule());
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
SqlBindings.addOperatorConversion(binder, ExternalOperatorConversion.class);
|
||||
SqlBindings.addOperatorConversion(binder, HttpOperatorConversion.class);
|
||||
SqlBindings.addOperatorConversion(binder, InlineOperatorConversion.class);
|
||||
|
@ -531,7 +522,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
objectMapper = setupObjectMapper(injector);
|
||||
objectMapper.registerModules(new StorageConnectorModule().getJacksonModules());
|
||||
objectMapper.registerModules(sqlModule.getJacksonModules());
|
||||
objectMapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
objectMapper.registerModules(BuiltInTypesModule.getJacksonModulesList());
|
||||
|
||||
doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString());
|
||||
|
||||
|
@ -755,6 +746,19 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
return mapper;
|
||||
}
|
||||
|
||||
public static WorkerMemoryParameters makeTestWorkerMemoryParameters()
|
||||
{
|
||||
return WorkerMemoryParameters.createInstance(
|
||||
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
|
||||
2,
|
||||
10,
|
||||
1,
|
||||
2,
|
||||
1,
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
private String runMultiStageQuery(String query, Map<String, Object> context)
|
||||
{
|
||||
final DirectStatement stmt = sqlStatementFactory.directStatement(
|
||||
|
@ -902,7 +906,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
return asBuilder();
|
||||
}
|
||||
|
||||
public Builder setExpectedSegment(Set<SegmentId> expectedSegments)
|
||||
public Builder setExpectedSegments(Set<SegmentId> expectedSegments)
|
||||
{
|
||||
Preconditions.checkArgument(expectedSegments != null, "Segments cannot be null");
|
||||
this.expectedSegments = expectedSegments;
|
||||
|
|
|
@ -87,9 +87,9 @@ public class MSQTestControllerClient implements ControllerClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<String> getTaskList()
|
||||
public List<String> getWorkerIds()
|
||||
{
|
||||
return controller.getTaskIds();
|
||||
return controller.getWorkerIds();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -156,32 +156,33 @@ public class MSQTestControllerContext implements ControllerContext
|
|||
Worker worker = new WorkerImpl(
|
||||
task,
|
||||
new MSQTestWorkerContext(
|
||||
task.getId(),
|
||||
inMemoryWorkers,
|
||||
controller,
|
||||
mapper,
|
||||
injector,
|
||||
workerMemoryParameters
|
||||
),
|
||||
workerMemoryParameters,
|
||||
workerStorageParameters
|
||||
)
|
||||
);
|
||||
inMemoryWorkers.put(task.getId(), worker);
|
||||
statusMap.put(task.getId(), TaskStatus.running(task.getId()));
|
||||
|
||||
ListenableFuture<TaskStatus> future = executor.submit(() -> {
|
||||
ListenableFuture<?> future = executor.submit(() -> {
|
||||
try {
|
||||
return worker.run();
|
||||
worker.run();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
|
||||
Futures.addCallback(future, new FutureCallback<TaskStatus>()
|
||||
Futures.addCallback(future, new FutureCallback<Object>()
|
||||
{
|
||||
@Override
|
||||
public void onSuccess(@Nullable TaskStatus result)
|
||||
public void onSuccess(@Nullable Object result)
|
||||
{
|
||||
statusMap.put(task.getId(), result);
|
||||
statusMap.put(task.getId(), TaskStatus.success(task.getId()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -261,7 +262,7 @@ public class MSQTestControllerContext implements ControllerContext
|
|||
{
|
||||
final Worker worker = inMemoryWorkers.remove(workerId);
|
||||
if (worker != null) {
|
||||
worker.stopGracefully();
|
||||
worker.stop();
|
||||
}
|
||||
return Futures.immediateFuture(null);
|
||||
}
|
||||
|
|
|
@ -80,11 +80,7 @@ public class MSQTestWorkerClient implements WorkerClient
|
|||
)
|
||||
{
|
||||
try {
|
||||
inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries(
|
||||
partitionBoundaries,
|
||||
stageId.getQueryId(),
|
||||
stageId.getStageNumber()
|
||||
);
|
||||
inMemoryWorkers.get(workerTaskId).postResultPartitionBoundaries(stageId, partitionBoundaries);
|
||||
return Futures.immediateFuture(null);
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
@ -122,8 +118,7 @@ public class MSQTestWorkerClient implements WorkerClient
|
|||
)
|
||||
{
|
||||
try (InputStream inputStream =
|
||||
inMemoryWorkers.get(workerTaskId)
|
||||
.readChannel(stageId.getQueryId(), stageId.getStageNumber(), partitionNumber, offset)) {
|
||||
inMemoryWorkers.get(workerTaskId).readStageOutput(stageId, partitionNumber, offset).get()) {
|
||||
byte[] buffer = new byte[8 * 1024];
|
||||
boolean didRead = false;
|
||||
int bytesRead;
|
||||
|
@ -138,12 +133,11 @@ public class MSQTestWorkerClient implements WorkerClient
|
|||
catch (Exception e) {
|
||||
throw new ISE(e, "Error reading frame file channel");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
inMemoryWorkers.forEach((k, v) -> v.stopGracefully());
|
||||
inMemoryWorkers.forEach((k, v) -> v.stop());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,59 +22,69 @@ package org.apache.druid.msq.test;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.frame.processor.Bouncer;
|
||||
import org.apache.druid.indexer.report.TaskReportFileWriter;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.task.NoopTestTaskReportFileWriter;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.msq.exec.Controller;
|
||||
import org.apache.druid.msq.exec.ControllerClient;
|
||||
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
|
||||
import org.apache.druid.msq.exec.OutputChannelMode;
|
||||
import org.apache.druid.msq.exec.Worker;
|
||||
import org.apache.druid.msq.exec.WorkerClient;
|
||||
import org.apache.druid.msq.exec.WorkerContext;
|
||||
import org.apache.druid.msq.exec.WorkerMemoryParameters;
|
||||
import org.apache.druid.msq.indexing.IndexerFrameContext;
|
||||
import org.apache.druid.msq.indexing.IndexerWorkerContext;
|
||||
import org.apache.druid.msq.exec.WorkerStorageParameters;
|
||||
import org.apache.druid.msq.kernel.FrameContext;
|
||||
import org.apache.druid.msq.kernel.QueryDefinition;
|
||||
import org.apache.druid.msq.querykit.DataSegmentProvider;
|
||||
import org.apache.druid.query.groupby.GroupingEngine;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.SegmentWrangler;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.NoopRowIngestionMeters;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.NoopChatHandlerProvider;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.server.coordination.DataSegmentAnnouncer;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Map;
|
||||
|
||||
public class MSQTestWorkerContext implements WorkerContext
|
||||
{
|
||||
private final String workerId;
|
||||
private final Controller controller;
|
||||
private final ObjectMapper mapper;
|
||||
private final Injector injector;
|
||||
private final Map<String, Worker> inMemoryWorkers;
|
||||
private final File file = FileUtils.createTempDir();
|
||||
private final Bouncer bouncer = new Bouncer(1);
|
||||
private final WorkerMemoryParameters workerMemoryParameters;
|
||||
private final WorkerStorageParameters workerStorageParameters;
|
||||
|
||||
public MSQTestWorkerContext(
|
||||
String workerId,
|
||||
Map<String, Worker> inMemoryWorkers,
|
||||
Controller controller,
|
||||
ObjectMapper mapper,
|
||||
Injector injector,
|
||||
WorkerMemoryParameters workerMemoryParameters
|
||||
WorkerMemoryParameters workerMemoryParameters,
|
||||
WorkerStorageParameters workerStorageParameters
|
||||
)
|
||||
{
|
||||
this.workerId = workerId;
|
||||
this.inMemoryWorkers = inMemoryWorkers;
|
||||
this.controller = controller;
|
||||
this.mapper = mapper;
|
||||
this.injector = injector;
|
||||
this.workerMemoryParameters = workerMemoryParameters;
|
||||
this.workerStorageParameters = workerStorageParameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String queryId()
|
||||
{
|
||||
return controller.queryId();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -96,7 +106,13 @@ public class MSQTestWorkerContext implements WorkerContext
|
|||
}
|
||||
|
||||
@Override
|
||||
public ControllerClient makeControllerClient(String controllerId)
|
||||
public String workerId()
|
||||
{
|
||||
return workerId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ControllerClient makeControllerClient()
|
||||
{
|
||||
return new MSQTestControllerClient(controller);
|
||||
}
|
||||
|
@ -114,42 +130,9 @@ public class MSQTestWorkerContext implements WorkerContext
|
|||
}
|
||||
|
||||
@Override
|
||||
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber)
|
||||
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode)
|
||||
{
|
||||
IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
mapper,
|
||||
indexIO,
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
true
|
||||
);
|
||||
final TaskReportFileWriter reportFileWriter = new NoopTestTaskReportFileWriter();
|
||||
|
||||
return new IndexerFrameContext(
|
||||
new IndexerWorkerContext(
|
||||
new TaskToolbox.Builder()
|
||||
.segmentPusher(injector.getInstance(DataSegmentPusher.class))
|
||||
.segmentAnnouncer(injector.getInstance(DataSegmentAnnouncer.class))
|
||||
.jsonMapper(mapper)
|
||||
.taskWorkDir(tempDir())
|
||||
.indexIO(indexIO)
|
||||
.indexMergerV9(indexMerger)
|
||||
.taskReportFileWriter(reportFileWriter)
|
||||
.authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER)
|
||||
.chatHandlerProvider(new NoopChatHandlerProvider())
|
||||
.rowIngestionMetersFactory(NoopRowIngestionMeters::new)
|
||||
.build(),
|
||||
injector,
|
||||
indexIO,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
indexIO,
|
||||
injector.getInstance(DataSegmentProvider.class),
|
||||
injector.getInstance(DataServerQueryHandlerFactory.class),
|
||||
workerMemoryParameters
|
||||
);
|
||||
return new FrameContextImpl(new File(tempDir(), queryDef.getStageDefinition(stageNumber).getId().toString()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -165,9 +148,9 @@ public class MSQTestWorkerContext implements WorkerContext
|
|||
}
|
||||
|
||||
@Override
|
||||
public Bouncer processorBouncer()
|
||||
public int maxConcurrentStages()
|
||||
{
|
||||
return injector.getInstance(Bouncer.class);
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -175,4 +158,109 @@ public class MSQTestWorkerContext implements WorkerContext
|
|||
{
|
||||
return injector.getInstance(DataServerQueryHandlerFactory.class);
|
||||
}
|
||||
|
||||
class FrameContextImpl implements FrameContext
|
||||
{
|
||||
private final File tempDir;
|
||||
|
||||
public FrameContextImpl(File tempDir)
|
||||
{
|
||||
this.tempDir = tempDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SegmentWrangler segmentWrangler()
|
||||
{
|
||||
return injector.getInstance(SegmentWrangler.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public GroupingEngine groupingEngine()
|
||||
{
|
||||
return injector.getInstance(GroupingEngine.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowIngestionMeters rowIngestionMeters()
|
||||
{
|
||||
return new NoopRowIngestionMeters();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegmentProvider dataSegmentProvider()
|
||||
{
|
||||
return injector.getInstance(DataSegmentProvider.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataServerQueryHandlerFactory dataServerQueryHandlerFactory()
|
||||
{
|
||||
return injector.getInstance(DataServerQueryHandlerFactory.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public File tempDir()
|
||||
{
|
||||
return new File(tempDir, "tmp");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectMapper jsonMapper()
|
||||
{
|
||||
return mapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexIO indexIO()
|
||||
{
|
||||
return new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public File persistDir()
|
||||
{
|
||||
return new File(tempDir, "persist");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegmentPusher segmentPusher()
|
||||
{
|
||||
return injector.getInstance(DataSegmentPusher.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexMergerV9 indexMerger()
|
||||
{
|
||||
return new IndexMergerV9(
|
||||
mapper,
|
||||
indexIO(),
|
||||
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Bouncer processorBouncer()
|
||||
{
|
||||
return bouncer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WorkerMemoryParameters memoryParameters()
|
||||
{
|
||||
return workerMemoryParameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WorkerStorageParameters storageParameters()
|
||||
{
|
||||
return workerStorageParameters;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import com.google.common.base.Strings;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.guice.annotations.PublicApi;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
|
@ -110,10 +111,9 @@ public abstract class DimensionSchema
|
|||
return name == null ? ofDefault() : valueOf(StringUtils.toUpperCase(name));
|
||||
}
|
||||
|
||||
// this can be system configuration
|
||||
public static MultiValueHandling ofDefault()
|
||||
{
|
||||
return SORTED_ARRAY;
|
||||
return BuiltInTypesModule.getStringMultiValueHandlingMode();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -104,6 +104,14 @@ public class ReadableFileFrameChannel implements ReadableFrameChannel
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether this channel represents the entire underlying {@link FrameFile}.
|
||||
*/
|
||||
public boolean isEntireFile()
|
||||
{
|
||||
return currentFrame == 0 && endFrame == frameFile.numFrames();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a new reference to the {@link FrameFile} that this channel is reading from. Callers should close this
|
||||
* reference when done reading.
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Provides;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.segment.DefaultColumnFormatConfig;
|
||||
import org.apache.druid.segment.DimensionHandler;
|
||||
|
@ -38,11 +39,19 @@ import org.apache.druid.segment.nested.StructuredDataJsonSerializer;
|
|||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
public class NestedDataModule implements DruidModule
|
||||
public class BuiltInTypesModule implements DruidModule
|
||||
{
|
||||
/**
|
||||
* Initialized with a default value so tests can just get it via {@link #getStringMultiValueHandlingMode} without any
|
||||
* explicit initialization. In production, this default may be overridden if a value is configured via
|
||||
* {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)}.
|
||||
*/
|
||||
private static DimensionSchema.MultiValueHandling STRING_MV_MODE = DimensionSchema.MultiValueHandling.SORTED_ARRAY;
|
||||
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
|
@ -53,14 +62,15 @@ public class NestedDataModule implements DruidModule
|
|||
public void configure(Binder binder)
|
||||
{
|
||||
registerSerde();
|
||||
// binding our side effect class to the lifecycle causes registerHandler to be called on service start, allowing
|
||||
// use of the config to get the system default format version
|
||||
LifecycleModule.register(binder, SideEffectHandlerRegisterer.class);
|
||||
// binding our side effect classes to the lifecycle causes the initDimensionHandlerAndMvHandlingMode to be
|
||||
// called on service start, allowing use of the config to get the system default format version and string multi
|
||||
// value handling mode.
|
||||
LifecycleModule.register(binder, SideEffectRegisterer.class);
|
||||
}
|
||||
|
||||
@Provides
|
||||
@LazySingleton
|
||||
public SideEffectHandlerRegisterer registerHandler(DefaultColumnFormatConfig formatsConfig)
|
||||
public SideEffectRegisterer initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig formatsConfig)
|
||||
{
|
||||
if (formatsConfig.getNestedColumnFormatVersion() != null && formatsConfig.getNestedColumnFormatVersion() == 4) {
|
||||
DimensionHandlerUtils.registerDimensionHandlerProvider(
|
||||
|
@ -73,7 +83,25 @@ public class NestedDataModule implements DruidModule
|
|||
new NestedCommonFormatHandlerProvider()
|
||||
);
|
||||
}
|
||||
return new SideEffectHandlerRegisterer();
|
||||
|
||||
setStringMultiValueHandlingModeIfConfigured(formatsConfig.getStringMultiValueHandlingMode());
|
||||
return new SideEffectRegisterer();
|
||||
}
|
||||
|
||||
private static void setStringMultiValueHandlingModeIfConfigured(@Nullable String stringMultiValueHandlingMode)
|
||||
{
|
||||
if (stringMultiValueHandlingMode != null) {
|
||||
STRING_MV_MODE = DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the configured string multi value handling mode from the system config if set; otherwise, returns
|
||||
* the default.
|
||||
*/
|
||||
public static DimensionSchema.MultiValueHandling getStringMultiValueHandlingMode()
|
||||
{
|
||||
return STRING_MV_MODE;
|
||||
}
|
||||
|
||||
public static List<SimpleModule> getJacksonModulesList()
|
||||
|
@ -126,13 +154,15 @@ public class NestedDataModule implements DruidModule
|
|||
return new NestedDataColumnHandlerV4(dimensionName);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* this is used as a vehicle to register the correct version of the system default nested column handler by side
|
||||
* effect with the help of binding to {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that
|
||||
* {@link #registerHandler(DefaultColumnFormatConfig)} can be called with the injected
|
||||
* this is used as a vehicle to register the correct version of the system default nested column handler and multi
|
||||
* value handling mode by side effect with the help of binding to
|
||||
* {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} so that
|
||||
* {@link #initDimensionHandlerAndMvHandlingMode(DefaultColumnFormatConfig)} can be called with the injected
|
||||
* {@link DefaultColumnFormatConfig}.
|
||||
*/
|
||||
public static class SideEffectHandlerRegisterer
|
||||
public static class SideEffectRegisterer
|
||||
{
|
||||
// nothing to see here
|
||||
}
|
|
@ -21,9 +21,11 @@ package org.apache.druid.segment;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.error.DruidException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DefaultColumnFormatConfig
|
||||
|
@ -39,16 +41,44 @@ public class DefaultColumnFormatConfig
|
|||
}
|
||||
}
|
||||
|
||||
private static void validateMultiValueHandlingMode(@Nullable String stringMultiValueHandlingMode)
|
||||
{
|
||||
if (stringMultiValueHandlingMode != null) {
|
||||
try {
|
||||
DimensionSchema.MultiValueHandling.fromString(stringMultiValueHandlingMode);
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
throw DruidException.forPersona(DruidException.Persona.OPERATOR)
|
||||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build(
|
||||
"Invalid value[%s] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'."
|
||||
+ " Supported values are [%s].",
|
||||
stringMultiValueHandlingMode,
|
||||
Arrays.toString(DimensionSchema.MultiValueHandling.values())
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty("nestedColumnFormatVersion")
|
||||
private final Integer nestedColumnFormatVersion;
|
||||
|
||||
@Nullable
|
||||
@JsonProperty("stringMultiValueHandlingMode")
|
||||
private final String stringMultiValueHandlingMode;
|
||||
|
||||
@JsonCreator
|
||||
public DefaultColumnFormatConfig(
|
||||
@JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion
|
||||
@JsonProperty("nestedColumnFormatVersion") @Nullable Integer nestedColumnFormatVersion,
|
||||
@JsonProperty("stringMultiValueHandlingMode") @Nullable String stringMultiValueHandlingMode
|
||||
)
|
||||
{
|
||||
validateNestedFormatVersion(nestedColumnFormatVersion);
|
||||
validateMultiValueHandlingMode(stringMultiValueHandlingMode);
|
||||
|
||||
this.nestedColumnFormatVersion = nestedColumnFormatVersion;
|
||||
validateNestedFormatVersion(this.nestedColumnFormatVersion);
|
||||
this.stringMultiValueHandlingMode = stringMultiValueHandlingMode;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -58,6 +88,13 @@ public class DefaultColumnFormatConfig
|
|||
return nestedColumnFormatVersion;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty("stringMultiValueHandlingMode")
|
||||
public String getStringMultiValueHandlingMode()
|
||||
{
|
||||
return stringMultiValueHandlingMode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
@ -68,13 +105,14 @@ public class DefaultColumnFormatConfig
|
|||
return false;
|
||||
}
|
||||
DefaultColumnFormatConfig that = (DefaultColumnFormatConfig) o;
|
||||
return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion);
|
||||
return Objects.equals(nestedColumnFormatVersion, that.nestedColumnFormatVersion)
|
||||
&& Objects.equals(stringMultiValueHandlingMode, that.stringMultiValueHandlingMode);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(nestedColumnFormatVersion);
|
||||
return Objects.hash(nestedColumnFormatVersion, stringMultiValueHandlingMode);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -82,6 +120,7 @@ public class DefaultColumnFormatConfig
|
|||
{
|
||||
return "DefaultColumnFormatConfig{" +
|
||||
"nestedColumnFormatVersion=" + nestedColumnFormatVersion +
|
||||
", stringMultiValueHandlingMode=" + stringMultiValueHandlingMode +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -66,7 +66,7 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer<int[]
|
|||
private volatile boolean hasMultipleValues = false;
|
||||
|
||||
public StringDimensionIndexer(
|
||||
MultiValueHandling multiValueHandling,
|
||||
@Nullable MultiValueHandling multiValueHandling,
|
||||
boolean hasBitmapIndexes,
|
||||
boolean hasSpatialIndexes,
|
||||
boolean useMaxMemoryEstimates
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|||
import com.fasterxml.jackson.dataformat.smile.SmileGenerator;
|
||||
import it.unimi.dsi.fastutil.Hash;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
|
@ -62,7 +62,7 @@ public class NestedDataComplexTypeSerde extends ComplexMetricSerde
|
|||
smileFactory.delegateToTextual(true);
|
||||
final ObjectMapper mapper = new DefaultObjectMapper(smileFactory, null);
|
||||
mapper.getFactory().setCodec(mapper);
|
||||
mapper.registerModules(NestedDataModule.getJacksonModulesList());
|
||||
mapper.registerModules(BuiltInTypesModule.getJacksonModulesList());
|
||||
OBJECT_MAPPER = mapper;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,104 @@
|
|||
/*
|
||||
* 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.frame.processor;
|
||||
|
||||
import org.apache.druid.frame.FrameType;
|
||||
import org.apache.druid.frame.channel.ReadableFileFrameChannel;
|
||||
import org.apache.druid.frame.file.FrameFile;
|
||||
import org.apache.druid.frame.read.FrameReader;
|
||||
import org.apache.druid.frame.testutil.FrameSequenceBuilder;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public class ReadableFileFrameChannelTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final int ROWS_PER_FRAME = 20;
|
||||
|
||||
private List<List<Object>> allRows;
|
||||
private FrameReader frameReader;
|
||||
private FrameFile frameFile;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException
|
||||
{
|
||||
final StorageAdapter adapter = new QueryableIndexStorageAdapter(TestIndex.getNoRollupMMappedTestIndex());
|
||||
final File file = FrameTestUtil.writeFrameFile(
|
||||
FrameSequenceBuilder.fromAdapter(adapter)
|
||||
.frameType(FrameType.ROW_BASED)
|
||||
.maxRowsPerFrame(ROWS_PER_FRAME)
|
||||
.frames(),
|
||||
temporaryFolder.newFile()
|
||||
);
|
||||
allRows = FrameTestUtil.readRowsFromAdapter(adapter, adapter.getRowSignature(), false).toList();
|
||||
frameReader = FrameReader.create(adapter.getRowSignature());
|
||||
frameFile = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
frameFile.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_fullFile()
|
||||
{
|
||||
final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile);
|
||||
Assert.assertTrue(channel.isEntireFile());
|
||||
|
||||
FrameTestUtil.assertRowsEqual(
|
||||
Sequences.simple(allRows),
|
||||
FrameTestUtil.readRowsFromFrameChannel(channel, frameReader)
|
||||
);
|
||||
|
||||
Assert.assertFalse(channel.isEntireFile());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_partialFile()
|
||||
{
|
||||
final ReadableFileFrameChannel channel = new ReadableFileFrameChannel(frameFile, 1, 2);
|
||||
Assert.assertFalse(channel.isEntireFile());
|
||||
|
||||
FrameTestUtil.assertRowsEqual(
|
||||
Sequences.simple(allRows).skip(ROWS_PER_FRAME).limit(ROWS_PER_FRAME),
|
||||
FrameTestUtil.readRowsFromFrameChannel(channel, frameReader)
|
||||
);
|
||||
|
||||
Assert.assertFalse(channel.isEntireFile());
|
||||
}
|
||||
}
|
|
@ -37,7 +37,7 @@ import org.apache.druid.frame.read.FrameReader;
|
|||
import org.apache.druid.frame.segment.FrameSegment;
|
||||
import org.apache.druid.frame.segment.FrameStorageAdapter;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.BuiltInTypesModule;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
|
@ -92,7 +92,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest
|
|||
|
||||
static {
|
||||
ComplexMetrics.registerSerde(HyperUniquesSerde.TYPE_NAME, new HyperUniquesSerde());
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
BuiltInTypesModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
private static final int DEFAULT_ALLOCATOR_CAPACITY = 1_000_000;
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.guice;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.segment.DefaultColumnFormatConfig;
|
||||
import org.apache.druid.segment.DimensionHandlerProvider;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
@ -35,7 +36,7 @@ import org.junit.Test;
|
|||
import javax.annotation.Nullable;
|
||||
import java.util.Properties;
|
||||
|
||||
public class NestedDataModuleTest
|
||||
public class BuiltInTypesModuleTest
|
||||
{
|
||||
@Nullable
|
||||
private static DimensionHandlerProvider DEFAULT_HANDLER_PROVIDER;
|
||||
|
@ -69,12 +70,17 @@ public class NestedDataModuleTest
|
|||
Injector gadget = makeInjector(props);
|
||||
|
||||
// side effects
|
||||
gadget.getInstance(NestedDataModule.SideEffectHandlerRegisterer.class);
|
||||
gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class);
|
||||
|
||||
DimensionHandlerProvider provider = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get(
|
||||
NestedDataComplexTypeSerde.TYPE_NAME
|
||||
);
|
||||
Assert.assertTrue(provider.get("test") instanceof NestedCommonFormatColumnHandler);
|
||||
|
||||
Assert.assertEquals(
|
||||
DimensionSchema.MultiValueHandling.SORTED_ARRAY,
|
||||
BuiltInTypesModule.getStringMultiValueHandlingMode()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -82,16 +88,54 @@ public class NestedDataModuleTest
|
|||
{
|
||||
DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.remove(NestedDataComplexTypeSerde.TYPE_NAME);
|
||||
Properties props = new Properties();
|
||||
props.put("druid.indexing.formats.nestedColumnFormatVersion", "4");
|
||||
props.setProperty("druid.indexing.formats.nestedColumnFormatVersion", "4");
|
||||
props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "sorted_array");
|
||||
Injector gadget = makeInjector(props);
|
||||
|
||||
// side effects
|
||||
gadget.getInstance(NestedDataModule.SideEffectHandlerRegisterer.class);
|
||||
gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class);
|
||||
|
||||
DimensionHandlerProvider provider = DimensionHandlerUtils.DIMENSION_HANDLER_PROVIDERS.get(
|
||||
NestedDataComplexTypeSerde.TYPE_NAME
|
||||
);
|
||||
Assert.assertTrue(provider.get("test") instanceof NestedDataColumnHandlerV4);
|
||||
|
||||
Assert.assertEquals(
|
||||
DimensionSchema.MultiValueHandling.SORTED_ARRAY,
|
||||
BuiltInTypesModule.getStringMultiValueHandlingMode()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverrideMultiValueHandlingModeCaseInsensitive()
|
||||
{
|
||||
final Properties props = new Properties();
|
||||
props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "ARRAY");
|
||||
final Injector gadget = makeInjector(props);
|
||||
|
||||
gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class);
|
||||
|
||||
Assert.assertEquals(
|
||||
DimensionSchema.MultiValueHandling.ARRAY,
|
||||
BuiltInTypesModule.getStringMultiValueHandlingMode()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidMultiValueHandlingMode()
|
||||
{
|
||||
final Properties props = new Properties();
|
||||
props.setProperty("druid.indexing.formats.stringMultiValueHandlingMode", "boo");
|
||||
final Injector gadget = makeInjector(props);
|
||||
|
||||
final Exception exception = Assert.assertThrows(
|
||||
Exception.class,
|
||||
() -> gadget.getInstance(BuiltInTypesModule.SideEffectRegisterer.class)
|
||||
);
|
||||
Assert.assertTrue(exception.getMessage().contains(
|
||||
"Invalid value[boo] specified for 'druid.indexing.formats.stringMultiValueHandlingMode'."
|
||||
+ " Supported values are [[SORTED_ARRAY, SORTED_SET, ARRAY]]."
|
||||
));
|
||||
}
|
||||
|
||||
private Injector makeInjector(Properties props)
|
||||
|
@ -104,7 +148,7 @@ public class NestedDataModuleTest
|
|||
binder -> {
|
||||
JsonConfigProvider.bind(binder, "druid.indexing.formats", DefaultColumnFormatConfig.class);
|
||||
},
|
||||
new NestedDataModule()
|
||||
new BuiltInTypesModule()
|
||||
)
|
||||
);
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue