mirror of https://github.com/apache/druid.git
Merge remote-tracking branch 'apache/master' into quidem-msq
This commit is contained in:
commit
8a92ec6b2e
|
@ -304,9 +304,7 @@ public class BaseColumnarLongsBenchmark
|
|||
}
|
||||
|
||||
serializer.open();
|
||||
for (long val : vals) {
|
||||
serializer.add(val);
|
||||
}
|
||||
serializer.addAll(vals, 0, vals.length);
|
||||
serializer.writeTo(output, null);
|
||||
return (int) serializer.getSerializedSize();
|
||||
}
|
||||
|
|
|
@ -160,7 +160,6 @@ Druid interacts with ZooKeeper through a set of standard path configurations. We
|
|||
|`druid.zk.paths.announcementsPath`|Druid service announcement path.|`${druid.zk.paths.base}/announcements`|
|
||||
|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid services announce their segments.|`${druid.zk.paths.base}/segments`|
|
||||
|`druid.zk.paths.coordinatorPath`|Used by the Coordinator for leader election.|`${druid.zk.paths.base}/coordinator`|
|
||||
|`druid.zk.paths.servedSegmentsPath`|Deprecated. Legacy path for where Druid services announce their segments.|`${druid.zk.paths.base}/servedSegments`|
|
||||
|
||||
The indexing service also uses its own set of paths. These configs can be included in the common configuration.
|
||||
|
||||
|
|
|
@ -53,7 +53,7 @@ ${druid.zk.paths.coordinatorPath}/_COORDINATOR
|
|||
|
||||
## Segment "publishing" protocol from Historical and Realtime
|
||||
|
||||
The `announcementsPath` and `servedSegmentsPath` are used for this.
|
||||
The `announcementsPath` and `liveSegmentsPath` are used for this.
|
||||
|
||||
All [Historical](../design/historical.md) processes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
|
||||
|
||||
|
@ -64,13 +64,13 @@ ${druid.zk.paths.announcementsPath}/${druid.host}
|
|||
Which signifies that they exist. They will also subsequently create a permanent znode at
|
||||
|
||||
```
|
||||
${druid.zk.paths.servedSegmentsPath}/${druid.host}
|
||||
${druid.zk.paths.liveSegmentsPath}/${druid.host}
|
||||
```
|
||||
|
||||
And as they load up segments, they will attach ephemeral znodes that look like
|
||||
|
||||
```
|
||||
${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_
|
||||
${druid.zk.paths.liveSegmentsPath}/${druid.host}/_segment_identifier_
|
||||
```
|
||||
|
||||
Processes like the [Coordinator](../design/coordinator.md) and [Broker](../design/broker.md) can then watch these paths to see which processes are currently serving which segments.
|
||||
|
|
|
@ -655,20 +655,23 @@ Returns the following:
|
|||
|
||||
## CHAR_LENGTH
|
||||
|
||||
`CHAR_LENGTH(expr)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Alias for [`LENGTH`](#length).
|
||||
|
||||
* **Syntax:** `CHAR_LENGTH(expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## CHARACTER_LENGTH
|
||||
|
||||
`CHARACTER_LENGTH(expr)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Alias for [`LENGTH`](#length).
|
||||
|
||||
* **Syntax:** `CHARACTER_LENGTH(expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
|
||||
## COALESCE
|
||||
|
||||
`COALESCE(expr, expr, ...)`
|
||||
|
@ -679,19 +682,64 @@ Returns the first non-null value.
|
|||
|
||||
## CONCAT
|
||||
|
||||
`CONCAT(expr, expr...)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Concatenates a list of expressions.
|
||||
|
||||
* **Syntax:** `CONCAT(expr[, expr,...])`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example concatenates the `OriginCityName` column from `flight-carriers`, the string ` to `, and the `DestCityName` column from `flight-carriers`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"OriginCityName" AS "origin_city",
|
||||
"DestCityName" AS "destination_city",
|
||||
CONCAT("OriginCityName", ' to ', "DestCityName") AS "concatenate_flight_details"
|
||||
FROM "flight-carriers"
|
||||
LIMIT 1
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `origin_city` | `destination_city` | `concatenate_flight_details` |
|
||||
| -- | -- | -- |
|
||||
| `San Juan, PR` | `Washington, DC` | `San Juan, PR to Washington, DC` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## CONTAINS_STRING
|
||||
|
||||
`CONTAINS_STRING(<CHARACTER>, <CHARACTER>)`
|
||||
Returns `true` if `str` is a substring of `expr`, case-sensitive. Otherwise returns `false`.
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
* **Syntax:** `CONTAINS_STRING(expr, str)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
Finds whether a string is in a given expression, case-sensitive.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the substring `San`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"OriginCityName" AS "origin_city",
|
||||
CONTAINS_STRING("OriginCityName", 'San') AS "contains_string"
|
||||
FROM "flight-carriers"
|
||||
LIMIT 2
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `origin_city` | `contains_string` |
|
||||
| -- | -- |
|
||||
| `San Juan, PR` | `true` |
|
||||
| `Boston, MA` | `false` |
|
||||
|
||||
</details>
|
||||
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## COS
|
||||
|
||||
|
@ -791,13 +839,31 @@ Decodes a Base64-encoded string into a complex data type, where `dataType` is th
|
|||
|
||||
## DECODE_BASE64_UTF8
|
||||
|
||||
`DECODE_BASE64_UTF8(expr)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
|
||||
Decodes a Base64-encoded string into a UTF-8 encoded string.
|
||||
|
||||
* **Syntax:** `DECODE_BASE64_UTF8(expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example converts the base64 encoded string `SGVsbG8gV29ybGQhCg==` into an UTF-8 encoded string.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
'SGVsbG8gV29ybGQhCg==' AS "base64_encoding",
|
||||
DECODE_BASE64_UTF8('SGVsbG8gV29ybGQhCg==') AS "convert_to_UTF8_encoding"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `base64_encoding` | `convert_to_UTF8_encoding` |
|
||||
| -- | -- |
|
||||
| `SGVsbG8gV29ybGQhCg==` | `Hello World!` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## DEGREES
|
||||
|
||||
Converts an angle from radians to degrees.
|
||||
|
@ -1191,11 +1257,33 @@ Returns the following:
|
|||
|
||||
## ICONTAINS_STRING
|
||||
|
||||
`ICONTAINS_STRING(<expr>, str)`
|
||||
Returns `true` if `str` is a substring of `expr`, case-insensitive. Otherwise returns `false`.
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
* **Syntax:** `ICONTAINS_STRING(expr, str)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
Finds whether a string is in a given expression, case-insensitive.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the case-insensitive substring `san`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"OriginCityName" AS "origin_city",
|
||||
ICONTAINS_STRING("OriginCityName", 'san') AS "contains_case_insensitive_string"
|
||||
FROM "flight-carriers"
|
||||
LIMIT 2
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `origin_city` | `contains_case_insensitive_string` |
|
||||
| -- | -- |
|
||||
| `San Juan, PR` | `true` |
|
||||
| `Boston, MA` | `false` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## IPV4_MATCH
|
||||
|
||||
|
@ -1327,19 +1415,59 @@ Returns the minimum value from the provided arguments.
|
|||
|
||||
## LEFT
|
||||
|
||||
`LEFT(expr, [length])`
|
||||
Returns the `N` leftmost characters of an expression, where `N` is an integer value.
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
* **Syntax:** `LEFT(expr, N)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
Returns the leftmost number of characters from an expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns the `3` leftmost characters of the expression `ABCDEFG`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
'ABCDEFG' AS "expression",
|
||||
LEFT('ABCDEFG', 3) AS "leftmost_characters"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `expression` | `leftmost_characters` |
|
||||
| -- | -- |
|
||||
| `ABCDEFG` | `ABC` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## LENGTH
|
||||
|
||||
`LENGTH(expr)`
|
||||
Returns the length of the expression in UTF-16 code units.
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
* **Syntax:** `LENGTH(expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
Returns the length of the expression in UTF-16 encoding.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns the character length of the `OriginCityName` column from the `flight-carriers` datasource.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"OriginCityName" AS "origin_city_name",
|
||||
LENGTH("OriginCityName") AS "city_name_length"
|
||||
FROM "flight-carriers"
|
||||
LIMIT 1
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `origin_city_name` | `city_name_length` |
|
||||
| -- | -- |
|
||||
| `San Juan, PR` | `12` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## LN
|
||||
|
||||
|
@ -1755,11 +1883,30 @@ Reverses the given expression.
|
|||
|
||||
## RIGHT
|
||||
|
||||
`RIGHT(expr, [length])`
|
||||
Returns the `N` rightmost characters of an expression, where `N` is an integer value.
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
* **Syntax:** `RIGHT(expr, N)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
Returns the rightmost number of characters from an expression.
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example returns the `3` rightmost characters of the expression `ABCDEFG`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
'ABCDEFG' AS "expression",
|
||||
RIGHT('ABCDEFG', 3) AS "rightmost_characters"
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `expression` | `rightmost_characters` |
|
||||
| -- | -- |
|
||||
| `ABCDEFG` | `EFG` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## ROUND
|
||||
|
||||
|
@ -1943,12 +2090,13 @@ Splits `str1` into an multi-value string on the delimiter specified by `str2`, w
|
|||
|
||||
## STRLEN
|
||||
|
||||
`STRLEN(expr)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Alias for [`LENGTH`](#length).
|
||||
|
||||
* **Syntax:** `STRLEN(expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## STRPOS
|
||||
|
||||
`STRPOS(<CHARACTER>, <CHARACTER>)`
|
||||
|
@ -2022,12 +2170,33 @@ Returns the quantile for the specified fraction from a T-Digest sketch construct
|
|||
|
||||
## TEXTCAT
|
||||
|
||||
`TEXTCAT(<CHARACTER>, <CHARACTER>)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Concatenates two string expressions.
|
||||
|
||||
* **Syntax:** `TEXTCAT(expr, expr)`
|
||||
* **Function type:** Scalar, string
|
||||
|
||||
<details><summary>Example</summary>
|
||||
|
||||
The following example concatenates the `OriginState` column from the `flight-carriers` datasource to `, USA`.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"OriginState" AS "origin_state",
|
||||
TEXTCAT("OriginState", ', USA') AS "concatenate_state_with_USA"
|
||||
FROM "flight-carriers"
|
||||
LIMIT 1
|
||||
```
|
||||
|
||||
Returns the following:
|
||||
|
||||
| `origin_state` | `concatenate_state_with_USA` |
|
||||
| -- | -- |
|
||||
| `PR` | `PR, USA` |
|
||||
|
||||
</details>
|
||||
|
||||
[Learn more](sql-scalar.md#string-functions)
|
||||
|
||||
## THETA_SKETCH_ESTIMATE
|
||||
|
||||
`THETA_SKETCH_ESTIMATE(expr)`
|
||||
|
|
|
@ -66,7 +66,6 @@ import org.apache.druid.segment.VirtualColumns;
|
|||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.indexing.CombinedDataSchema;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.server.coordinator.CompactionConfigValidationResult;
|
||||
|
@ -120,15 +119,14 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
* <ul>
|
||||
* <li>partitionsSpec of type HashedParititionsSpec.</li>
|
||||
* <li>maxTotalRows in DynamicPartitionsSpec.</li>
|
||||
* <li>rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.</li>
|
||||
* <li>queryGranularity set to ALL in granularitySpec.</li>
|
||||
* <li>Each metric has output column name same as the input name.</li>
|
||||
* <li>rollup in granularitySpec set to false when metricsSpec is specified or true when it's null.
|
||||
* Null is treated as true if metricsSpec exist and false if empty.</li>
|
||||
* <li>any metric is non-idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A != A.combiningFactory()'.</li>
|
||||
* </ul>
|
||||
*/
|
||||
@Override
|
||||
public CompactionConfigValidationResult validateCompactionTask(
|
||||
CompactionTask compactionTask,
|
||||
Map<Interval, DataSchema> intervalToDataSchemaMap
|
||||
CompactionTask compactionTask
|
||||
)
|
||||
{
|
||||
List<CompactionConfigValidationResult> validationResults = new ArrayList<>();
|
||||
|
@ -144,57 +142,13 @@ public class MSQCompactionRunner implements CompactionRunner
|
|||
));
|
||||
}
|
||||
validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext()));
|
||||
validationResults.add(validateRolledUpSegments(intervalToDataSchemaMap));
|
||||
validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec()));
|
||||
return validationResults.stream()
|
||||
.filter(result -> !result.isValid())
|
||||
.findFirst()
|
||||
.orElse(CompactionConfigValidationResult.success());
|
||||
}
|
||||
|
||||
/**
|
||||
* Valides that there are no rolled-up segments where either:
|
||||
* <ul>
|
||||
* <li>aggregator factory differs from its combining factory </li>
|
||||
* <li>input col name is different from the output name (non-idempotent)</li>
|
||||
* </ul>
|
||||
*/
|
||||
private CompactionConfigValidationResult validateRolledUpSegments(Map<Interval, DataSchema> intervalToDataSchemaMap)
|
||||
{
|
||||
for (Map.Entry<Interval, DataSchema> intervalDataSchema : intervalToDataSchemaMap.entrySet()) {
|
||||
if (intervalDataSchema.getValue() instanceof CombinedDataSchema) {
|
||||
CombinedDataSchema combinedDataSchema = (CombinedDataSchema) intervalDataSchema.getValue();
|
||||
if (combinedDataSchema.hasRolledUpSegments()) {
|
||||
for (AggregatorFactory aggregatorFactory : combinedDataSchema.getAggregators()) {
|
||||
// This is a conservative check as existing rollup may have been idempotent but the aggregator provided in
|
||||
// compaction spec isn't. This would get properly compacted yet fails in the below pre-check.
|
||||
if (
|
||||
!(
|
||||
aggregatorFactory.getClass().equals(aggregatorFactory.getCombiningFactory().getClass()) &&
|
||||
(
|
||||
aggregatorFactory.requiredFields().isEmpty() ||
|
||||
(aggregatorFactory.requiredFields().size() == 1 &&
|
||||
aggregatorFactory.requiredFields()
|
||||
.get(0)
|
||||
.equals(aggregatorFactory.getName()))
|
||||
)
|
||||
)
|
||||
) {
|
||||
// MSQ doesn't support rolling up already rolled-up segments when aggregate column name is different from
|
||||
// the aggregated column name. This is because the aggregated values would then get overwritten by new
|
||||
// values and the existing values would be lost. Note that if no rollup is specified in an index spec,
|
||||
// the default value is true.
|
||||
return CompactionConfigValidationResult.failure(
|
||||
"MSQ: Rolled-up segments in compaction interval[%s].",
|
||||
intervalDataSchema.getKey()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return CompactionConfigValidationResult.success();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CurrentSubTaskHolder getCurrentSubTaskHolder()
|
||||
{
|
||||
|
|
|
@ -42,7 +42,6 @@ import org.apache.druid.indexing.common.task.CompactionTask;
|
|||
import org.apache.druid.indexing.common.task.TuningConfigBuilder;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.granularity.GranularityType;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -60,7 +59,6 @@ import org.apache.druid.segment.IndexSpec;
|
|||
import org.apache.druid.segment.data.CompressionFactory;
|
||||
import org.apache.druid.segment.data.CompressionStrategy;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.indexing.CombinedDataSchema;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
|
@ -131,7 +129,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -144,7 +142,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -157,7 +155,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -170,7 +168,7 @@ public class MSQCompactionRunnerTest
|
|||
null,
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -183,7 +181,7 @@ public class MSQCompactionRunnerTest
|
|||
new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null),
|
||||
null
|
||||
);
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -196,7 +194,41 @@ public class MSQCompactionRunnerTest
|
|||
new ClientCompactionTaskGranularitySpec(null, null, false),
|
||||
AGGREGATORS.toArray(new AggregatorFactory[0])
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid());
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollupTrueWithoutMetricsSpecIsInValid()
|
||||
{
|
||||
CompactionTask compactionTask = createCompactionTask(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
new ClientCompactionTaskGranularitySpec(null, null, true),
|
||||
null
|
||||
);
|
||||
Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQEngineWithUnsupportedMetricsSpecIsInValid()
|
||||
{
|
||||
// Aggregators having different input and ouput column names are unsupported.
|
||||
final String inputColName = "added";
|
||||
final String outputColName = "sum_added";
|
||||
CompactionTask compactionTask = createCompactionTask(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
new ClientCompactionTaskGranularitySpec(null, null, null),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(
|
||||
"MSQ: Non-idempotent aggregator[sum_added] not supported in 'metricsSpec'.",
|
||||
validationResult.getReason()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -345,48 +377,6 @@ public class MSQCompactionRunnerTest
|
|||
Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntervalsWithRolledUpSegmentsAndNonIdempotentAggregatorFails()
|
||||
{
|
||||
final String inputColName = "added";
|
||||
final String outputColName = "sum_added";
|
||||
CompactionTask compactionTask = createCompactionTask(
|
||||
null,
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
new AggregatorFactory[]{
|
||||
new LongSumAggregatorFactory(
|
||||
outputColName,
|
||||
inputColName
|
||||
)
|
||||
}
|
||||
);
|
||||
CombinedDataSchema dataSchema = new CombinedDataSchema(
|
||||
DATA_SOURCE,
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, null, null),
|
||||
new DimensionsSpec(DIMENSIONS),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)},
|
||||
new UniformGranularitySpec(
|
||||
SEGMENT_GRANULARITY.getDefaultGranularity(),
|
||||
null,
|
||||
false,
|
||||
Collections.singletonList(COMPACTION_INTERVAL)
|
||||
),
|
||||
null,
|
||||
true
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(
|
||||
compactionTask,
|
||||
Collections.singletonMap(COMPACTION_INTERVAL, dataSchema)
|
||||
);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(validationResult.getReason(), StringUtils.format(
|
||||
"MSQ: Rolled-up segments in compaction interval[%s].",
|
||||
COMPACTION_INTERVAL
|
||||
));
|
||||
}
|
||||
|
||||
private CompactionTask createCompactionTask(
|
||||
@Nullable PartitionsSpec partitionsSpec,
|
||||
@Nullable DimFilter dimFilter,
|
||||
|
|
|
@ -57,9 +57,6 @@ public interface CompactionRunner
|
|||
* Checks if the provided compaction config is supported by the runner.
|
||||
* The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask}
|
||||
*/
|
||||
CompactionConfigValidationResult validateCompactionTask(
|
||||
CompactionTask compactionTask,
|
||||
Map<Interval, DataSchema> intervalToDataSchemaMap
|
||||
);
|
||||
CompactionConfigValidationResult validateCompactionTask(CompactionTask compactionTask);
|
||||
|
||||
}
|
||||
|
|
|
@ -77,7 +77,6 @@ import org.apache.druid.segment.IndexSpec;
|
|||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.indexing.CombinedDataSchema;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.segment.indexing.TuningConfig;
|
||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||
|
@ -460,13 +459,11 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
transformSpec,
|
||||
metricsSpec,
|
||||
granularitySpec,
|
||||
getMetricBuilder(),
|
||||
compactionRunner
|
||||
getMetricBuilder()
|
||||
);
|
||||
|
||||
registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder());
|
||||
CompactionConfigValidationResult supportsCompactionConfig =
|
||||
compactionRunner.validateCompactionTask(this, intervalDataSchemas);
|
||||
CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this);
|
||||
if (!supportsCompactionConfig.isValid()) {
|
||||
throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason());
|
||||
}
|
||||
|
@ -488,8 +485,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
@Nullable final ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable final AggregatorFactory[] metricsSpec,
|
||||
@Nullable final ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
final ServiceMetricEvent.Builder metricBuilder,
|
||||
CompactionRunner compactionRunner
|
||||
final ServiceMetricEvent.Builder metricBuilder
|
||||
) throws IOException
|
||||
{
|
||||
final Iterable<DataSegment> timelineSegments = retrieveRelevantTimelineHolders(
|
||||
|
@ -553,8 +549,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
metricsSpec,
|
||||
granularitySpec == null
|
||||
? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null)
|
||||
: granularitySpec.withSegmentGranularity(segmentGranularityToUse),
|
||||
compactionRunner
|
||||
: granularitySpec.withSegmentGranularity(segmentGranularityToUse)
|
||||
);
|
||||
intervalDataSchemaMap.put(interval, dataSchema);
|
||||
}
|
||||
|
@ -579,8 +574,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
dimensionsSpec,
|
||||
transformSpec,
|
||||
metricsSpec,
|
||||
granularitySpec,
|
||||
compactionRunner
|
||||
granularitySpec
|
||||
);
|
||||
return Collections.singletonMap(segmentProvider.interval, dataSchema);
|
||||
}
|
||||
|
@ -610,17 +604,13 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
@Nullable DimensionsSpec dimensionsSpec,
|
||||
@Nullable ClientCompactionTaskTransformSpec transformSpec,
|
||||
@Nullable AggregatorFactory[] metricsSpec,
|
||||
@Nonnull ClientCompactionTaskGranularitySpec granularitySpec,
|
||||
@Nullable CompactionRunner compactionRunner
|
||||
@Nonnull ClientCompactionTaskGranularitySpec granularitySpec
|
||||
)
|
||||
{
|
||||
// Check index metadata & decide which values to propagate (i.e. carry over) for rollup & queryGranularity
|
||||
final ExistingSegmentAnalyzer existingSegmentAnalyzer = new ExistingSegmentAnalyzer(
|
||||
segments,
|
||||
// For MSQ, always need rollup to check if there are some rollup segments already present.
|
||||
compactionRunner instanceof NativeCompactionRunner
|
||||
? (granularitySpec.isRollup() == null)
|
||||
: true,
|
||||
granularitySpec.isRollup() == null,
|
||||
granularitySpec.getQueryGranularity() == null,
|
||||
dimensionsSpec == null,
|
||||
metricsSpec == null
|
||||
|
@ -675,14 +665,13 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
finalMetricsSpec = metricsSpec;
|
||||
}
|
||||
|
||||
return new CombinedDataSchema(
|
||||
return new DataSchema(
|
||||
dataSource,
|
||||
new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null),
|
||||
finalDimensionsSpec,
|
||||
finalMetricsSpec,
|
||||
uniformGranularitySpec,
|
||||
transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null),
|
||||
existingSegmentAnalyzer.hasRolledUpSegments()
|
||||
transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -759,7 +748,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
|
||||
// For processRollup:
|
||||
private boolean rollup = true;
|
||||
private boolean hasRolledUpSegments = false;
|
||||
|
||||
// For processQueryGranularity:
|
||||
private Granularity queryGranularity;
|
||||
|
@ -827,11 +815,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
return rollup;
|
||||
}
|
||||
|
||||
public boolean hasRolledUpSegments()
|
||||
{
|
||||
return hasRolledUpSegments;
|
||||
}
|
||||
|
||||
public Granularity getQueryGranularity()
|
||||
{
|
||||
if (!needQueryGranularity) {
|
||||
|
@ -921,7 +904,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
|
|||
// Pick rollup value if all segments being compacted have the same, non-null, value otherwise set it to false
|
||||
final Boolean isIndexRollup = index.getMetadata().isRollup();
|
||||
rollup = rollup && Boolean.valueOf(true).equals(isIndexRollup);
|
||||
hasRolledUpSegments = hasRolledUpSegments || Boolean.valueOf(true).equals(isIndexRollup);
|
||||
}
|
||||
|
||||
private void processQueryGranularity(final QueryableIndex index)
|
||||
|
|
|
@ -85,8 +85,7 @@ public class NativeCompactionRunner implements CompactionRunner
|
|||
|
||||
@Override
|
||||
public CompactionConfigValidationResult validateCompactionTask(
|
||||
CompactionTask compactionTask,
|
||||
Map<Interval, DataSchema> intervalToDataSchemaMap
|
||||
CompactionTask compactionTask
|
||||
)
|
||||
{
|
||||
return CompactionConfigValidationResult.success();
|
||||
|
|
|
@ -737,7 +737,7 @@ public class CompactionTaskTest
|
|||
);
|
||||
provider.checkSegments(LockGranularity.TIME_CHUNK, ImmutableList.of());
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testCreateIngestionSchema() throws IOException
|
||||
{
|
||||
|
@ -749,8 +749,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -811,8 +810,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -874,8 +872,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -938,8 +935,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1009,8 +1005,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1060,8 +1055,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
customMetricsSpec,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1104,8 +1098,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1155,8 +1148,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1186,8 +1178,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1228,8 +1219,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null, null),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1273,8 +1263,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
dataSchemasForIntervals,
|
||||
|
@ -1319,8 +1308,7 @@ public class CompactionTaskTest
|
|||
new PeriodGranularity(Period.months(3), null, null),
|
||||
null
|
||||
),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1367,8 +1355,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1413,8 +1400,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new ClientCompactionTaskGranularitySpec(null, null, null),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1459,8 +1445,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new ClientCompactionTaskGranularitySpec(null, null, true),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1490,8 +1475,7 @@ public class CompactionTaskTest
|
|||
null,
|
||||
null,
|
||||
new ClientCompactionTaskGranularitySpec(null, null, null),
|
||||
METRIC_BUILDER,
|
||||
null
|
||||
METRIC_BUILDER
|
||||
);
|
||||
|
||||
final List<ParallelIndexIngestionSpec> ingestionSpecs = NativeCompactionRunner.createIngestionSpecs(
|
||||
|
@ -1871,14 +1855,6 @@ public class CompactionTaskTest
|
|||
}
|
||||
}
|
||||
|
||||
final Metadata metadata = new Metadata(
|
||||
null,
|
||||
aggregatorFactories.toArray(new AggregatorFactory[0]),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
queryableIndexMap.put(
|
||||
entry.getValue(),
|
||||
new SimpleQueryableIndex(
|
||||
|
@ -1887,9 +1863,21 @@ public class CompactionTaskTest
|
|||
null,
|
||||
columnMap,
|
||||
null,
|
||||
metadata,
|
||||
false
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
return new Metadata(
|
||||
null,
|
||||
aggregatorFactories.toArray(new AggregatorFactory[0]),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -1912,10 +1900,15 @@ public class CompactionTaskTest
|
|||
index.getBitmapFactoryForDimensions(),
|
||||
index.getColumns(),
|
||||
index.getFileMapper(),
|
||||
null,
|
||||
false
|
||||
)
|
||||
);
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -80,7 +80,6 @@ import org.testng.annotations.DataProvider;
|
|||
import org.testng.annotations.Guice;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
@ -184,7 +183,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), null, false, false),
|
||||
new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null)
|
||||
},
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// should now only have 1 row after compaction
|
||||
// added = null, count = 3, sum_added = 93.0
|
||||
|
@ -286,7 +286,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
),
|
||||
new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null)
|
||||
},
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// should now only have 1 row after compaction
|
||||
// added = null, count = 3, sum_added = 93
|
||||
|
@ -328,8 +329,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test(dataProvider = "engine")
|
||||
public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception
|
||||
@Test()
|
||||
public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception
|
||||
{
|
||||
// added = 31, count = null, sum_added = null
|
||||
loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS);
|
||||
|
@ -357,7 +358,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
null,
|
||||
new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")},
|
||||
false,
|
||||
engine
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// should now only have 1 row after compaction
|
||||
// added = null, count = 2, sum_added = 62
|
||||
|
@ -480,7 +481,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))),
|
||||
null,
|
||||
new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")},
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// should now only have 1 row after compaction
|
||||
// added = null, count = 4, sum_added = 124
|
||||
|
@ -521,7 +523,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
verifySegmentsCount(4);
|
||||
verifyQuery(INDEX_QUERIES_RESOURCE);
|
||||
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1));
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), CompactionEngine.NATIVE);
|
||||
//...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total)
|
||||
forceTriggerAutoCompaction(3);
|
||||
verifyQuery(INDEX_QUERIES_RESOURCE);
|
||||
|
@ -539,7 +541,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
0,
|
||||
1,
|
||||
1);
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET);
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, CompactionEngine.NATIVE);
|
||||
//...compacted into 1 new segment for the remaining one day. 2 day compacted and 0 day uncompacted. (2 total)
|
||||
forceTriggerAutoCompaction(2);
|
||||
verifyQuery(INDEX_QUERIES_RESOURCE);
|
||||
|
@ -651,7 +653,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
verifySegmentsCount(4);
|
||||
verifyQuery(INDEX_QUERIES_RESOURCE);
|
||||
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET);
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, CompactionEngine.NATIVE);
|
||||
// ...should remains unchanged (4 total)
|
||||
forceTriggerAutoCompaction(4);
|
||||
verifyQuery(INDEX_QUERIES_RESOURCE);
|
||||
|
@ -863,7 +865,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
Granularity newGranularity = Granularities.YEAR;
|
||||
// Set dropExisting to true
|
||||
// "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z",
|
||||
submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine);
|
||||
submitCompactionConfig(
|
||||
1000,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
true,
|
||||
engine
|
||||
);
|
||||
|
||||
List<String> expectedIntervalAfterCompaction = new ArrayList<>();
|
||||
for (String interval : intervalsBeforeCompaction) {
|
||||
|
@ -881,7 +889,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
// "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z",
|
||||
newGranularity = Granularities.MONTH;
|
||||
// Set dropExisting to true
|
||||
submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine);
|
||||
submitCompactionConfig(
|
||||
1000,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
true,
|
||||
engine
|
||||
);
|
||||
|
||||
// Since dropExisting is set to true...
|
||||
// Again data is only in two days
|
||||
|
@ -950,7 +964,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
|
||||
Granularity newGranularity = Granularities.YEAR;
|
||||
// Set dropExisting to false
|
||||
submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false);
|
||||
submitCompactionConfig(
|
||||
1000,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
|
||||
LOG.info("Auto compaction test with YEAR segment granularity");
|
||||
|
||||
|
@ -967,7 +987,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
|
||||
newGranularity = Granularities.DAY;
|
||||
// Set dropExisting to false
|
||||
submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false);
|
||||
submitCompactionConfig(
|
||||
1000,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
|
||||
LOG.info("Auto compaction test with DAY segment granularity");
|
||||
|
||||
|
@ -1169,7 +1195,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
|
||||
Granularity newGranularity = Granularities.YEAR;
|
||||
// Set dropExisting to false
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false);
|
||||
submitCompactionConfig(
|
||||
MAX_ROWS_PER_SEGMENT_COMPACTED,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
|
||||
List<String> expectedIntervalAfterCompaction = new ArrayList<>();
|
||||
// We wil have one segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR)
|
||||
|
@ -1195,7 +1227,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
|
||||
newGranularity = Granularities.MONTH;
|
||||
// Set dropExisting to false
|
||||
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false);
|
||||
submitCompactionConfig(
|
||||
MAX_ROWS_PER_SEGMENT_COMPACTED,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(newGranularity, null, null),
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// Since dropExisting is set to true...
|
||||
// This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity
|
||||
expectedIntervalAfterCompaction = new ArrayList<>();
|
||||
|
@ -1239,7 +1277,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
MAX_ROWS_PER_SEGMENT_COMPACTED,
|
||||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(Granularities.WEEK, null, null),
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// Before compaction, we have segments with the interval 2013-08-01/2013-09-01 and 2013-09-01/2013-10-01
|
||||
// We will compact the latest segment, 2013-09-01/2013-10-01, to WEEK.
|
||||
|
@ -1319,8 +1358,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test(dataProvider = "engine")
|
||||
public void testAutoCompactionDutyWithRollup(CompactionEngine engine) throws Exception
|
||||
@Test()
|
||||
public void testAutoCompactionDutyWithRollup() throws Exception
|
||||
{
|
||||
final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles"));
|
||||
Map<String, Object> specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono))));
|
||||
|
@ -1337,7 +1376,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
NO_SKIP_OFFSET,
|
||||
new UserCompactionTaskGranularityConfig(null, null, true),
|
||||
false,
|
||||
engine
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
forceTriggerAutoCompaction(2);
|
||||
queryAndResultFields = ImmutableMap.of(
|
||||
|
@ -1470,7 +1509,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
null,
|
||||
new UserCompactionTaskTransformConfig(new SelectorDimFilter("page", "Striker Eureka", null)),
|
||||
null,
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
forceTriggerAutoCompaction(2);
|
||||
|
||||
|
@ -1517,7 +1557,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
null,
|
||||
null,
|
||||
new AggregatorFactory[] {new DoubleSumAggregatorFactory("double_sum_added", "added"), new LongSumAggregatorFactory("long_sum_added", "added")},
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
forceTriggerAutoCompaction(2);
|
||||
|
||||
|
@ -1577,7 +1618,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
false,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
// Compact the MONTH segment
|
||||
forceTriggerAutoCompaction(2);
|
||||
|
@ -1679,57 +1721,31 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
queryHelper.testQueriesFromString(queryResponseTemplate);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest)
|
||||
throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, null);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
@Nullable CompactionEngine engine
|
||||
CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, null);
|
||||
}
|
||||
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
@Nullable CompactionEngine engine
|
||||
CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, engine);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
boolean dropExisting
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, dropExisting, null);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
boolean dropExisting,
|
||||
@Nullable CompactionEngine engine
|
||||
CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(
|
||||
|
@ -1744,28 +1760,6 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
UserCompactionTaskGranularityConfig granularitySpec,
|
||||
UserCompactionTaskDimensionsConfig dimensionsSpec,
|
||||
UserCompactionTaskTransformConfig transformSpec,
|
||||
AggregatorFactory[] metricsSpec,
|
||||
boolean dropExisting
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(
|
||||
maxRowsPerSegment,
|
||||
skipOffsetFromLatest,
|
||||
granularitySpec,
|
||||
dimensionsSpec,
|
||||
transformSpec,
|
||||
metricsSpec,
|
||||
dropExisting,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
private void submitCompactionConfig(
|
||||
Integer maxRowsPerSegment,
|
||||
Period skipOffsetFromLatest,
|
||||
|
@ -1774,7 +1768,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
UserCompactionTaskTransformConfig transformSpec,
|
||||
AggregatorFactory[] metricsSpec,
|
||||
boolean dropExisting,
|
||||
@Nullable CompactionEngine engine
|
||||
CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
submitCompactionConfig(
|
||||
|
@ -1799,7 +1793,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
UserCompactionTaskTransformConfig transformSpec,
|
||||
AggregatorFactory[] metricsSpec,
|
||||
boolean dropExisting,
|
||||
@Nullable CompactionEngine engine
|
||||
CompactionEngine engine
|
||||
) throws Exception
|
||||
{
|
||||
DataSourceCompactionConfig dataSourceCompactionConfig = new DataSourceCompactionConfig(
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols;
|
||||
package org.apache.druid.common.semantic;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Retention;
|
||||
|
@ -26,8 +28,8 @@ import java.lang.annotation.Target;
|
|||
|
||||
/**
|
||||
* Annotation used to indicate that the method is used as a creator for a semantic interface.
|
||||
*
|
||||
* Used in conjuction with {@link RowsAndColumns#makeAsMap(Class)} to build maps for simplified implementation of
|
||||
* <p>
|
||||
* Used in conjuction with {@link SemanticUtils#makeAsMap(Class)} to build maps for simplified implementation of
|
||||
* the {@link RowsAndColumns#as(Class)} method.
|
||||
*/
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.common.semantic;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class SemanticUtils
|
||||
{
|
||||
private static final Map<Class<?>, Map<Class<?>, Function<?, ?>>> OVERRIDES = new LinkedHashMap<>();
|
||||
|
||||
/**
|
||||
* Allows the registration of overrides, which allows overriding of already existing mappings.
|
||||
* This allows extensions to register mappings.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
public static <C, T> void registerAsOverride(Class<C> clazz, Class<T> asInterface, Function<C, T> fn)
|
||||
{
|
||||
final Map<Class<?>, Function<?, ?>> classOverrides = OVERRIDES.computeIfAbsent(
|
||||
clazz,
|
||||
theClazz -> new LinkedHashMap<>()
|
||||
);
|
||||
|
||||
final Function<?, ?> oldVal = classOverrides.get(asInterface);
|
||||
if (oldVal != null) {
|
||||
throw DruidException.defensive(
|
||||
"Attempt to side-override the same interface [%s] multiple times for the same class [%s].",
|
||||
asInterface,
|
||||
clazz
|
||||
);
|
||||
} else {
|
||||
classOverrides.put(asInterface, fn);
|
||||
}
|
||||
}
|
||||
|
||||
public static <T> Map<Class<?>, Function<T, ?>> makeAsMap(Class<T> clazz)
|
||||
{
|
||||
final Map<Class<?>, Function<T, ?>> retVal = new HashMap<>();
|
||||
|
||||
for (Method method : clazz.getMethods()) {
|
||||
if (method.isAnnotationPresent(SemanticCreator.class)) {
|
||||
if (method.getParameterCount() != 0) {
|
||||
throw DruidException.defensive("Method [%s] annotated with SemanticCreator was not 0-argument.", method);
|
||||
}
|
||||
|
||||
retVal.put(method.getReturnType(), arg -> {
|
||||
try {
|
||||
return method.invoke(arg);
|
||||
}
|
||||
catch (InvocationTargetException | IllegalAccessException e) {
|
||||
throw DruidException.defensive().build(e, "Problem invoking method [%s]", method);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
final Map<Class<?>, Function<?, ?>> classOverrides = OVERRIDES.get(clazz);
|
||||
if (classOverrides != null) {
|
||||
for (Map.Entry<Class<?>, Function<?, ?>> overrideEntry : classOverrides.entrySet()) {
|
||||
//noinspection unchecked
|
||||
retVal.put(overrideEntry.getKey(), (Function<T, ?>) overrideEntry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import javax.annotation.concurrent.NotThreadSafe;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -130,6 +131,8 @@ import java.util.Map;
|
|||
@NotThreadSafe
|
||||
public class DruidException extends RuntimeException
|
||||
{
|
||||
public static final String CLASS_NAME_STR = DruidException.class.getName();
|
||||
|
||||
/**
|
||||
* Starts building a "general" DruidException targeting the specified persona.
|
||||
*
|
||||
|
@ -176,6 +179,17 @@ public class DruidException extends RuntimeException
|
|||
return defensive().build(format, args);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a "defensive" exception, this is an exception that should never actually be triggered, but we are
|
||||
* throwing it inside a defensive check.
|
||||
*
|
||||
* @return A builder for a defensive exception.
|
||||
*/
|
||||
public static DruidException defensive(Throwable cause, String format, Object... args)
|
||||
{
|
||||
return defensive().build(cause, format, args);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build a "defensive" exception, this is an exception that should never actually be triggered. Throw to
|
||||
* allow messages to be seen by developers
|
||||
|
@ -467,7 +481,7 @@ public class DruidException extends RuntimeException
|
|||
|
||||
public DruidException build(Throwable cause, String formatMe, Object... vals)
|
||||
{
|
||||
return new DruidException(
|
||||
final DruidException retVal = new DruidException(
|
||||
cause,
|
||||
errorCode,
|
||||
targetPersona,
|
||||
|
@ -475,6 +489,19 @@ public class DruidException extends RuntimeException
|
|||
StringUtils.nonStrictFormat(formatMe, vals),
|
||||
deserialized
|
||||
);
|
||||
|
||||
StackTraceElement[] stackTrace = retVal.getStackTrace();
|
||||
int firstNonDruidExceptionIndex = 0;
|
||||
while (
|
||||
firstNonDruidExceptionIndex < stackTrace.length
|
||||
&& stackTrace[firstNonDruidExceptionIndex].getClassName().startsWith(CLASS_NAME_STR)) {
|
||||
++firstNonDruidExceptionIndex;
|
||||
}
|
||||
if (firstNonDruidExceptionIndex > 0) {
|
||||
retVal.setStackTrace(Arrays.copyOfRange(stackTrace, firstNonDruidExceptionIndex, stackTrace.length));
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
/**
|
||||
* A failure class that is used to indicate that something is just not implemented yet. This is useful when a
|
||||
* developer builds something and they intentionally do not implement a specific branch of code or type of object.
|
||||
* <p>
|
||||
* The lack of implementation is not necessarily a statement that it SHOULDN'T be implemented, it's just an indication
|
||||
* that it has not YET been implemented. When one of these exceptions is seen, it is usually an indication that it is
|
||||
* now time to actually implement the path that was previously elided.
|
||||
* <p>
|
||||
* Oftentimes, the code path wasn't implemented because the developer thought that it wasn't actually possible to
|
||||
* see it executed. So, collecting and providing information about why the particular path got executed is often
|
||||
* extremely helpful in understanding why it happened and accelerating the implementation of what the correct behavior
|
||||
* should be.
|
||||
*/
|
||||
public class NotYetImplemented extends DruidException.Failure
|
||||
{
|
||||
public static DruidException ex(Throwable t, String msg, Object... args)
|
||||
{
|
||||
return DruidException.fromFailure(new NotYetImplemented(t, msg, args));
|
||||
}
|
||||
|
||||
private final Throwable t;
|
||||
private final String msg;
|
||||
private final Object[] args;
|
||||
|
||||
public NotYetImplemented(Throwable t, String msg, Object[] args)
|
||||
{
|
||||
super("notYetImplemented");
|
||||
this.t = t;
|
||||
this.msg = msg;
|
||||
this.args = args;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected DruidException makeException(DruidException.DruidExceptionBuilder bob)
|
||||
{
|
||||
bob = bob.forPersona(DruidException.Persona.DEVELOPER)
|
||||
.ofCategory(DruidException.Category.DEFENSIVE);
|
||||
|
||||
if (t == null) {
|
||||
return bob.build(msg, args);
|
||||
} else {
|
||||
return bob.build(t, msg, args);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.frame.read;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.field.FieldReader;
|
||||
import org.apache.druid.frame.field.FieldReaders;
|
||||
|
@ -31,7 +32,6 @@ import org.apache.druid.frame.read.columnar.FrameColumnReaders;
|
|||
import org.apache.druid.frame.segment.row.FrameCursorFactory;
|
||||
import org.apache.druid.frame.write.FrameWriterUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.segment.CursorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -44,7 +44,7 @@ import java.util.Set;
|
|||
|
||||
/**
|
||||
* Embeds the logic to read frames with a given {@link RowSignature}.
|
||||
*
|
||||
* <p>
|
||||
* Stateless and immutable.
|
||||
*/
|
||||
public class FrameReader
|
||||
|
@ -146,7 +146,7 @@ public class FrameReader
|
|||
case ROW_BASED:
|
||||
return new FrameCursorFactory(frame, this, fieldReaders);
|
||||
default:
|
||||
throw new ISE("Unrecognized frame type [%s]", frame.type());
|
||||
throw DruidException.defensive("Unrecognized frame type [%s]", frame.type());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -236,7 +236,7 @@ public class JsonConfigurator
|
|||
// to configure ParametrizedUriEmitterConfig object. So skipping xxx=yyy key-value pair when configuring Emitter
|
||||
// doesn't make any difference. That is why we just log this situation, instead of throwing an exception.
|
||||
log.info(
|
||||
"Skipping %s property: one of it's prefixes is also used as a property key. Prefix: %s",
|
||||
"Skipping property [%s]: one of it's prefixes [%s] is also used as a property key.",
|
||||
originalProperty,
|
||||
propertyPrefix
|
||||
);
|
||||
|
|
|
@ -23,6 +23,8 @@ import it.unimi.dsi.fastutil.Arrays;
|
|||
import it.unimi.dsi.fastutil.ints.IntArrayList;
|
||||
import it.unimi.dsi.fastutil.ints.IntComparator;
|
||||
import it.unimi.dsi.fastutil.ints.IntList;
|
||||
import org.apache.druid.common.semantic.SemanticCreator;
|
||||
import org.apache.druid.common.semantic.SemanticUtils;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
|
@ -73,7 +75,7 @@ import java.util.function.Function;
|
|||
public class ArrayListRowsAndColumns<RowType> implements AppendableRowsAndColumns
|
||||
{
|
||||
@SuppressWarnings("rawtypes")
|
||||
private static final Map<Class<?>, Function<ArrayListRowsAndColumns, ?>> AS_MAP = RowsAndColumns
|
||||
private static final Map<Class<?>, Function<ArrayListRowsAndColumns, ?>> AS_MAP = SemanticUtils
|
||||
.makeAsMap(ArrayListRowsAndColumns.class);
|
||||
|
||||
private final ArrayList<RowType> rows;
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.druid.query.rowsandcols;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.common.semantic.SemanticCreator;
|
||||
import org.apache.druid.common.semantic.SemanticUtils;
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
|
||||
import org.apache.druid.frame.key.KeyColumn;
|
||||
|
@ -66,7 +68,7 @@ import java.util.function.Function;
|
|||
|
||||
public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
||||
{
|
||||
private static final Map<Class<?>, Function<LazilyDecoratedRowsAndColumns, ?>> AS_MAP = RowsAndColumns
|
||||
private static final Map<Class<?>, Function<LazilyDecoratedRowsAndColumns, ?>> AS_MAP = SemanticUtils
|
||||
.makeAsMap(LazilyDecoratedRowsAndColumns.class);
|
||||
|
||||
private RowsAndColumns base;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.query.groupby.ResultRow;
|
|||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.LongArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -170,6 +171,12 @@ public class MapOfColumnsRowsAndColumns implements RowsAndColumns
|
|||
return add(name, new IntArrayColumn(vals));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public Builder add(String name, long[] vals)
|
||||
{
|
||||
return add(name, new LongArrayColumn(vals));
|
||||
}
|
||||
|
||||
public Builder add(String name, double[] vals)
|
||||
{
|
||||
return add(name, new DoubleArrayColumn(vals));
|
||||
|
|
|
@ -19,19 +19,13 @@
|
|||
|
||||
package org.apache.druid.query.rowsandcols;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* An interface representing a chunk of RowsAndColumns. Essentially a RowsAndColumns is just a batch of rows
|
||||
|
@ -75,31 +69,6 @@ public interface RowsAndColumns
|
|||
return retVal;
|
||||
}
|
||||
|
||||
static <T> Map<Class<?>, Function<T, ?>> makeAsMap(Class<T> clazz)
|
||||
{
|
||||
Map<Class<?>, Function<T, ?>> retVal = new HashMap<>();
|
||||
|
||||
for (Method method : clazz.getMethods()) {
|
||||
if (method.isAnnotationPresent(SemanticCreator.class)) {
|
||||
if (method.getParameterCount() != 0) {
|
||||
throw DruidException.defensive("Method [%s] annotated with SemanticCreator was not 0-argument.", method);
|
||||
}
|
||||
|
||||
retVal.put(method.getReturnType(), arg -> {
|
||||
try {
|
||||
return method.invoke(arg);
|
||||
}
|
||||
catch (InvocationTargetException | IllegalAccessException e) {
|
||||
throw DruidException.defensive().build(e, "Problem invoking method [%s]", method);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* The set of column names available from the RowsAndColumns
|
||||
*
|
||||
|
|
|
@ -22,6 +22,10 @@ package org.apache.druid.query.rowsandcols.column;
|
|||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
|
||||
/**
|
||||
* The implementations of this interface will not validate that things are sorted for the binary search, it assumes that
|
||||
* they must be. As such, behavior are undefined if the column is not actually sorted.
|
||||
*/
|
||||
public interface BinarySearchableAccessor extends ColumnAccessor
|
||||
{
|
||||
static BinarySearchableAccessor fromColumn(Column col)
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.apache.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
|
@ -55,7 +55,13 @@ public class ConstantObjectColumn implements Column
|
|||
if (VectorCopier.class.equals(clazz)) {
|
||||
return (T) (VectorCopier) (into, intoStart) -> {
|
||||
if (Integer.MAX_VALUE - numRows < intoStart) {
|
||||
throw new ISE("too many rows!!! intoStart[%,d], numRows[%,d] combine to exceed max_int", intoStart, numRows);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.CAPACITY_EXCEEDED)
|
||||
.build(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
numRows
|
||||
);
|
||||
}
|
||||
Arrays.fill(into, intoStart, intoStart + numRows, obj);
|
||||
};
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package org.apache.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Numbers;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.error.NotYetImplemented;
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
|
@ -54,11 +54,13 @@ public class DoubleArrayColumn implements Column
|
|||
if (VectorCopier.class.equals(clazz)) {
|
||||
return (T) (VectorCopier) (into, intoStart) -> {
|
||||
if (Integer.MAX_VALUE - vals.length < intoStart) {
|
||||
throw new ISE(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
vals.length
|
||||
);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.CAPACITY_EXCEEDED)
|
||||
.build(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
vals.length
|
||||
);
|
||||
}
|
||||
for (int i = 0; i < vals.length; ++i) {
|
||||
into[intoStart + i] = vals[i];
|
||||
|
@ -183,13 +185,13 @@ public class DoubleArrayColumn implements Column
|
|||
@Override
|
||||
public FindResult findString(int startIndex, int endIndex, String val)
|
||||
{
|
||||
return findDouble(startIndex, endIndex, Numbers.tryParseDouble(val, 0));
|
||||
throw NotYetImplemented.ex(null, "findString is not currently supported for DoubleArrayColumns");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findComplex(int startIndex, int endIndex, Object val)
|
||||
{
|
||||
return findDouble(startIndex, endIndex, Numbers.tryParseDouble(val, 0));
|
||||
throw NotYetImplemented.ex(null, "findComplex is not currently supported for DoubleArrayColumns");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,8 @@
|
|||
|
||||
package org.apache.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Numbers;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.error.NotYetImplemented;
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
|
@ -54,11 +54,13 @@ public class IntArrayColumn implements Column
|
|||
if (VectorCopier.class.equals(clazz)) {
|
||||
return (T) (VectorCopier) (into, intoStart) -> {
|
||||
if (Integer.MAX_VALUE - vals.length < intoStart) {
|
||||
throw new ISE(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
vals.length
|
||||
);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.CAPACITY_EXCEEDED)
|
||||
.build(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
vals.length
|
||||
);
|
||||
}
|
||||
for (int i = 0; i < vals.length; ++i) {
|
||||
into[intoStart + i] = vals[i];
|
||||
|
@ -189,13 +191,13 @@ public class IntArrayColumn implements Column
|
|||
@Override
|
||||
public FindResult findString(int startIndex, int endIndex, String val)
|
||||
{
|
||||
return findInt(startIndex, endIndex, (int) Numbers.tryParseLong(val, 0));
|
||||
throw NotYetImplemented.ex(null, "findString is not currently supported for IntArrayColumns");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findComplex(int startIndex, int endIndex, Object val)
|
||||
{
|
||||
return findDouble(startIndex, endIndex, (int) Numbers.tryParseLong(val, 0));
|
||||
throw NotYetImplemented.ex(null, "findComplex is not currently supported for IntArrayColumns");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,204 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.error.NotYetImplemented;
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class LongArrayColumn implements Column
|
||||
{
|
||||
private final long[] vals;
|
||||
|
||||
public LongArrayColumn(
|
||||
long[] vals
|
||||
)
|
||||
{
|
||||
this.vals = vals;
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
@Override
|
||||
public ColumnAccessor toAccessor()
|
||||
{
|
||||
return new MyColumnAccessor();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
if (VectorCopier.class.equals(clazz)) {
|
||||
return (T) (VectorCopier) (into, intoStart) -> {
|
||||
if (Integer.MAX_VALUE - vals.length < intoStart) {
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.CAPACITY_EXCEEDED)
|
||||
.build(
|
||||
"too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int",
|
||||
intoStart,
|
||||
vals.length
|
||||
);
|
||||
}
|
||||
for (int i = 0; i < vals.length; ++i) {
|
||||
into[intoStart + i] = vals[i];
|
||||
}
|
||||
};
|
||||
}
|
||||
if (ColumnValueSwapper.class.equals(clazz)) {
|
||||
return (T) (ColumnValueSwapper) (lhs, rhs) -> {
|
||||
long tmp = vals[lhs];
|
||||
vals[lhs] = vals[rhs];
|
||||
vals[rhs] = tmp;
|
||||
};
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private class MyColumnAccessor implements BinarySearchableAccessor
|
||||
{
|
||||
@Override
|
||||
public ColumnType getType()
|
||||
{
|
||||
return ColumnType.LONG;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numRows()
|
||||
{
|
||||
return vals.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(int rowNum)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(int rowNum)
|
||||
{
|
||||
return vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(int rowNum)
|
||||
{
|
||||
return (int) vals[rowNum];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareRows(int lhsRowNum, int rhsRowNum)
|
||||
{
|
||||
return Long.compare(vals[lhsRowNum], vals[rhsRowNum]);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FindResult findNull(int startIndex, int endIndex)
|
||||
{
|
||||
return FindResult.notFound(endIndex);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findDouble(int startIndex, int endIndex, double val)
|
||||
{
|
||||
return findLong(startIndex, endIndex, (long) val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findFloat(int startIndex, int endIndex, float val)
|
||||
{
|
||||
return findLong(startIndex, endIndex, (long) val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findLong(int startIndex, int endIndex, long val)
|
||||
{
|
||||
if (vals[startIndex] == val) {
|
||||
int end = startIndex + 1;
|
||||
|
||||
while (end < endIndex && vals[end] == val) {
|
||||
++end;
|
||||
}
|
||||
return FindResult.found(startIndex, end);
|
||||
}
|
||||
|
||||
int i = Arrays.binarySearch(vals, startIndex, endIndex, val);
|
||||
if (i > 0) {
|
||||
int foundStart = i;
|
||||
int foundEnd = i + 1;
|
||||
|
||||
while (foundStart - 1 >= startIndex && vals[foundStart - 1] == val) {
|
||||
--foundStart;
|
||||
}
|
||||
|
||||
while (foundEnd < endIndex && vals[foundEnd] == val) {
|
||||
++foundEnd;
|
||||
}
|
||||
|
||||
return FindResult.found(foundStart, foundEnd);
|
||||
} else {
|
||||
return FindResult.notFound(-(i + 1));
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public FindResult findInt(int startIndex, int endIndex, int val)
|
||||
{
|
||||
return findLong(startIndex, endIndex, val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findString(int startIndex, int endIndex, String val)
|
||||
{
|
||||
throw NotYetImplemented.ex(null, "findString is not currently supported for LongArrayColumns");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FindResult findComplex(int startIndex, int endIndex, Object val)
|
||||
{
|
||||
throw NotYetImplemented.ex(null, "findComplex is not currently supported for LongArrayColumns");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.semantic.WireTransferable;
|
||||
import org.apache.druid.segment.CloseableShapeshifter;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -80,7 +79,6 @@ public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose
|
|||
}
|
||||
}
|
||||
return colCache.get(name);
|
||||
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -91,9 +89,6 @@ public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose
|
|||
if (StorageAdapter.class.equals(clazz)) {
|
||||
return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY);
|
||||
}
|
||||
if (WireTransferable.class.equals(clazz)) {
|
||||
return (T) this;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -91,7 +91,7 @@ public class ColumnHolderRACColumn implements Column, Closeable
|
|||
public boolean isNull(int rowNum)
|
||||
{
|
||||
offset.set(rowNum);
|
||||
return valueSelector.isNull();
|
||||
return valueSelector.getObject() == null;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -19,10 +19,11 @@
|
|||
|
||||
package org.apache.druid.query.rowsandcols.concrete;
|
||||
|
||||
import org.apache.druid.common.semantic.SemanticCreator;
|
||||
import org.apache.druid.common.semantic.SemanticUtils;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.SemanticCreator;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.segment.CloseableShapeshifter;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
|
@ -41,7 +42,7 @@ import java.util.function.Function;
|
|||
|
||||
public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter
|
||||
{
|
||||
private static final Map<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> AS_MAP = RowsAndColumns
|
||||
private static final Map<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> AS_MAP = SemanticUtils
|
||||
.makeAsMap(QueryableIndexRowsAndColumns.class);
|
||||
|
||||
private final QueryableIndex index;
|
||||
|
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.semantic;
|
||||
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
|
||||
import org.apache.druid.frame.write.FrameWriter;
|
||||
import org.apache.druid.frame.write.FrameWriters;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class DefaultFrameMaker implements FrameMaker
|
||||
{
|
||||
private final RowsAndColumns rac;
|
||||
|
||||
public DefaultFrameMaker(RowsAndColumns rac)
|
||||
{
|
||||
this.rac = rac;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RowSignature computeSignature()
|
||||
{
|
||||
final RowSignature.Builder signatureBuilder = RowSignature.builder();
|
||||
for (String column : rac.getColumnNames()) {
|
||||
final Column racColumn = rac.findColumn(column);
|
||||
if (racColumn == null) {
|
||||
continue;
|
||||
}
|
||||
signatureBuilder.add(column, racColumn.toAccessor().getType());
|
||||
}
|
||||
|
||||
return signatureBuilder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Frame toColumnBasedFrame()
|
||||
{
|
||||
final AtomicInteger rowId = new AtomicInteger(0);
|
||||
final int numRows = rac.numRows();
|
||||
final ColumnSelectorFactoryMaker csfm = ColumnSelectorFactoryMaker.fromRAC(rac);
|
||||
final ColumnSelectorFactory selectorFactory = csfm.make(rowId);
|
||||
|
||||
final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(200 << 20); // 200 MB
|
||||
|
||||
final FrameWriter frameWriter = FrameWriters.makeColumnBasedFrameWriterFactory(
|
||||
memFactory,
|
||||
computeSignature(),
|
||||
Collections.emptyList()
|
||||
).newFrameWriter(selectorFactory);
|
||||
|
||||
rowId.set(0);
|
||||
for (; rowId.get() < numRows; rowId.incrementAndGet()) {
|
||||
frameWriter.addSelection();
|
||||
}
|
||||
|
||||
return Frame.wrap(frameWriter.toByteArray());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.semantic;
|
||||
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
public interface FrameMaker
|
||||
{
|
||||
static FrameMaker fromRAC(RowsAndColumns rac)
|
||||
{
|
||||
FrameMaker retVal = rac.as(FrameMaker.class);
|
||||
if (retVal == null) {
|
||||
retVal = new DefaultFrameMaker(rac);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
RowSignature computeSignature();
|
||||
|
||||
Frame toColumnBasedFrame();
|
||||
}
|
|
@ -22,7 +22,7 @@ package org.apache.druid.segment;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.PeekingIterator;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.column.ColumnDescriptor;
|
||||
|
@ -212,7 +212,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
);
|
||||
break;
|
||||
default:
|
||||
throw new ISE(
|
||||
throw DruidException.defensive(
|
||||
"How did we get here? Column [%s] with type [%s] does not have specialized serializer",
|
||||
name,
|
||||
logicalType
|
||||
|
@ -349,7 +349,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
@Override
|
||||
public ColumnDescriptor makeColumnDescriptor()
|
||||
{
|
||||
ColumnDescriptor.Builder descriptorBuilder = new ColumnDescriptor.Builder();
|
||||
ColumnDescriptor.Builder descriptorBuilder = ColumnDescriptor.builder();
|
||||
|
||||
final NestedCommonFormatColumnPartSerde partSerde =
|
||||
NestedCommonFormatColumnPartSerde.serializerBuilder()
|
||||
|
|
|
@ -510,9 +510,15 @@ public class IndexIO
|
|||
new ConciseBitmapFactory(),
|
||||
columns,
|
||||
index.getFileMapper(),
|
||||
null,
|
||||
lazy
|
||||
);
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private Supplier<ColumnHolder> getColumnHolderSupplier(ColumnBuilder builder, boolean lazy)
|
||||
|
@ -604,25 +610,6 @@ public class IndexIO
|
|||
allDims = null;
|
||||
}
|
||||
|
||||
Metadata metadata = null;
|
||||
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
||||
if (metadataBB != null) {
|
||||
try {
|
||||
metadata = mapper.readValue(
|
||||
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
||||
Metadata.class
|
||||
);
|
||||
}
|
||||
catch (JsonParseException | JsonMappingException ex) {
|
||||
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
|
||||
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
|
||||
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
|
||||
}
|
||||
catch (IOException ex) {
|
||||
throw new IOException("Failed to read metadata", ex);
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Supplier<ColumnHolder>> columns = new LinkedHashMap<>();
|
||||
|
||||
// Register the time column
|
||||
|
@ -663,9 +650,32 @@ public class IndexIO
|
|||
segmentBitmapSerdeFactory.getBitmapFactory(),
|
||||
columns,
|
||||
smooshedFiles,
|
||||
metadata,
|
||||
lazy
|
||||
);
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
try {
|
||||
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
||||
if (metadataBB != null) {
|
||||
return mapper.readValue(
|
||||
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
||||
Metadata.class
|
||||
);
|
||||
}
|
||||
}
|
||||
catch (JsonParseException | JsonMappingException ex) {
|
||||
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
|
||||
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
|
||||
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
|
||||
}
|
||||
catch (IOException ex) {
|
||||
log.warn(ex, "Failed to read metadata for segment [%s]", inDir);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
|
||||
|
||||
|
|
|
@ -19,17 +19,24 @@
|
|||
|
||||
package org.apache.druid.segment;
|
||||
|
||||
import org.apache.druid.common.semantic.SemanticCreator;
|
||||
import org.apache.druid.common.semantic.SemanticUtils;
|
||||
import org.apache.druid.query.rowsandcols.concrete.QueryableIndexRowsAndColumns;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class QueryableIndexSegment implements Segment
|
||||
{
|
||||
private static final Map<Class<?>, Function<QueryableIndexSegment, ?>> AS_MAP = SemanticUtils
|
||||
.makeAsMap(QueryableIndexSegment.class);
|
||||
|
||||
private final QueryableIndex index;
|
||||
private final QueryableIndexStorageAdapter storageAdapter;
|
||||
private final SegmentId segmentId;
|
||||
|
@ -77,10 +84,18 @@ public class QueryableIndexSegment implements Segment
|
|||
@Override
|
||||
public <T> T as(@Nonnull Class<T> clazz)
|
||||
{
|
||||
if (CloseableShapeshifter.class.equals(clazz)) {
|
||||
return (T) new QueryableIndexRowsAndColumns(index);
|
||||
final Function<QueryableIndexSegment, ?> fn = AS_MAP.get(clazz);
|
||||
if (fn != null) {
|
||||
return (T) fn.apply(this);
|
||||
}
|
||||
|
||||
return Segment.super.as(clazz);
|
||||
}
|
||||
|
||||
@SemanticCreator
|
||||
@SuppressWarnings("unused")
|
||||
public CloseableShapeshifter toCloseableShapeshifter()
|
||||
{
|
||||
return new QueryableIndexRowsAndColumns(index);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ import java.util.Map;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
public class SimpleQueryableIndex implements QueryableIndex
|
||||
public abstract class SimpleQueryableIndex implements QueryableIndex
|
||||
{
|
||||
private final Interval dataInterval;
|
||||
private final List<String> columnNames;
|
||||
|
@ -46,8 +46,6 @@ public class SimpleQueryableIndex implements QueryableIndex
|
|||
private final BitmapFactory bitmapFactory;
|
||||
private final Map<String, Supplier<ColumnHolder>> columns;
|
||||
private final SmooshedFileMapper fileMapper;
|
||||
@Nullable
|
||||
private final Metadata metadata;
|
||||
private final Supplier<Map<String, DimensionHandler>> dimensionHandlers;
|
||||
|
||||
public SimpleQueryableIndex(
|
||||
|
@ -56,7 +54,6 @@ public class SimpleQueryableIndex implements QueryableIndex
|
|||
BitmapFactory bitmapFactory,
|
||||
Map<String, Supplier<ColumnHolder>> columns,
|
||||
SmooshedFileMapper fileMapper,
|
||||
@Nullable Metadata metadata,
|
||||
boolean lazy
|
||||
)
|
||||
{
|
||||
|
@ -73,7 +70,6 @@ public class SimpleQueryableIndex implements QueryableIndex
|
|||
this.bitmapFactory = bitmapFactory;
|
||||
this.columns = columns;
|
||||
this.fileMapper = fileMapper;
|
||||
this.metadata = metadata;
|
||||
|
||||
if (lazy) {
|
||||
this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions));
|
||||
|
@ -141,10 +137,7 @@ public class SimpleQueryableIndex implements QueryableIndex
|
|||
}
|
||||
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
return metadata;
|
||||
}
|
||||
public abstract Metadata getMetadata();
|
||||
|
||||
@Override
|
||||
public Map<String, DimensionHandler> getDimensionHandlers()
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.segment.vector.ReadableVectorOffset;
|
|||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
|
||||
public interface BaseColumn extends Closeable
|
||||
|
@ -41,4 +42,11 @@ public interface BaseColumn extends Closeable
|
|||
{
|
||||
throw new UOE("Cannot make VectorObjectSelector for column with class[%s]", getClass().getName());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
@Nullable
|
||||
default <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,8 @@ import org.apache.druid.segment.data.ReadableOffset;
|
|||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongsColumn implements NumericColumn
|
||||
|
@ -75,6 +77,13 @@ public class LongsColumn implements NumericColumn
|
|||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return column.as(clazz);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
|
|
|
@ -528,11 +528,11 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
/**
|
||||
* Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* <p>
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringSingleValueDimensionVectorSelector
|
||||
implements SingleValueDimensionVectorSelector, IdLookup
|
||||
implements SingleValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarInts column;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
@ -540,8 +540,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringSingleValueDimensionVectorSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
|
@ -601,11 +601,11 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
/**
|
||||
* Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* <p>
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringMultiValueDimensionVectorSelector
|
||||
implements MultiValueDimensionVectorSelector, IdLookup
|
||||
implements MultiValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
@ -614,8 +614,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringMultiValueDimensionVectorSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
|
@ -670,6 +670,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
|
@ -697,8 +698,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringVectorObjectSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
|
@ -757,8 +758,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public MultiValueStringVectorObjectSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
|
|
|
@ -20,7 +20,10 @@
|
|||
package org.apache.druid.segment.data;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ConciseBitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import org.apache.druid.error.DruidException;
|
||||
|
||||
public class BitmapSerde
|
||||
{
|
||||
|
@ -48,4 +51,14 @@ public class BitmapSerde
|
|||
{
|
||||
return new LegacyBitmapSerdeFactory();
|
||||
}
|
||||
|
||||
public static BitmapSerdeFactory forBitmapFactory(BitmapFactory factory)
|
||||
{
|
||||
if (factory instanceof RoaringBitmapFactory) {
|
||||
return new DefaultBitmapSerdeFactory();
|
||||
} else if (factory instanceof ConciseBitmapFactory) {
|
||||
return new ConciseBitmapSerdeFactory();
|
||||
}
|
||||
throw DruidException.defensive("Unknown type of bitmapFactory [%s]", factory.getClass());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,7 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier<ColumnarLongs>
|
|||
CompressionStrategy strategy
|
||||
)
|
||||
{
|
||||
baseLongBuffers = GenericIndexed.read(fromBuffer, DecompressingByteBufferObjectStrategy.of(order, strategy));
|
||||
this.baseLongBuffers = GenericIndexed.read(fromBuffer, DecompressingByteBufferObjectStrategy.of(order, strategy));
|
||||
this.totalSize = totalSize;
|
||||
this.sizePer = sizePer;
|
||||
this.baseReader = reader;
|
||||
|
@ -156,6 +156,12 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier<ColumnarLongs>
|
|||
|
||||
@Override
|
||||
public void get(final long[] out, final int start, final int length)
|
||||
{
|
||||
get(out, 0, start, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void get(long[] out, int offset, int start, int length)
|
||||
{
|
||||
// division + remainder is optimized by the compiler so keep those together
|
||||
int bufferNum = start / sizePer;
|
||||
|
@ -169,7 +175,7 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier<ColumnarLongs>
|
|||
}
|
||||
|
||||
final int limit = Math.min(length - p, sizePer - bufferIndex);
|
||||
reader.read(out, p, bufferIndex, limit);
|
||||
reader.read(out, offset + p, bufferIndex, limit);
|
||||
p += limit;
|
||||
bufferNum++;
|
||||
bufferIndex = 0;
|
||||
|
|
|
@ -28,4 +28,10 @@ import java.io.Closeable;
|
|||
*/
|
||||
public interface ColumnarInts extends IndexedInts, Closeable
|
||||
{
|
||||
default void get(int[] out, int offset, int start, int length)
|
||||
{
|
||||
for (int i = 0; i < length; i++) {
|
||||
out[offset + i] = get(i + start);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,9 +46,14 @@ public interface ColumnarLongs extends Closeable
|
|||
long get(int index);
|
||||
|
||||
default void get(long[] out, int start, int length)
|
||||
{
|
||||
get(out, 0, start, length);
|
||||
}
|
||||
|
||||
default void get(long[] out, int offset, int start, int length)
|
||||
{
|
||||
for (int i = 0; i < length; i++) {
|
||||
out[i] = get(i + start);
|
||||
out[offset + i] = get(i + start);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -62,6 +67,12 @@ public interface ColumnarLongs extends Closeable
|
|||
@Override
|
||||
void close();
|
||||
|
||||
@Nullable
|
||||
default <T> T as(Class<? extends T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
default ColumnValueSelector<Long> makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap)
|
||||
{
|
||||
if (nullValueBitmap.isEmpty()) {
|
||||
|
|
|
@ -29,6 +29,15 @@ import java.io.IOException;
|
|||
public interface ColumnarLongsSerializer extends Serializer
|
||||
{
|
||||
void open() throws IOException;
|
||||
|
||||
int size();
|
||||
|
||||
void add(long value) throws IOException;
|
||||
|
||||
default void addAll(long[] values, int start, int end) throws IOException
|
||||
{
|
||||
for (int i = start; i < end; ++i) {
|
||||
add(values[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -233,6 +233,14 @@ public class CompressionFactory
|
|||
|
||||
void write(long value) throws IOException;
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
default void write(long[] values, int offset, int length) throws IOException
|
||||
{
|
||||
for (int i = offset; i < length; ++i) {
|
||||
write(values[i]);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Flush the unwritten content to the current output.
|
||||
*/
|
||||
|
@ -294,6 +302,9 @@ public class CompressionFactory
|
|||
* various duplicates.
|
||||
*/
|
||||
LongEncodingReader duplicate();
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
LongEncodingStrategy getStrategy();
|
||||
}
|
||||
|
||||
public static Supplier<ColumnarLongs> getLongSupplier(
|
||||
|
|
|
@ -82,4 +82,10 @@ public class DeltaLongEncodingReader implements CompressionFactory.LongEncodingR
|
|||
{
|
||||
return new DeltaLongEncodingReader(buffer.duplicate(), base, bitsPerValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompressionFactory.LongEncodingStrategy getStrategy()
|
||||
{
|
||||
return CompressionFactory.LongEncodingStrategy.AUTO;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,12 +28,41 @@ public interface DictionaryWriter<T> extends Serializer
|
|||
{
|
||||
boolean isSorted();
|
||||
|
||||
/**
|
||||
* Prepares the writer for writing
|
||||
*
|
||||
* @throws IOException if there is a problem with IO
|
||||
*/
|
||||
void open() throws IOException;
|
||||
|
||||
void write(@Nullable T objectToWrite) throws IOException;
|
||||
/**
|
||||
* Writes an object to the dictionary.
|
||||
* <p>
|
||||
* Returns the index of the value that was just written. This is defined as the `int` value that can be passed
|
||||
* into {@link #get} such that it will return the same value back.
|
||||
*
|
||||
* @param objectToWrite object to be written to the dictionary
|
||||
* @return index of the value that was just written
|
||||
* @throws IOException if there is a problem with IO
|
||||
*/
|
||||
int write(@Nullable T objectToWrite) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns an object that has already been written via the {@link #write} method.
|
||||
*
|
||||
* @param dictId index of the object to return
|
||||
* @return the object identified by the given index
|
||||
* @throws IOException if there is a problem with IO
|
||||
*/
|
||||
@Nullable
|
||||
T get(int dictId) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns the number of items that have been written so far in this dictionary. Any number lower than this
|
||||
* cardinality can be passed into {@link #get} and a value will be returned. If a value greater than or equal to
|
||||
* the cardinality is passed into {@link #get} all sorts of things could happen, but likely none of them are good.
|
||||
*
|
||||
* @return the number of items that have been written so far
|
||||
*/
|
||||
int getCardinality();
|
||||
}
|
||||
|
|
|
@ -58,9 +58,9 @@ public class EncodedStringDictionaryWriter implements DictionaryWriter<String>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(@Nullable String objectToWrite) throws IOException
|
||||
public int write(@Nullable String objectToWrite) throws IOException
|
||||
{
|
||||
delegate.write(StringUtils.toUtf8Nullable(NullHandling.emptyToNullIfNeeded(objectToWrite)));
|
||||
return delegate.write(StringUtils.toUtf8Nullable(NullHandling.emptyToNullIfNeeded(objectToWrite)));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -20,8 +20,8 @@
|
|||
package org.apache.druid.segment.data;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.io.Channels;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||
import org.apache.druid.segment.column.TypeStrategy;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||
|
@ -46,14 +46,16 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
private final Comparator<T> comparator;
|
||||
private final ByteBuffer scratch;
|
||||
private final ByteBuffer readBuffer;
|
||||
private int numWritten;
|
||||
private final boolean isSorted;
|
||||
private final int width;
|
||||
|
||||
private int cardinality = 0;
|
||||
|
||||
@Nullable
|
||||
private WriteOutBytes valuesOut = null;
|
||||
private boolean hasNulls = false;
|
||||
private boolean isSorted;
|
||||
@Nullable
|
||||
private T prevObject = null;
|
||||
private final int width;
|
||||
|
||||
public FixedIndexedWriter(
|
||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||
|
@ -87,7 +89,7 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return hasNulls ? numWritten + 1 : numWritten;
|
||||
return cardinality;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -97,28 +99,31 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(@Nullable T objectToWrite) throws IOException
|
||||
public int write(@Nullable T objectToWrite) throws IOException
|
||||
{
|
||||
if (prevObject != null && isSorted && comparator.compare(prevObject, objectToWrite) >= 0) {
|
||||
throw new ISE(
|
||||
throw DruidException.defensive(
|
||||
"Values must be sorted and unique. Element [%s] with value [%s] is before or equivalent to [%s]",
|
||||
numWritten,
|
||||
cardinality,
|
||||
objectToWrite,
|
||||
prevObject
|
||||
);
|
||||
}
|
||||
|
||||
if (objectToWrite == null) {
|
||||
if (cardinality != 0) {
|
||||
throw DruidException.defensive("Null must come first, got it at cardinality[%,d]!=0", cardinality);
|
||||
}
|
||||
hasNulls = true;
|
||||
return;
|
||||
return cardinality++;
|
||||
}
|
||||
|
||||
scratch.clear();
|
||||
typeStrategy.write(scratch, objectToWrite, width);
|
||||
scratch.flip();
|
||||
Channels.writeFully(valuesOut, scratch);
|
||||
numWritten++;
|
||||
prevObject = objectToWrite;
|
||||
return cardinality++;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -141,7 +146,7 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
scratch.flip();
|
||||
Channels.writeFully(channel, scratch);
|
||||
scratch.clear();
|
||||
scratch.putInt(numWritten);
|
||||
scratch.putInt(hasNulls ? cardinality - 1 : cardinality); // we don't actually write the null entry, so subtract 1
|
||||
scratch.flip();
|
||||
Channels.writeFully(channel, scratch);
|
||||
valuesOut.writeTo(channel);
|
||||
|
@ -166,7 +171,7 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
public Iterator<T> getIterator()
|
||||
{
|
||||
final ByteBuffer iteratorBuffer = ByteBuffer.allocate(width * PAGE_SIZE).order(readBuffer.order());
|
||||
final int totalCount = hasNulls ? 1 + numWritten : numWritten;
|
||||
final int totalCount = cardinality;
|
||||
|
||||
final int startPos = hasNulls ? 1 : 0;
|
||||
return new Iterator<T>()
|
||||
|
@ -197,13 +202,8 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
|||
{
|
||||
iteratorBuffer.clear();
|
||||
try {
|
||||
if (numWritten - (pos - startPos) < PAGE_SIZE) {
|
||||
int size = (numWritten - (pos - startPos)) * width;
|
||||
iteratorBuffer.limit(size);
|
||||
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
|
||||
} else {
|
||||
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
|
||||
}
|
||||
iteratorBuffer.limit(Math.min(PAGE_SIZE, (cardinality - pos) * width));
|
||||
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
|
||||
iteratorBuffer.clear();
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.data;
|
|||
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.io.Channels;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -102,7 +103,7 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(@Nullable byte[] value) throws IOException
|
||||
public int write(@Nullable byte[] value) throws IOException
|
||||
{
|
||||
if (prevObject != null && compareNullableUtf8UsingJavaStringOrdering(prevObject, value) >= 0) {
|
||||
throw new ISE(
|
||||
|
@ -114,8 +115,11 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
|
|||
}
|
||||
|
||||
if (value == null) {
|
||||
if (numWritten != 0) {
|
||||
throw DruidException.defensive("Null must come first, got it at cardinality[%,d]!=0", numWritten);
|
||||
}
|
||||
hasNulls = true;
|
||||
return;
|
||||
return 0;
|
||||
}
|
||||
|
||||
// if the bucket buffer is full, write the bucket
|
||||
|
@ -143,8 +147,9 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
|
|||
|
||||
bucketBuffer[numWritten % bucketSize] = value;
|
||||
|
||||
++numWritten;
|
||||
int retVal = numWritten++;
|
||||
prevObject = value;
|
||||
return retVal + (hasNulls ? 1 : 0);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.data;
|
|||
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.io.Channels;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -81,6 +82,10 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
private boolean isClosed = false;
|
||||
private boolean hasNulls = false;
|
||||
|
||||
private int readCachedBucket = -1;
|
||||
@Nullable
|
||||
private ByteBuffer readBufferCache = null;
|
||||
|
||||
public FrontCodedIntArrayIndexedWriter(
|
||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||
ByteOrder byteOrder,
|
||||
|
@ -107,7 +112,7 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(@Nullable int[] value) throws IOException
|
||||
public int write(@Nullable int[] value) throws IOException
|
||||
{
|
||||
|
||||
if (prevObject != null && ARRAY_COMPARATOR.compare(prevObject, value) >= 0) {
|
||||
|
@ -120,8 +125,11 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
}
|
||||
|
||||
if (value == null) {
|
||||
if (numWritten != 0) {
|
||||
throw DruidException.defensive("Null must come first, got it at numWritten[%,d]!=0", numWritten);
|
||||
}
|
||||
hasNulls = true;
|
||||
return;
|
||||
return 0;
|
||||
}
|
||||
|
||||
// if the bucket buffer is full, write the bucket
|
||||
|
@ -147,8 +155,9 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
|
||||
bucketBuffer[numWritten % bucketSize] = value;
|
||||
|
||||
++numWritten;
|
||||
int retVal = numWritten++;
|
||||
prevObject = value;
|
||||
return retVal + (hasNulls ? 1 : 0);
|
||||
}
|
||||
|
||||
|
||||
|
@ -206,6 +215,11 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
return bucketBuffer[relativeIndex];
|
||||
} else {
|
||||
final int bucket = adjustedIndex >> div;
|
||||
if (readCachedBucket == bucket) {
|
||||
readBufferCache.position(0);
|
||||
return getFromBucket(readBufferCache, relativeIndex);
|
||||
}
|
||||
|
||||
long startOffset;
|
||||
if (bucket == 0) {
|
||||
startOffset = 0;
|
||||
|
@ -217,10 +231,17 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
|||
if (currentBucketSize == 0) {
|
||||
return null;
|
||||
}
|
||||
final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder);
|
||||
valuesOut.readFully(startOffset, bucketBuffer);
|
||||
bucketBuffer.clear();
|
||||
return getFromBucket(bucketBuffer, relativeIndex);
|
||||
if (readBufferCache == null || readBufferCache.capacity() < currentBucketSize) {
|
||||
readBufferCache = ByteBuffer.allocate(currentBucketSize).order(byteOrder);
|
||||
}
|
||||
readBufferCache.clear();
|
||||
readBufferCache.limit(currentBucketSize);
|
||||
valuesOut.readFully(startOffset, readBufferCache);
|
||||
|
||||
readCachedBucket = bucket;
|
||||
|
||||
readBufferCache.position(0);
|
||||
return getFromBucket(readBufferCache, relativeIndex);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -242,7 +242,7 @@ public class GenericIndexedWriter<T> implements DictionaryWriter<T>
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(@Nullable T objectToWrite) throws IOException
|
||||
public int write(@Nullable T objectToWrite) throws IOException
|
||||
{
|
||||
if (objectsSorted && prevObject != null && strategy.compare(prevObject, objectToWrite) >= 0) {
|
||||
objectsSorted = false;
|
||||
|
@ -263,7 +263,7 @@ public class GenericIndexedWriter<T> implements DictionaryWriter<T>
|
|||
|
||||
// Increment number of values written. Important to do this after the check above, since numWritten is
|
||||
// accessed during "initializeHeaderOutLong" to determine the length of the header.
|
||||
++numWritten;
|
||||
int retVal = numWritten++;
|
||||
|
||||
if (!requireMultipleFiles) {
|
||||
headerOut.writeInt(checkedCastNonnegativeLongToInt(valuesOut.size()));
|
||||
|
@ -280,6 +280,7 @@ public class GenericIndexedWriter<T> implements DictionaryWriter<T>
|
|||
if (objectsSorted) {
|
||||
prevObject = objectToWrite;
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -71,4 +71,10 @@ public class LongsLongEncodingReader implements CompressionFactory.LongEncodingR
|
|||
{
|
||||
return new LongsLongEncodingReader(buffer.getByteBuffer(), buffer.getTypeByteOrder());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompressionFactory.LongEncodingStrategy getStrategy()
|
||||
{
|
||||
return CompressionFactory.LongEncodingStrategy.LONGS;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.segment.data;
|
|||
import org.apache.druid.segment.writeout.WriteOutBytes;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
|
|
@ -84,7 +84,7 @@ public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory
|
|||
@Override
|
||||
public byte[] toBytes(@Nullable ImmutableBitmap val)
|
||||
{
|
||||
if (val == null || val.size() == 0) {
|
||||
if (val == null || val.isEmpty()) {
|
||||
return new byte[]{};
|
||||
}
|
||||
return val.toBytes();
|
||||
|
|
|
@ -88,4 +88,10 @@ public class TableLongEncodingReader implements CompressionFactory.LongEncodingR
|
|||
{
|
||||
return new TableLongEncodingReader(buffer.duplicate(), table, bitsPerValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompressionFactory.LongEncodingStrategy getStrategy()
|
||||
{
|
||||
return CompressionFactory.LongEncodingStrategy.AUTO;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -452,7 +452,10 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
@Override
|
||||
public int getLength()
|
||||
{
|
||||
return -1;
|
||||
if (compressedRawColumn == null) {
|
||||
compressedRawColumn = closer.register(compressedRawColumnSupplier.get());
|
||||
}
|
||||
return compressedRawColumn.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -535,9 +538,14 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
if (arrayFieldIndex >= 0) {
|
||||
final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
}
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(
|
||||
arrayField,
|
||||
arrayFieldIndex
|
||||
).getColumn();
|
||||
ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset);
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
|
@ -634,9 +642,14 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
if (arrayFieldIndex >= 0) {
|
||||
final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
}
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(
|
||||
arrayField,
|
||||
arrayFieldIndex
|
||||
).getColumn();
|
||||
VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
|
||||
|
||||
return new VectorObjectSelector()
|
||||
|
@ -702,9 +715,14 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
if (arrayFieldIndex >= 0) {
|
||||
final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
throw DruidException.forPersona(DruidException.Persona.USER)
|
||||
.ofCategory(DruidException.Category.INVALID_INPUT)
|
||||
.build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
|
||||
}
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField, arrayFieldIndex).getColumn();
|
||||
DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(
|
||||
arrayField,
|
||||
arrayFieldIndex
|
||||
).getColumn();
|
||||
VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
|
||||
|
||||
return new VectorValueSelector()
|
||||
|
|
|
@ -51,7 +51,7 @@ import java.util.EnumSet;
|
|||
|
||||
/**
|
||||
* Value to dictionary id lookup, backed with memory mapped dictionaries populated lazily by the supplied
|
||||
* @link DictionaryWriter}.
|
||||
* {@link DictionaryWriter}.
|
||||
*/
|
||||
public final class DictionaryIdLookup implements Closeable
|
||||
{
|
||||
|
|
|
@ -112,7 +112,11 @@ public final class MetaSerdeHelper<T>
|
|||
|
||||
public int size(T x)
|
||||
{
|
||||
return fieldWriters.stream().mapToInt(w -> w.size(x)).sum();
|
||||
int retVal = 0;
|
||||
for (FieldWriter<T> fieldWriter : fieldWriters) {
|
||||
retVal += fieldWriter.size(x);
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public interface FieldWriter<T>
|
||||
|
|
|
@ -22,6 +22,12 @@ package org.apache.druid.segment.serde.cell;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An Iterator-like interface that is intentionally not extending Iterator. This is because it is Closeable
|
||||
* and we never want to lose track of the fact that the object needs to be closed.
|
||||
*
|
||||
* @param <T>
|
||||
*/
|
||||
public interface IOIterator<T> extends Closeable
|
||||
{
|
||||
boolean hasNext() throws IOException;
|
||||
|
|
|
@ -17,10 +17,9 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.semantic;
|
||||
package org.apache.druid.common.semantic;
|
||||
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.rowsandcols.SemanticCreator;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -80,7 +79,7 @@ public class SemanticCreatorUsageTest
|
|||
|
||||
/**
|
||||
* {@link SemanticCreator} must return with an interface.
|
||||
*
|
||||
* <p>
|
||||
* An exact implementation may indicate that some interface methods might be missing.
|
||||
*/
|
||||
@Test
|
||||
|
@ -95,7 +94,7 @@ public class SemanticCreatorUsageTest
|
|||
|
||||
/**
|
||||
* {@link SemanticCreator} method names must follow the naming pattern toReturnType().
|
||||
*
|
||||
* <p>
|
||||
* For example: a method returning with a type of Ball should be named as "toBall"
|
||||
*/
|
||||
@Test
|
|
@ -0,0 +1,137 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.common.semantic;
|
||||
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.segment.CloseableShapeshifter;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
|
||||
public class SemanticUtilsTest
|
||||
{
|
||||
@Test
|
||||
public void testInvalidParameters()
|
||||
{
|
||||
Assert.assertThrows(
|
||||
DruidException.class,
|
||||
() -> SemanticUtils.makeAsMap(InvalidShapeshifter.class)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidParameters()
|
||||
{
|
||||
TestShapeshifter testShapeshifter = new TestShapeshifter();
|
||||
Assert.assertTrue(testShapeshifter.as(A.class) instanceof A);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverrideForNewMapping()
|
||||
{
|
||||
SemanticUtils.registerAsOverride(
|
||||
TestShapeshifter.class,
|
||||
OverrideClass.class,
|
||||
(testShapeshifter) -> new OverrideClass()
|
||||
);
|
||||
TestShapeshifter testShapeshifter = new TestShapeshifter();
|
||||
Assert.assertTrue(testShapeshifter.as(A.class) instanceof A);
|
||||
Assert.assertTrue(testShapeshifter.as(OverrideClass.class) instanceof OverrideClass);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverrideForExistingMapping()
|
||||
{
|
||||
SemanticUtils.registerAsOverride(
|
||||
TestShapeshifter.class,
|
||||
A.class,
|
||||
(testShapeshifter) -> new OverrideClass()
|
||||
);
|
||||
TestShapeshifter testShapeshifter = new TestShapeshifter();
|
||||
Assert.assertTrue(testShapeshifter.as(A.class) instanceof OverrideClass);
|
||||
}
|
||||
|
||||
static class TestShapeshifter implements CloseableShapeshifter
|
||||
{
|
||||
private final Map<Class<?>, Function<TestShapeshifter, ?>> asMap;
|
||||
|
||||
public TestShapeshifter()
|
||||
{
|
||||
this.asMap = SemanticUtils.makeAsMap(TestShapeshifter.class);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
@Nullable
|
||||
public <T> T as(@Nonnull Class<T> clazz)
|
||||
{
|
||||
//noinspection ReturnOfNull
|
||||
return (T) asMap.getOrDefault(clazz, arg -> null).apply(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
|
||||
@SemanticCreator
|
||||
public AInterface toAInterface()
|
||||
{
|
||||
return new A();
|
||||
}
|
||||
}
|
||||
|
||||
static class InvalidShapeshifter implements CloseableShapeshifter
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public <T> T as(@Nonnull Class<T> clazz)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
}
|
||||
|
||||
@SemanticCreator
|
||||
public AInterface toAInterface(String invalidParameter)
|
||||
{
|
||||
return new A();
|
||||
}
|
||||
}
|
||||
|
||||
interface AInterface
|
||||
{
|
||||
}
|
||||
|
||||
static class A implements AInterface
|
||||
{
|
||||
}
|
||||
|
||||
static class OverrideClass extends A
|
||||
{
|
||||
}
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.error;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class ExceptionTest
|
||||
{
|
||||
@Test
|
||||
public void testNoCause()
|
||||
{
|
||||
DruidException exception = DruidException.defensive().build("defensive");
|
||||
StackTraceElement[] stackTrace = exception.getStackTrace();
|
||||
for (StackTraceElement stackTraceElement : stackTrace) {
|
||||
Assert.assertFalse(stackTraceElement.getClassName().startsWith(DruidException.CLASS_NAME_STR));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoStacktrace()
|
||||
{
|
||||
ErrorResponse errorResponse = new ErrorResponse(Forbidden.exception());
|
||||
final Map<String, Object> asMap = errorResponse.getAsMap();
|
||||
DruidException exception = ErrorResponse.fromMap(asMap).getUnderlyingException();
|
||||
Assert.assertTrue(exception.getCause() instanceof DruidException);
|
||||
Assert.assertEquals(0, exception.getCause().getStackTrace().length);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,82 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.column;
|
||||
|
||||
import org.apache.druid.query.rowsandcols.util.FindResult;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class LongArrayColumnTest
|
||||
{
|
||||
@Test
|
||||
public void testLongArrayColumnWithLongValues()
|
||||
{
|
||||
Column column = new LongArrayColumn(new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9});
|
||||
ColumnAccessor accessor = column.toAccessor();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
Assert.assertFalse(accessor.isNull(i));
|
||||
Assert.assertEquals(i, accessor.getLong(i));
|
||||
Assert.assertEquals((long) i, accessor.getObject(i));
|
||||
Assert.assertEquals(i, accessor.getDouble(i), 0);
|
||||
Assert.assertEquals(i, accessor.getInt(i));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFindLong()
|
||||
{
|
||||
Column column = new LongArrayColumn(new long[] {1, 1, 1, 3, 5, 5, 6, 7, 8, 9});
|
||||
BinarySearchableAccessor accessor = (BinarySearchableAccessor) column.toAccessor();
|
||||
|
||||
FindResult findResult = accessor.findLong(0, accessor.numRows(), 1);
|
||||
Assert.assertTrue(findResult.wasFound());
|
||||
Assert.assertEquals(0, findResult.getStartRow());
|
||||
Assert.assertEquals(3, findResult.getEndRow());
|
||||
|
||||
findResult = accessor.findLong(0, accessor.numRows(), 6);
|
||||
Assert.assertTrue(findResult.wasFound());
|
||||
Assert.assertEquals(6, findResult.getStartRow());
|
||||
Assert.assertEquals(7, findResult.getEndRow());
|
||||
|
||||
Assert.assertFalse(accessor.findLong(0, accessor.numRows(), 2).wasFound());
|
||||
Assert.assertFalse(accessor.findLong(0, 3, 9).wasFound());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOtherTypeFinds()
|
||||
{
|
||||
Column column = new LongArrayColumn(new long[] {0, 1, 2, 3, 4, 5, Long.MAX_VALUE});
|
||||
BinarySearchableAccessor accessor = (BinarySearchableAccessor) column.toAccessor();
|
||||
|
||||
FindResult findResult = accessor.findNull(0, accessor.numRows());
|
||||
Assert.assertFalse(findResult.wasFound()); // Always false for long array columns
|
||||
|
||||
findResult = accessor.findDouble(0, accessor.numRows(), 3.0);
|
||||
Assert.assertTrue(findResult.wasFound());
|
||||
Assert.assertEquals(3, findResult.getStartRow());
|
||||
Assert.assertEquals(4, findResult.getEndRow());
|
||||
|
||||
findResult = accessor.findFloat(0, accessor.numRows(), 1.0f);
|
||||
Assert.assertTrue(findResult.wasFound());
|
||||
Assert.assertEquals(1, findResult.getStartRow());
|
||||
Assert.assertEquals(2, findResult.getEndRow());
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
|
|||
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.LongArrayColumn;
|
||||
import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.junit.Assert;
|
||||
|
@ -48,7 +49,7 @@ public class AppendableRowsAndColumnsTest extends SemanticTestBase
|
|||
RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(
|
||||
ImmutableMap.of(
|
||||
"colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}),
|
||||
"colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
"colB", new LongArrayColumn(new long[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
)
|
||||
));
|
||||
|
||||
|
@ -58,7 +59,7 @@ public class AppendableRowsAndColumnsTest extends SemanticTestBase
|
|||
|
||||
new RowsAndColumnsHelper()
|
||||
.expectColumn("colA", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
.expectColumn("colB", new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
.expectColumn("colB", new long[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0})
|
||||
.expectColumn("newCol", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})
|
||||
.allColumnsRegistered()
|
||||
.validate(appender);
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.rowsandcols.semantic;
|
||||
|
||||
import org.apache.druid.frame.Frame;
|
||||
import org.apache.druid.query.rowsandcols.ArrayListRowsAndColumnsTest;
|
||||
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class FrameMakerTest
|
||||
{
|
||||
public static RowSignature ROW_SIGNATURE = RowSignature.builder()
|
||||
.add("dim1", ColumnType.STRING)
|
||||
.add("dim2", ColumnType.STRING)
|
||||
.add("dim3", ColumnType.STRING)
|
||||
.add("m1", ColumnType.LONG)
|
||||
.add("m2", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
@Test
|
||||
public void testFrameMaker()
|
||||
{
|
||||
final MapOfColumnsRowsAndColumns mapOfColumnsRowsAndColumns = MapOfColumnsRowsAndColumns
|
||||
.builder()
|
||||
.add("dim1", ColumnType.STRING, "a", "b", "c")
|
||||
.add("dim2", ColumnType.STRING, "m", "d", "e")
|
||||
.add("dim3", ColumnType.STRING, "a")
|
||||
.add("m1", ColumnType.LONG, 1L, 2L, 3L)
|
||||
.add("m2", ColumnType.LONG, 52L, 42L)
|
||||
.build();
|
||||
|
||||
final FrameMaker frameMaker = FrameMaker.fromRAC(ArrayListRowsAndColumnsTest.MAKER.apply(mapOfColumnsRowsAndColumns));
|
||||
|
||||
Assert.assertEquals(ROW_SIGNATURE, frameMaker.computeSignature());
|
||||
|
||||
final Frame frame = frameMaker.toColumnBasedFrame();
|
||||
ColumnBasedFrameRowsAndColumns columnBasedFrameRowsAndColumns = new ColumnBasedFrameRowsAndColumns(
|
||||
frame,
|
||||
frameMaker.computeSignature()
|
||||
);
|
||||
for (String columnName : mapOfColumnsRowsAndColumns.getColumnNames()) {
|
||||
ColumnAccessor expectedColumn = mapOfColumnsRowsAndColumns.findColumn(columnName).toAccessor();
|
||||
ColumnAccessor actualColumn = columnBasedFrameRowsAndColumns.findColumn(columnName).toAccessor();
|
||||
|
||||
for (int i = 0; i < expectedColumn.numRows(); i++) {
|
||||
Assert.assertEquals(
|
||||
expectedColumn.getObject(i),
|
||||
actualColumn.getObject(i)
|
||||
);
|
||||
}
|
||||
}
|
||||
Assert.assertEquals(3, frame.numRows());
|
||||
}
|
||||
}
|
|
@ -31,6 +31,7 @@ import com.google.common.io.Files;
|
|||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.error.DruidException;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.io.smoosh.Smoosh;
|
||||
|
@ -184,20 +185,6 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling
|
|||
segmentBitmapSerdeFactory = new BitmapSerde.LegacyBitmapSerdeFactory();
|
||||
}
|
||||
|
||||
Metadata metadata = null;
|
||||
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
||||
if (metadataBB != null) {
|
||||
try {
|
||||
metadata = mapper.readValue(
|
||||
IndexIO.SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
||||
Metadata.class
|
||||
);
|
||||
}
|
||||
catch (IOException ex) {
|
||||
throw new IOException("Failed to read metadata", ex);
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Supplier<ColumnHolder>> columns = new HashMap<>();
|
||||
|
||||
for (String columnName : cols) {
|
||||
|
@ -251,9 +238,28 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling
|
|||
segmentBitmapSerdeFactory.getBitmapFactory(),
|
||||
columns,
|
||||
smooshedFiles,
|
||||
metadata,
|
||||
lazy
|
||||
);
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
try {
|
||||
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
|
||||
if (metadataBB != null) {
|
||||
return mapper.readValue(
|
||||
IndexIO.SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
|
||||
Metadata.class
|
||||
);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
catch (IOException ex) {
|
||||
throw DruidException.defensive(ex, "Failed to read metadata");
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
return index;
|
||||
}
|
||||
|
|
|
@ -167,11 +167,15 @@ public class IndexMergerLongestSharedDimOrderTest
|
|||
mockBitmapFactory,
|
||||
ImmutableMap.of(ColumnHolder.TIME_COLUMN_NAME, mockSupplier),
|
||||
mockSmooshedFileMapper,
|
||||
null,
|
||||
true
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Metadata getMetadata()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.druid.segment.data;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.collections.bitmap.ConciseBitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -46,4 +48,11 @@ public class BitmapSerdeFactoryTest
|
|||
Assert.assertTrue(mapper.readValue("{\"type\":\"concise\"}", BitmapSerdeFactory.class) instanceof ConciseBitmapSerdeFactory);
|
||||
Assert.assertTrue(mapper.readValue("{\"type\":\"BitmapSerde$SomeRandomClass\"}", BitmapSerdeFactory.class) instanceof RoaringBitmapSerdeFactory);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testForBitmapFactory()
|
||||
{
|
||||
Assert.assertTrue(BitmapSerde.forBitmapFactory(new RoaringBitmapFactory()) instanceof BitmapSerde.DefaultBitmapSerdeFactory);
|
||||
Assert.assertTrue(BitmapSerde.forBitmapFactory(new ConciseBitmapFactory()) instanceof ConciseBitmapSerdeFactory);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import java.nio.ByteBuffer;
|
|||
import java.nio.ByteOrder;
|
||||
import java.nio.IntBuffer;
|
||||
import java.nio.channels.Channels;
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
@ -290,6 +291,11 @@ public class CompressedColumnarIntsSupplierTest extends CompressionStrategyTest
|
|||
indices[i] = i;
|
||||
}
|
||||
|
||||
int[] offsetValues = new int[columnarInts.size() + 1];
|
||||
columnarInts.get(offsetValues, 1, 0, columnarInts.size());
|
||||
Assert.assertEquals(0, offsetValues[0]);
|
||||
Assert.assertArrayEquals(vals, Arrays.copyOfRange(offsetValues, 1, offsetValues.length));
|
||||
|
||||
// random access, limited to 1000 elements for large lists (every element would take too long)
|
||||
IntArrays.shuffle(indices, ThreadLocalRandom.current());
|
||||
final int limit = Math.min(columnarInts.size(), 1000);
|
||||
|
|
|
@ -108,9 +108,7 @@ public class CompressedLongsAutoEncodingSerdeTest
|
|||
);
|
||||
serializer.open();
|
||||
|
||||
for (long value : values) {
|
||||
serializer.add(value);
|
||||
}
|
||||
serializer.addAll(values, 0, values.length);
|
||||
Assert.assertEquals(values.length, serializer.size());
|
||||
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
|
|
@ -186,9 +186,7 @@ public class CompressedLongsSerdeTest
|
|||
);
|
||||
serializer.open();
|
||||
|
||||
for (long value : values) {
|
||||
serializer.add(value);
|
||||
}
|
||||
serializer.addAll(values, 0, values.length);
|
||||
Assert.assertEquals(values.length, serializer.size());
|
||||
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
|
|
|
@ -436,7 +436,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
|
|||
while (sortedStrings.hasNext()) {
|
||||
final String next = sortedStrings.next();
|
||||
final byte[] nextBytes = StringUtils.toUtf8Nullable(next);
|
||||
writer.write(nextBytes);
|
||||
Assert.assertEquals(index, writer.write(nextBytes));
|
||||
if (nextBytes == null) {
|
||||
Assert.assertNull(writer.get(index));
|
||||
} else {
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -102,9 +103,8 @@ public class ClientCompactionRunnerInfo
|
|||
* <ul>
|
||||
* <li>partitionsSpec of type HashedParititionsSpec.</li>
|
||||
* <li>maxTotalRows in DynamicPartitionsSpec.</li>
|
||||
* <li>rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.</li>
|
||||
* <li>queryGranularity set to ALL in granularitySpec.</li>
|
||||
* <li>Each metric has output column name same as the input name.</li>
|
||||
* <li>rollup in granularitySpec set to false when metricsSpec is specified or true when it's empty.</li>
|
||||
* <li>any metric is non-idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A != A.combiningFactory()'.</li>
|
||||
* </ul>
|
||||
*/
|
||||
private static CompactionConfigValidationResult compactionConfigSupportedByMSQEngine(DataSourceCompactionConfig newConfig)
|
||||
|
@ -120,6 +120,7 @@ public class ClientCompactionRunnerInfo
|
|||
));
|
||||
}
|
||||
validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext()));
|
||||
validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec()));
|
||||
return validationResults.stream()
|
||||
.filter(result -> !result.isValid())
|
||||
.findFirst()
|
||||
|
@ -149,17 +150,23 @@ public class ClientCompactionRunnerInfo
|
|||
}
|
||||
|
||||
/**
|
||||
* Validate rollup is set to false in granularitySpec when metricsSpec is specified.
|
||||
* Validate rollup in granularitySpec is set to true when metricsSpec is specified and false if it's null.
|
||||
* If rollup set to null, all existing segments are analyzed, and it's set to true iff all segments have rollup
|
||||
* set to true.
|
||||
*/
|
||||
public static CompactionConfigValidationResult validateRollupForMSQ(
|
||||
AggregatorFactory[] metricsSpec,
|
||||
@Nullable Boolean isRollup
|
||||
)
|
||||
{
|
||||
if (metricsSpec != null && isRollup != null && !isRollup) {
|
||||
if (metricsSpec != null && metricsSpec.length != 0 && isRollup != null && !isRollup) {
|
||||
return CompactionConfigValidationResult.failure(
|
||||
"MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified"
|
||||
);
|
||||
} else if ((metricsSpec == null || metricsSpec.length == 0) && isRollup != null && isRollup) {
|
||||
return CompactionConfigValidationResult.failure(
|
||||
"MSQ: 'granularitySpec.rollup' must be false if 'metricsSpec' is null"
|
||||
);
|
||||
}
|
||||
return CompactionConfigValidationResult.success();
|
||||
}
|
||||
|
@ -181,4 +188,23 @@ public class ClientCompactionRunnerInfo
|
|||
}
|
||||
return CompactionConfigValidationResult.success();
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate each metric is idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A = A.combiningFactory()'.
|
||||
*/
|
||||
public static CompactionConfigValidationResult validateMetricsSpecForMSQ(AggregatorFactory[] metricsSpec)
|
||||
{
|
||||
if (metricsSpec == null) {
|
||||
return CompactionConfigValidationResult.success();
|
||||
}
|
||||
return Arrays.stream(metricsSpec)
|
||||
.filter(aggregatorFactory -> !aggregatorFactory.equals(aggregatorFactory.getCombiningFactory()))
|
||||
.findFirst()
|
||||
.map(aggregatorFactory ->
|
||||
CompactionConfigValidationResult.failure(
|
||||
"MSQ: Non-idempotent aggregator[%s] not supported in 'metricsSpec'.",
|
||||
aggregatorFactory.getName()
|
||||
)
|
||||
).orElse(CompactionConfigValidationResult.success());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ public class ZkEnablementConfig
|
|||
@JsonCreator
|
||||
public ZkEnablementConfig(@JsonProperty("enabled") Boolean enabled)
|
||||
{
|
||||
this.enabled = enabled == null ? true : enabled.booleanValue();
|
||||
this.enabled = enabled == null || enabled;
|
||||
}
|
||||
|
||||
public boolean isEnabled()
|
||||
|
@ -48,6 +48,6 @@ public class ZkEnablementConfig
|
|||
public static boolean isEnabled(Properties properties)
|
||||
{
|
||||
String value = properties.getProperty(PROP_KEY_ENABLED);
|
||||
return value == null ? true : Boolean.parseBoolean(value);
|
||||
return value == null || Boolean.parseBoolean(value);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,10 +41,6 @@ public class ServerViewModule implements Module
|
|||
public static final String SERVERVIEW_TYPE_HTTP = "http";
|
||||
public static final String SERVERVIEW_TYPE_BATCH = "batch";
|
||||
|
||||
// this value should be consistent with the default implementation used in
|
||||
// {@code ServerInventoryViewProvider} & {@code FilteredServerInventoryViewProvider}
|
||||
public static final String DEFAULT_SERVERVIEW_TYPE = "http";
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
|
|
@ -1,64 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.indexing;
|
||||
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Class representing the combined DataSchema of a set of segments, currently used only by Compaction.
|
||||
*/
|
||||
public class CombinedDataSchema extends DataSchema
|
||||
{
|
||||
private final boolean hasRolledUpSegments;
|
||||
|
||||
public CombinedDataSchema(
|
||||
String dataSource,
|
||||
@Nullable TimestampSpec timestampSpec,
|
||||
@Nullable DimensionsSpec dimensionsSpec,
|
||||
AggregatorFactory[] aggregators,
|
||||
GranularitySpec granularitySpec,
|
||||
TransformSpec transformSpec,
|
||||
@Nullable boolean hasRolledUpSegments
|
||||
)
|
||||
{
|
||||
super(
|
||||
dataSource,
|
||||
timestampSpec,
|
||||
dimensionsSpec,
|
||||
aggregators,
|
||||
granularitySpec,
|
||||
transformSpec,
|
||||
null,
|
||||
null
|
||||
);
|
||||
this.hasRolledUpSegments = hasRolledUpSegments;
|
||||
}
|
||||
|
||||
public boolean hasRolledUpSegments()
|
||||
{
|
||||
return hasRolledUpSegments;
|
||||
}
|
||||
}
|
|
@ -19,29 +19,25 @@
|
|||
|
||||
package org.apache.druid.server.coordination;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.framework.recipes.cache.ChildData;
|
||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
|
||||
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderConfig;
|
||||
import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig;
|
||||
import org.apache.druid.server.initialization.ZkPathsConfig;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
/**
|
||||
* We are gradually migrating to {@link org.apache.druid.server.http.SegmentListerResource} for driving segment
|
||||
* loads/drops on data server processes.
|
||||
* Creates paths for announcing served segments on Zookeeper.
|
||||
*
|
||||
* However, this class is still the default mechanism as of this writing (2020-12-03).
|
||||
* @deprecated as Druid has already migrated to HTTP-based segment loading and
|
||||
* will soon migrate to HTTP-based inventory view using {@code SegmentListerResource}.
|
||||
*
|
||||
* @see org.apache.druid.server.http.SegmentListerResource
|
||||
*/
|
||||
@Deprecated
|
||||
public class ZkCoordinator
|
||||
|
@ -50,36 +46,25 @@ public class ZkCoordinator
|
|||
|
||||
private final Object lock = new Object();
|
||||
|
||||
private final DataSegmentChangeHandler dataSegmentChangeHandler;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final ZkPathsConfig zkPaths;
|
||||
private final DruidServerMetadata me;
|
||||
private final CuratorFramework curator;
|
||||
private final BatchDataSegmentAnnouncerConfig announcerConfig;
|
||||
|
||||
@Nullable
|
||||
private volatile PathChildrenCache loadQueueCache;
|
||||
private volatile boolean started = false;
|
||||
private final ExecutorService segmentLoadUnloadService;
|
||||
|
||||
@Inject
|
||||
public ZkCoordinator(
|
||||
SegmentLoadDropHandler loadDropHandler,
|
||||
ObjectMapper jsonMapper,
|
||||
ZkPathsConfig zkPaths,
|
||||
DruidServerMetadata me,
|
||||
CuratorFramework curator,
|
||||
SegmentLoaderConfig config
|
||||
BatchDataSegmentAnnouncerConfig announcerConfig
|
||||
)
|
||||
{
|
||||
this.dataSegmentChangeHandler = loadDropHandler;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.zkPaths = zkPaths;
|
||||
this.me = me;
|
||||
this.curator = curator;
|
||||
this.segmentLoadUnloadService = Execs.multiThreaded(
|
||||
config.getNumLoadingThreads(),
|
||||
"ZKCoordinator--%d"
|
||||
);
|
||||
this.announcerConfig = announcerConfig;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
|
@ -92,40 +77,17 @@ public class ZkCoordinator
|
|||
|
||||
log.info("Starting zkCoordinator for server[%s]", me.getName());
|
||||
|
||||
final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName());
|
||||
final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName());
|
||||
final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName());
|
||||
if (announcerConfig.isSkipSegmentAnnouncementOnZk()) {
|
||||
log.info("Skipping zkPath creation as segment announcement on ZK is disabled.");
|
||||
started = true;
|
||||
return;
|
||||
}
|
||||
|
||||
loadQueueCache = new PathChildrenCache(
|
||||
curator,
|
||||
loadQueueLocation,
|
||||
true,
|
||||
true,
|
||||
Execs.singleThreaded("ZkCoordinator")
|
||||
);
|
||||
final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName());
|
||||
log.info("Creating zkPath[%s] for announcing live segments.", liveSegmentsLocation);
|
||||
|
||||
try {
|
||||
curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient());
|
||||
curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient());
|
||||
curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient());
|
||||
|
||||
loadQueueCache.getListenable().addListener(
|
||||
(client, event) -> {
|
||||
final ChildData child = event.getData();
|
||||
switch (event.getType()) {
|
||||
case CHILD_ADDED:
|
||||
childAdded(child);
|
||||
break;
|
||||
case CHILD_REMOVED:
|
||||
log.info("zNode[%s] was removed", event.getData().getPath());
|
||||
break;
|
||||
default:
|
||||
log.info("Ignoring event[%s]", event);
|
||||
}
|
||||
}
|
||||
|
||||
);
|
||||
loadQueueCache.start();
|
||||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagateIfPossible(e, IOException.class);
|
||||
|
@ -136,54 +98,6 @@ public class ZkCoordinator
|
|||
}
|
||||
}
|
||||
|
||||
private void childAdded(ChildData child)
|
||||
{
|
||||
segmentLoadUnloadService.submit(() -> {
|
||||
final String path = child.getPath();
|
||||
DataSegmentChangeRequest request = new SegmentChangeRequestNoop();
|
||||
try {
|
||||
final DataSegmentChangeRequest finalRequest = jsonMapper.readValue(
|
||||
child.getData(),
|
||||
DataSegmentChangeRequest.class
|
||||
);
|
||||
|
||||
finalRequest.go(
|
||||
dataSegmentChangeHandler,
|
||||
() -> {
|
||||
try {
|
||||
curator.delete().guaranteed().forPath(path);
|
||||
log.info("Completed request [%s]", finalRequest.asString());
|
||||
}
|
||||
catch (Exception e) {
|
||||
try {
|
||||
curator.delete().guaranteed().forPath(path);
|
||||
}
|
||||
catch (Exception e1) {
|
||||
log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path);
|
||||
}
|
||||
log.error(e, "Exception while removing zNode[%s]", path);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
// Something went wrong in either deserializing the request using jsonMapper or when invoking it
|
||||
try {
|
||||
curator.delete().guaranteed().forPath(path);
|
||||
}
|
||||
catch (Exception e1) {
|
||||
log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path);
|
||||
}
|
||||
|
||||
log.makeAlert(e, "Segment load/unload: uncaught exception.")
|
||||
.addData("node", path)
|
||||
.addData("nodeProperties", request)
|
||||
.emit();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
|
@ -193,21 +107,7 @@ public class ZkCoordinator
|
|||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
loadQueueCache.close();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
finally {
|
||||
loadQueueCache = null;
|
||||
started = false;
|
||||
}
|
||||
started = false;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isStarted()
|
||||
{
|
||||
return started;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,15 +31,10 @@ public class ZkPathsConfig
|
|||
@JsonProperty
|
||||
private String announcementsPath;
|
||||
@JsonProperty
|
||||
@Deprecated
|
||||
private String servedSegmentsPath;
|
||||
@JsonProperty
|
||||
private String liveSegmentsPath;
|
||||
@JsonProperty
|
||||
private String coordinatorPath;
|
||||
@JsonProperty
|
||||
private String loadQueuePath;
|
||||
@JsonProperty
|
||||
private String connectorPath;
|
||||
|
||||
public String getBase()
|
||||
|
@ -57,12 +52,12 @@ public class ZkPathsConfig
|
|||
return (null == announcementsPath) ? defaultPath("announcements") : announcementsPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Path to announce served segments on.
|
||||
*
|
||||
* @deprecated Use HTTP-based segment discovery instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public String getServedSegmentsPath()
|
||||
{
|
||||
return (null == servedSegmentsPath) ? defaultPath("servedSegments") : servedSegmentsPath;
|
||||
}
|
||||
|
||||
public String getLiveSegmentsPath()
|
||||
{
|
||||
return (null == liveSegmentsPath) ? defaultPath("segments") : liveSegmentsPath;
|
||||
|
@ -78,11 +73,6 @@ public class ZkPathsConfig
|
|||
return defaultPath("overlord");
|
||||
}
|
||||
|
||||
public String getLoadQueuePath()
|
||||
{
|
||||
return (null == loadQueuePath) ? defaultPath("loadQueue") : loadQueuePath;
|
||||
}
|
||||
|
||||
public String getConnectorPath()
|
||||
{
|
||||
return (null == connectorPath) ? defaultPath("connector") : connectorPath;
|
||||
|
@ -116,9 +106,7 @@ public class ZkPathsConfig
|
|||
this.getConnectorPath().equals(otherConfig.getConnectorPath()) &&
|
||||
this.getLiveSegmentsPath().equals(otherConfig.getLiveSegmentsPath()) &&
|
||||
this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()) &&
|
||||
this.getLoadQueuePath().equals(otherConfig.getLoadQueuePath()) &&
|
||||
this.getPropertiesPath().equals(otherConfig.getPropertiesPath()) &&
|
||||
this.getServedSegmentsPath().equals(otherConfig.getServedSegmentsPath())) {
|
||||
this.getPropertiesPath().equals(otherConfig.getPropertiesPath())) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
@ -130,10 +118,8 @@ public class ZkPathsConfig
|
|||
int result = base != null ? base.hashCode() : 0;
|
||||
result = 31 * result + (propertiesPath != null ? propertiesPath.hashCode() : 0);
|
||||
result = 31 * result + (announcementsPath != null ? announcementsPath.hashCode() : 0);
|
||||
result = 31 * result + (servedSegmentsPath != null ? servedSegmentsPath.hashCode() : 0);
|
||||
result = 31 * result + (liveSegmentsPath != null ? liveSegmentsPath.hashCode() : 0);
|
||||
result = 31 * result + (coordinatorPath != null ? coordinatorPath.hashCode() : 0);
|
||||
result = 31 * result + (loadQueuePath != null ? loadQueuePath.hashCode() : 0);
|
||||
result = 31 * result + (connectorPath != null ? connectorPath.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -52,7 +52,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithHashedPartitionsSpecIsInvalid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new HashedPartitionsSpec(100, null, null),
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
|
@ -72,7 +72,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithMaxTotalRowsIsInvalid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(100, 100L),
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
|
@ -92,7 +92,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithDynamicPartitionsSpecIsValid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(100, null),
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
|
@ -105,7 +105,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithDimensionRangePartitionsSpecIsValid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false),
|
||||
Collections.emptyMap(),
|
||||
null,
|
||||
|
@ -118,7 +118,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithQueryGranularityAllIsValid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
Collections.emptyMap(),
|
||||
new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false),
|
||||
|
@ -131,7 +131,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
@Test
|
||||
public void testMSQEngineWithRollupFalseWithMetricsSpecIsInvalid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
Collections.emptyMap(),
|
||||
new UserCompactionTaskGranularityConfig(null, null, false),
|
||||
|
@ -148,10 +148,53 @@ public class ClientCompactionRunnerInfoTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQEngineWithRollupTrueWithoutMetricsSpecIsInvalid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
Collections.emptyMap(),
|
||||
new UserCompactionTaskGranularityConfig(null, null, true),
|
||||
null
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig(
|
||||
compactionConfig,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(
|
||||
"MSQ: 'granularitySpec.rollup' must be false if 'metricsSpec' is null",
|
||||
validationResult.getReason()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQEngineWithUnsupportedMetricsSpecIsInvalid()
|
||||
{
|
||||
// Aggregators having combiningFactory different from the aggregatorFactory are unsupported.
|
||||
final String inputColName = "added";
|
||||
final String outputColName = "sum_added";
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
Collections.emptyMap(),
|
||||
new UserCompactionTaskGranularityConfig(null, null, null),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}
|
||||
);
|
||||
CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig(
|
||||
compactionConfig,
|
||||
CompactionEngine.NATIVE
|
||||
);
|
||||
Assert.assertFalse(validationResult.isValid());
|
||||
Assert.assertEquals(
|
||||
"MSQ: Non-idempotent aggregator[sum_added] not supported in 'metricsSpec'.",
|
||||
validationResult.getReason()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMSQEngineWithRollupNullWithMetricsSpecIsValid()
|
||||
{
|
||||
DataSourceCompactionConfig compactionConfig = createCompactionConfig(
|
||||
DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig(
|
||||
new DynamicPartitionsSpec(3, null),
|
||||
Collections.emptyMap(),
|
||||
new UserCompactionTaskGranularityConfig(null, null, null),
|
||||
|
@ -161,7 +204,7 @@ public class ClientCompactionRunnerInfoTest
|
|||
.isValid());
|
||||
}
|
||||
|
||||
private static DataSourceCompactionConfig createCompactionConfig(
|
||||
private static DataSourceCompactionConfig createMSQCompactionConfig(
|
||||
PartitionsSpec partitionsSpec,
|
||||
Map<String, Object> context,
|
||||
@Nullable UserCompactionTaskGranularityConfig granularitySpec,
|
||||
|
|
|
@ -56,10 +56,8 @@ public class ZkPathsConfigTest extends JsonConfigTesterBase<ZkPathsConfig>
|
|||
propertyValues.put(StringUtils.format("%s.base", CONFIG_PREFIX), base);
|
||||
propertyValues.put(StringUtils.format("%s.propertiesPath", CONFIG_PREFIX), ZKPaths.makePath(base, "properties"));
|
||||
propertyValues.put(StringUtils.format("%s.announcementsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "announcements"));
|
||||
propertyValues.put(StringUtils.format("%s.servedSegmentsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "servedSegments"));
|
||||
propertyValues.put(StringUtils.format("%s.liveSegmentsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "segments"));
|
||||
propertyValues.put(StringUtils.format("%s.coordinatorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "coordinator"));
|
||||
propertyValues.put(StringUtils.format("%s.loadQueuePath", CONFIG_PREFIX), ZKPaths.makePath(base, "loadQueue"));
|
||||
propertyValues.put(StringUtils.format("%s.connectorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "connector"));
|
||||
|
||||
ZkPathsConfig zkPathsConfigObj = zkPathsConfig.get();
|
||||
|
|
|
@ -684,24 +684,4 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
Assert.assertSame(oldSchema.getParserMap(), newSchema.getParserMap());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCombinedDataSchemaSetsHasRolledUpSegments()
|
||||
{
|
||||
CombinedDataSchema schema = new CombinedDataSchema(
|
||||
IdUtilsTest.VALID_ID_CHARS,
|
||||
new TimestampSpec("time", "auto", null),
|
||||
DimensionsSpec.builder()
|
||||
.setDimensions(
|
||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimA", "dimB", "metric1"))
|
||||
)
|
||||
.setDimensionExclusions(ImmutableList.of("dimC"))
|
||||
.build(),
|
||||
null,
|
||||
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
|
||||
null,
|
||||
true
|
||||
);
|
||||
Assert.assertTrue(schema.hasRolledUpSegments());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,34 +19,16 @@
|
|||
|
||||
package org.apache.druid.server.coordination;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.curator.framework.CuratorFramework;
|
||||
import org.apache.curator.utils.EnsurePath;
|
||||
import org.apache.curator.utils.ZKPaths;
|
||||
import org.apache.druid.curator.CuratorTestBase;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.loading.SegmentLoaderConfig;
|
||||
import org.apache.druid.server.SegmentManager;
|
||||
import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig;
|
||||
import org.apache.druid.server.initialization.ZkPathsConfig;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ZkCoordinatorTest extends CuratorTestBase
|
||||
public class ZkCoordinatorTest
|
||||
{
|
||||
private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
|
||||
private final DruidServerMetadata me = new DruidServerMetadata(
|
||||
"dummyServer",
|
||||
"dummyHost",
|
||||
|
@ -65,100 +47,55 @@ public class ZkCoordinatorTest extends CuratorTestBase
|
|||
}
|
||||
};
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
@Test(timeout = 60_000L)
|
||||
public void testSegmentPathIsCreatedIfZkAnnouncementIsEnabled() throws Exception
|
||||
{
|
||||
setupServerAndCurator();
|
||||
curator.start();
|
||||
curator.blockUntilConnected();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown()
|
||||
{
|
||||
tearDownServerAndCurator();
|
||||
testSegmentPathCreated(true);
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
public void testLoadDrop() throws Exception
|
||||
public void testSegmentPathIsNotCreatedIfZkAnnouncementIsDisabled() throws Exception
|
||||
{
|
||||
EmittingLogger.registerEmitter(new NoopServiceEmitter());
|
||||
DataSegment segment = new DataSegment(
|
||||
"test",
|
||||
Intervals.of("P1d/2011-04-02"),
|
||||
"v0",
|
||||
ImmutableMap.of("version", "v0", "interval", Intervals.of("P1d/2011-04-02"), "cacheDir", "/no"),
|
||||
Arrays.asList("dim1", "dim2", "dim3"),
|
||||
Arrays.asList("metric1", "metric2"),
|
||||
NoneShardSpec.instance(),
|
||||
IndexIO.CURRENT_VERSION_ID,
|
||||
123L
|
||||
testSegmentPathCreated(false);
|
||||
}
|
||||
|
||||
private void testSegmentPathCreated(boolean announceSegmentsOnZk) throws Exception
|
||||
{
|
||||
final String liveSegmentsPath = ZKPaths.makePath(
|
||||
zkPaths.getLiveSegmentsPath(),
|
||||
me.getName()
|
||||
);
|
||||
|
||||
CountDownLatch loadLatch = new CountDownLatch(1);
|
||||
CountDownLatch dropLatch = new CountDownLatch(1);
|
||||
final EnsurePath mockEnsurePath = EasyMock.mock(EnsurePath.class);
|
||||
final CuratorFramework mockCurator = EasyMock.mock(CuratorFramework.class);
|
||||
|
||||
SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler(
|
||||
new SegmentLoaderConfig(),
|
||||
EasyMock.createNiceMock(DataSegmentAnnouncer.class),
|
||||
EasyMock.createNiceMock(SegmentManager.class)
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public void addSegment(DataSegment s, DataSegmentChangeCallback callback)
|
||||
{
|
||||
if (segment.getId().equals(s.getId())) {
|
||||
loadLatch.countDown();
|
||||
callback.execute();
|
||||
}
|
||||
}
|
||||
if (announceSegmentsOnZk) {
|
||||
EasyMock.expect(mockCurator.newNamespaceAwareEnsurePath(liveSegmentsPath))
|
||||
.andReturn(mockEnsurePath).once();
|
||||
|
||||
@Override
|
||||
public void removeSegment(DataSegment s, DataSegmentChangeCallback callback)
|
||||
{
|
||||
if (segment.getId().equals(s.getId())) {
|
||||
dropLatch.countDown();
|
||||
callback.execute();
|
||||
}
|
||||
}
|
||||
};
|
||||
EasyMock.expect(mockCurator.getZookeeperClient())
|
||||
.andReturn(null).once();
|
||||
|
||||
ZkCoordinator zkCoordinator = new ZkCoordinator(
|
||||
segmentLoadDropHandler,
|
||||
jsonMapper,
|
||||
mockEnsurePath.ensure(EasyMock.anyObject());
|
||||
EasyMock.expectLastCall().once();
|
||||
}
|
||||
|
||||
EasyMock.replay(mockCurator, mockEnsurePath);
|
||||
final ZkCoordinator zkCoordinator = new ZkCoordinator(
|
||||
zkPaths,
|
||||
me,
|
||||
curator,
|
||||
new SegmentLoaderConfig()
|
||||
mockCurator,
|
||||
new BatchDataSegmentAnnouncerConfig() {
|
||||
@Override
|
||||
public boolean isSkipSegmentAnnouncementOnZk()
|
||||
{
|
||||
return !announceSegmentsOnZk;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
zkCoordinator.start();
|
||||
|
||||
String segmentZkPath = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName(), segment.getId().toString());
|
||||
|
||||
curator
|
||||
.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.withMode(CreateMode.EPHEMERAL)
|
||||
.forPath(segmentZkPath, jsonMapper.writeValueAsBytes(new SegmentChangeRequestLoad(segment)));
|
||||
|
||||
loadLatch.await();
|
||||
|
||||
while (curator.checkExists().forPath(segmentZkPath) != null) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
curator
|
||||
.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.withMode(CreateMode.EPHEMERAL)
|
||||
.forPath(segmentZkPath, jsonMapper.writeValueAsBytes(new SegmentChangeRequestDrop(segment)));
|
||||
|
||||
dropLatch.await();
|
||||
|
||||
while (curator.checkExists().forPath(segmentZkPath) != null) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
EasyMock.verify();
|
||||
zkCoordinator.stop();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,7 +63,6 @@ of the coordinator in these situations.
|
|||
interfaces to communicate with external dependencies have been provided as simple in-memory implementations:
|
||||
- communication with metadata store: `SegmentMetadataManager`, `MetadataRuleManager`
|
||||
- communication with historicals: `HttpClient`, `ServerInventoryView`
|
||||
- `CuratorFramework`: provided as a mock as simulations of `CuratorLoadQueuePeon` are not supported yet
|
||||
4. __Inventory__: The coordinator maintains an inventory view of the cluster state. Simulations can choose from two
|
||||
modes of inventory update - auto and manual. In auto update mode, any change made to the cluster is immediately
|
||||
reflected in the inventory view. In manual update mode, the inventory must be explicitly synchronized with the
|
||||
|
|
|
@ -202,7 +202,7 @@ public class CoordinatorCompactionConfigsResourceTest
|
|||
.withInputSegmentSizeBytes(1000L)
|
||||
.withSkipOffsetFromLatest(Period.hours(3))
|
||||
.withGranularitySpec(
|
||||
new UserCompactionTaskGranularityConfig(Granularities.DAY, null, true)
|
||||
new UserCompactionTaskGranularityConfig(Granularities.DAY, null, false)
|
||||
)
|
||||
.withEngine(CompactionEngine.MSQ)
|
||||
.build();
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.calcite.rel.type.RelDataType;
|
|||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.schema.ProjectableFilterableTable;
|
||||
import org.apache.calcite.schema.ScannableTable;
|
||||
import org.apache.calcite.sql.SqlExplain;
|
||||
import org.apache.calcite.sql.SqlNode;
|
||||
|
@ -285,7 +286,8 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
{
|
||||
if (node instanceof TableScan) {
|
||||
RelOptTable table = node.getTable();
|
||||
if (table.unwrap(ScannableTable.class) != null && table.unwrap(DruidTable.class) == null) {
|
||||
if ((table.unwrap(ScannableTable.class) != null || table.unwrap(ProjectableFilterableTable.class) != null)
|
||||
&& table.unwrap(DruidTable.class) == null) {
|
||||
found.add(table);
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -32,6 +32,8 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.inject.Inject;
|
||||
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
|
||||
import it.unimi.dsi.fastutil.ints.IntSet;
|
||||
import org.apache.calcite.DataContext;
|
||||
import org.apache.calcite.linq4j.DefaultEnumerable;
|
||||
import org.apache.calcite.linq4j.Enumerable;
|
||||
|
@ -39,6 +41,8 @@ import org.apache.calcite.linq4j.Enumerator;
|
|||
import org.apache.calcite.linq4j.Linq4j;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rel.type.RelDataTypeFactory;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.schema.ProjectableFilterableTable;
|
||||
import org.apache.calcite.schema.ScannableTable;
|
||||
import org.apache.calcite.schema.Table;
|
||||
import org.apache.calcite.schema.impl.AbstractSchema;
|
||||
|
@ -68,6 +72,7 @@ import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHo
|
|||
import org.apache.druid.rpc.indexing.OverlordClient;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.metadata.AvailableSegmentMetadata;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.server.security.Access;
|
||||
|
@ -99,6 +104,7 @@ import java.util.Map.Entry;
|
|||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
public class SystemSchema extends AbstractSchema
|
||||
{
|
||||
|
@ -157,6 +163,23 @@ public class SystemSchema extends AbstractSchema
|
|||
.add("replication_factor", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
/**
|
||||
* List of [0..n) where n is the size of {@link #SEGMENTS_SIGNATURE}.
|
||||
*/
|
||||
private static final int[] SEGMENTS_PROJECT_ALL = IntStream.range(0, SEGMENTS_SIGNATURE.size()).toArray();
|
||||
|
||||
/**
|
||||
* Fields in {@link #SEGMENTS_SIGNATURE} that are serialized with {@link ObjectMapper#writeValueAsString(Object)}.
|
||||
*/
|
||||
private static final IntSet SEGMENTS_JSON_FIELDS = new IntOpenHashSet(
|
||||
new int[]{
|
||||
SEGMENTS_SIGNATURE.indexOf("shard_spec"),
|
||||
SEGMENTS_SIGNATURE.indexOf("dimensions"),
|
||||
SEGMENTS_SIGNATURE.indexOf("metrics"),
|
||||
SEGMENTS_SIGNATURE.indexOf("last_compaction_state")
|
||||
}
|
||||
);
|
||||
|
||||
static final RowSignature SERVERS_SIGNATURE = RowSignature
|
||||
.builder()
|
||||
.add("server", ColumnType.STRING)
|
||||
|
@ -241,7 +264,7 @@ public class SystemSchema extends AbstractSchema
|
|||
/**
|
||||
* This table contains row per segment from metadata store as well as served segments.
|
||||
*/
|
||||
static class SegmentsTable extends AbstractTable implements ScannableTable
|
||||
static class SegmentsTable extends AbstractTable implements ProjectableFilterableTable
|
||||
{
|
||||
private final DruidSchema druidSchema;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
@ -274,7 +297,11 @@ public class SystemSchema extends AbstractSchema
|
|||
}
|
||||
|
||||
@Override
|
||||
public Enumerable<Object[]> scan(DataContext root)
|
||||
public Enumerable<Object[]> scan(
|
||||
final DataContext root,
|
||||
final List<RexNode> filters,
|
||||
@Nullable final int[] projects
|
||||
)
|
||||
{
|
||||
// get available segments from druidSchema
|
||||
final Map<SegmentId, AvailableSegmentMetadata> availableSegmentMetadata =
|
||||
|
@ -327,35 +354,30 @@ public class SystemSchema extends AbstractSchema
|
|||
// is_active is true for published segments that are not overshadowed or else they should be realtime
|
||||
boolean isActive = isPublished ? !val.isOvershadowed() : val.isRealtime();
|
||||
|
||||
try {
|
||||
return new Object[]{
|
||||
segment.getId(),
|
||||
segment.getDataSource(),
|
||||
segment.getInterval().getStart().toString(),
|
||||
segment.getInterval().getEnd().toString(),
|
||||
segment.getSize(),
|
||||
segment.getVersion(),
|
||||
(long) segment.getShardSpec().getPartitionNum(),
|
||||
numReplicas,
|
||||
numRows,
|
||||
isActive ? IS_ACTIVE_TRUE : IS_ACTIVE_FALSE,
|
||||
isPublished ? IS_PUBLISHED_TRUE : IS_PUBLISHED_FALSE,
|
||||
isAvailable,
|
||||
isRealtime,
|
||||
val.isOvershadowed() ? IS_OVERSHADOWED_TRUE : IS_OVERSHADOWED_FALSE,
|
||||
segment.getShardSpec() == null ? null : jsonMapper.writeValueAsString(segment.getShardSpec()),
|
||||
segment.getDimensions() == null ? null : jsonMapper.writeValueAsString(segment.getDimensions()),
|
||||
segment.getMetrics() == null ? null : jsonMapper.writeValueAsString(segment.getMetrics()),
|
||||
segment.getLastCompactionState() == null ? null : jsonMapper.writeValueAsString(segment.getLastCompactionState()),
|
||||
// If the segment is unpublished, we won't have this information yet.
|
||||
// If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor.
|
||||
// This should be automatically updated in the next Coordinator poll.
|
||||
val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor()
|
||||
};
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return new Object[]{
|
||||
segment.getId(),
|
||||
segment.getDataSource(),
|
||||
segment.getInterval().getStart(),
|
||||
segment.getInterval().getEnd(),
|
||||
segment.getSize(),
|
||||
segment.getVersion(),
|
||||
(long) segment.getShardSpec().getPartitionNum(),
|
||||
numReplicas,
|
||||
numRows,
|
||||
isActive ? IS_ACTIVE_TRUE : IS_ACTIVE_FALSE,
|
||||
isPublished ? IS_PUBLISHED_TRUE : IS_PUBLISHED_FALSE,
|
||||
isAvailable,
|
||||
isRealtime,
|
||||
val.isOvershadowed() ? IS_OVERSHADOWED_TRUE : IS_OVERSHADOWED_FALSE,
|
||||
segment.getShardSpec(),
|
||||
segment.getDimensions(),
|
||||
segment.getMetrics(),
|
||||
segment.getLastCompactionState(),
|
||||
// If the segment is unpublished, we won't have this information yet.
|
||||
// If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor.
|
||||
// This should be automatically updated in the next Coordinator poll.
|
||||
val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor()
|
||||
};
|
||||
});
|
||||
|
||||
// If druid.centralizedDatasourceSchema.enabled is set on the Coordinator, all the segments in this loop
|
||||
|
@ -369,45 +391,43 @@ public class SystemSchema extends AbstractSchema
|
|||
if (segmentsAlreadySeen.contains(val.getKey())) {
|
||||
return null;
|
||||
}
|
||||
final DataSegment segment = val.getValue().getSegment();
|
||||
final PartialSegmentData partialSegmentData = partialSegmentDataMap.get(val.getKey());
|
||||
final long numReplicas = partialSegmentData == null ? 0L : partialSegmentData.getNumReplicas();
|
||||
try {
|
||||
return new Object[]{
|
||||
val.getKey(),
|
||||
val.getKey().getDataSource(),
|
||||
val.getKey().getInterval().getStart().toString(),
|
||||
val.getKey().getInterval().getEnd().toString(),
|
||||
val.getValue().getSegment().getSize(),
|
||||
val.getKey().getVersion(),
|
||||
(long) val.getValue().getSegment().getShardSpec().getPartitionNum(),
|
||||
numReplicas,
|
||||
val.getValue().getNumRows(),
|
||||
// is_active is true for unpublished segments iff they are realtime
|
||||
val.getValue().isRealtime() /* is_active */,
|
||||
// is_published is false for unpublished segments
|
||||
IS_PUBLISHED_FALSE,
|
||||
// is_available is assumed to be always true for segments announced by historicals or realtime tasks
|
||||
IS_AVAILABLE_TRUE,
|
||||
val.getValue().isRealtime(),
|
||||
IS_OVERSHADOWED_FALSE,
|
||||
// there is an assumption here that unpublished segments are never overshadowed
|
||||
val.getValue().getSegment().getShardSpec() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getShardSpec()),
|
||||
val.getValue().getSegment().getDimensions() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getDimensions()),
|
||||
val.getValue().getSegment().getMetrics() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getMetrics()),
|
||||
null, // unpublished segments from realtime tasks will not be compacted yet
|
||||
REPLICATION_FACTOR_UNKNOWN // If the segment is unpublished, we won't have this information yet.
|
||||
};
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return new Object[]{
|
||||
val.getKey(),
|
||||
val.getKey().getDataSource(),
|
||||
val.getKey().getInterval().getStart(),
|
||||
val.getKey().getInterval().getEnd(),
|
||||
segment.getSize(),
|
||||
val.getKey().getVersion(),
|
||||
(long) segment.getShardSpec().getPartitionNum(),
|
||||
numReplicas,
|
||||
val.getValue().getNumRows(),
|
||||
// is_active is true for unpublished segments iff they are realtime
|
||||
val.getValue().isRealtime() /* is_active */,
|
||||
// is_published is false for unpublished segments
|
||||
IS_PUBLISHED_FALSE,
|
||||
// is_available is assumed to be always true for segments announced by historicals or realtime tasks
|
||||
IS_AVAILABLE_TRUE,
|
||||
val.getValue().isRealtime(),
|
||||
IS_OVERSHADOWED_FALSE,
|
||||
// there is an assumption here that unpublished segments are never overshadowed
|
||||
segment.getShardSpec(),
|
||||
segment.getDimensions(),
|
||||
segment.getMetrics(),
|
||||
null, // unpublished segments from realtime tasks will not be compacted yet
|
||||
REPLICATION_FACTOR_UNKNOWN // If the segment is unpublished, we won't have this information yet.
|
||||
};
|
||||
});
|
||||
|
||||
final Iterable<Object[]> allSegments = Iterables.unmodifiableIterable(
|
||||
Iterables.concat(publishedSegments, availableSegments)
|
||||
);
|
||||
|
||||
return Linq4j.asEnumerable(allSegments).where(Objects::nonNull);
|
||||
return Linq4j.asEnumerable(allSegments)
|
||||
.where(Objects::nonNull)
|
||||
.select(row -> projectSegmentsRow(row, projects, jsonMapper));
|
||||
}
|
||||
|
||||
private Iterator<SegmentStatusInCluster> getAuthorizedPublishedSegments(
|
||||
|
@ -638,7 +658,10 @@ public class SystemSchema extends AbstractSchema
|
|||
/**
|
||||
* Returns a row for all node types which don't serve data. The returned row contains only static information.
|
||||
*/
|
||||
private static Object[] buildRowForNonDataServerWithLeadership(DiscoveryDruidNode discoveryDruidNode, boolean isLeader)
|
||||
private static Object[] buildRowForNonDataServerWithLeadership(
|
||||
DiscoveryDruidNode discoveryDruidNode,
|
||||
boolean isLeader
|
||||
)
|
||||
{
|
||||
final DruidNode node = discoveryDruidNode.getDruidNode();
|
||||
return new Object[]{
|
||||
|
@ -775,7 +798,7 @@ public class SystemSchema extends AbstractSchema
|
|||
for (DataSegment segment : authorizedServerSegments) {
|
||||
Object[] row = new Object[serverSegmentsTableSize];
|
||||
row[0] = druidServer.getHost();
|
||||
row[1] = segment.getId();
|
||||
row[1] = segment.getId().toString();
|
||||
rows.add(row);
|
||||
}
|
||||
}
|
||||
|
@ -1138,4 +1161,44 @@ public class SystemSchema extends AbstractSchema
|
|||
throw new ForbiddenException("Insufficient permission to view servers: " + stateAccess.toMessage());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Project a row using "projects" from {@link SegmentsTable#scan(DataContext, List, int[])}.
|
||||
*
|
||||
* Also, fix up types so {@link ColumnType#STRING} are transformed to Strings if they aren't yet. This defers
|
||||
* computation of {@link ObjectMapper#writeValueAsString(Object)} or {@link Object#toString()} until we know we
|
||||
* actually need it.
|
||||
*/
|
||||
private static Object[] projectSegmentsRow(
|
||||
final Object[] row,
|
||||
@Nullable final int[] projects,
|
||||
final ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
final int[] nonNullProjects = projects == null ? SEGMENTS_PROJECT_ALL : projects;
|
||||
final Object[] projectedRow = new Object[nonNullProjects.length];
|
||||
|
||||
for (int i = 0; i < nonNullProjects.length; i++) {
|
||||
final Object o = row[nonNullProjects[i]];
|
||||
|
||||
if (SEGMENTS_SIGNATURE.getColumnType(nonNullProjects[i]).get().is(ValueType.STRING)
|
||||
&& o != null
|
||||
&& !(o instanceof String)) {
|
||||
// Delay calling toString() or ObjectMapper#writeValueAsString() until we know we actually need this field.
|
||||
if (SEGMENTS_JSON_FIELDS.contains(nonNullProjects[i])) {
|
||||
try {
|
||||
projectedRow[i] = jsonMapper.writeValueAsString(o);
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
} else {
|
||||
projectedRow[i] = o.toString();
|
||||
}
|
||||
} else {
|
||||
projectedRow[i] = o;
|
||||
}
|
||||
}
|
||||
return projectedRow;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.sql.calcite.schema;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -579,7 +580,7 @@ public class SystemSchemaTest extends CalciteTestBase
|
|||
|
||||
EasyMock.replay(client, request, responseHolder, responseHandler, metadataView);
|
||||
DataContext dataContext = createDataContext(Users.SUPER);
|
||||
final List<Object[]> rows = segmentsTable.scan(dataContext).toList();
|
||||
final List<Object[]> rows = segmentsTable.scan(dataContext, Collections.emptyList(), null).toList();
|
||||
rows.sort((Object[] row1, Object[] row2) -> ((Comparable) row1[0]).compareTo(row2[0]));
|
||||
|
||||
// total segments = 8
|
||||
|
@ -717,6 +718,74 @@ public class SystemSchemaTest extends CalciteTestBase
|
|||
verifyTypes(rows, SystemSchema.SEGMENTS_SIGNATURE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSegmentsTableWithProjection() throws JsonProcessingException
|
||||
{
|
||||
final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper);
|
||||
final Set<SegmentStatusInCluster> publishedSegments = new HashSet<>(Arrays.asList(
|
||||
new SegmentStatusInCluster(publishedCompactedSegment1, true, 2, null, false),
|
||||
new SegmentStatusInCluster(publishedCompactedSegment2, false, 0, null, false),
|
||||
new SegmentStatusInCluster(publishedUncompactedSegment3, false, 2, null, false),
|
||||
new SegmentStatusInCluster(segment1, true, 2, null, false),
|
||||
new SegmentStatusInCluster(segment2, false, 0, null, false)
|
||||
));
|
||||
|
||||
EasyMock.expect(metadataView.getSegments()).andReturn(publishedSegments.iterator()).once();
|
||||
|
||||
EasyMock.replay(client, request, responseHolder, responseHandler, metadataView);
|
||||
DataContext dataContext = createDataContext(Users.SUPER);
|
||||
final List<Object[]> rows = segmentsTable.scan(
|
||||
dataContext,
|
||||
Collections.emptyList(),
|
||||
new int[]{
|
||||
SystemSchema.SEGMENTS_SIGNATURE.indexOf("last_compaction_state"),
|
||||
SystemSchema.SEGMENTS_SIGNATURE.indexOf("segment_id")
|
||||
}
|
||||
).toList();
|
||||
rows.sort((Object[] row1, Object[] row2) -> ((Comparable) row1[1]).compareTo(row2[1]));
|
||||
|
||||
// total segments = 8
|
||||
// segments test1, test2 are published and available
|
||||
// segment test3 is served by historical but unpublished or unused
|
||||
// segments test4, test5 are not published but available (realtime segments)
|
||||
// segment test2 is both published and served by a realtime server.
|
||||
|
||||
Assert.assertEquals(8, rows.size());
|
||||
|
||||
Assert.assertNull(null, rows.get(0)[0]);
|
||||
Assert.assertEquals("test1_2010-01-01T00:00:00.000Z_2011-01-01T00:00:00.000Z_version1", rows.get(0)[1]);
|
||||
|
||||
Assert.assertNull(null, rows.get(1)[0]);
|
||||
Assert.assertEquals("test2_2011-01-01T00:00:00.000Z_2012-01-01T00:00:00.000Z_version2", rows.get(1)[1]);
|
||||
|
||||
Assert.assertNull(null, rows.get(2)[0]);
|
||||
Assert.assertEquals("test3_2012-01-01T00:00:00.000Z_2013-01-01T00:00:00.000Z_version3_2", rows.get(2)[1]);
|
||||
|
||||
Assert.assertNull(null, rows.get(3)[0]);
|
||||
Assert.assertEquals("test4_2014-01-01T00:00:00.000Z_2015-01-01T00:00:00.000Z_version4", rows.get(3)[1]);
|
||||
|
||||
Assert.assertNull(null, rows.get(4)[0]);
|
||||
Assert.assertEquals("test5_2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z_version5", rows.get(4)[1]);
|
||||
|
||||
Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(5)[0]);
|
||||
Assert.assertEquals("wikipedia1_2007-01-01T00:00:00.000Z_2008-01-01T00:00:00.000Z_version1", rows.get(5)[1]);
|
||||
|
||||
Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(6)[0]);
|
||||
Assert.assertEquals("wikipedia2_2008-01-01T00:00:00.000Z_2009-01-01T00:00:00.000Z_version2", rows.get(6)[1]);
|
||||
|
||||
Assert.assertNull(null, rows.get(7)[0]);
|
||||
Assert.assertEquals("wikipedia3_2009-01-01T00:00:00.000Z_2010-01-01T00:00:00.000Z_version3", rows.get(7)[1]);
|
||||
|
||||
// Verify value types.
|
||||
verifyTypes(
|
||||
rows,
|
||||
RowSignature.builder()
|
||||
.add("last_compaction_state", ColumnType.STRING)
|
||||
.add("segment_id", ColumnType.STRING)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
private void verifyRow(
|
||||
Object[] row,
|
||||
String segmentId,
|
||||
|
@ -1519,11 +1588,7 @@ public class SystemSchemaTest extends CalciteTestBase
|
|||
expectedClass = Double.class;
|
||||
break;
|
||||
case STRING:
|
||||
if (signature.getColumnName(i).equals("segment_id")) {
|
||||
expectedClass = SegmentId.class;
|
||||
} else {
|
||||
expectedClass = String.class;
|
||||
}
|
||||
expectedClass = String.class;
|
||||
break;
|
||||
default:
|
||||
throw new IAE("Don't know what class to expect for valueType[%s]", columnType);
|
||||
|
|
Loading…
Reference in New Issue