Merge remote-tracking branch 'kgyrtkirk/quidem-runner-extension-submit' into quidem-record

This commit is contained in:
Zoltan Haindrich 2024-05-27 10:52:01 +00:00
commit 9ae80f05de
118 changed files with 4327 additions and 747 deletions

View File

@ -47,13 +47,14 @@ jobs:
strategy:
fail-fast: false
matrix:
testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-smq-disabled]
testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction, cds-task-schema-publish-disabled, cds-coordinator-metadata-query-disabled]
uses: ./.github/workflows/reusable-standard-its.yml
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
with:
build_jdk: 8
runtime_jdk: 8
testing_groups: -Dgroups=${{ matrix.testing_group }}
override_config_path: ./environment-configs/test-groups/prepopulated-data
use_indexer: middleManager
group: ${{ matrix.testing_group }}
@ -196,6 +197,6 @@ jobs:
with:
build_jdk: 8
runtime_jdk: 8
testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-smq-disabled
testing_groups: -DexcludedGroups=batch-index,input-format,input-source,perfect-rollup-parallel-batch-index,kafka-index,query,query-retry,query-error,realtime-index,security,ldap-security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage,append-ingestion,compaction,high-availability,upgrade,shuffle-deep-store,custom-coordinator-duties,centralized-datasource-schema,cds-task-schema-publish-disabled,cds-coordinator-metadata-query-disabled
use_indexer: ${{ matrix.indexer }}
group: other

View File

@ -157,4 +157,27 @@ public class SqlReverseLookupBenchmark
blackhole.consume(plannerResult);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void planEqualsInsideAndOutsideCase(Blackhole blackhole)
{
final String sql = StringUtils.format(
"SELECT COUNT(*) FROM foo\n"
+ "WHERE\n"
+ " CASE WHEN LOOKUP(dimZipf, 'benchmark-lookup', 'N/A') = '%s'\n"
+ " THEN NULL\n"
+ " ELSE LOOKUP(dimZipf, 'benchmark-lookup', 'N/A')\n"
+ " END IN ('%s', '%s', '%s')",
LookupBenchmarkUtil.makeKeyOrValue(0),
LookupBenchmarkUtil.makeKeyOrValue(1),
LookupBenchmarkUtil.makeKeyOrValue(2),
LookupBenchmarkUtil.makeKeyOrValue(3)
);
try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) {
final PlannerResult plannerResult = planner.plan();
blackhole.consume(plannerResult);
}
}
}

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
@ -36,6 +37,7 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
@ -204,7 +206,7 @@ public class InPlanningBenchmark
);
String prefix = ("explain plan for select long1 from foo where long1 in ");
final String sql = createQuery(prefix, inClauseLiteralsCount);
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.LONG);
final Sequence<Object[]> resultSequence = getPlan(sql, null);
final Object[] planResult = resultSequence.toList().get(0);
@ -222,12 +224,13 @@ public class InPlanningBenchmark
closer.close();
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryInSql(Blackhole blackhole)
{
String prefix = "explain plan for select long1 from foo where long1 in ";
final String sql = createQuery(prefix, inClauseLiteralsCount);
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.LONG);
getPlan(sql, blackhole);
}
@ -238,7 +241,7 @@ public class InPlanningBenchmark
{
String prefix =
"explain plan for select long1 from foo where string1 = '7' or long1 in ";
final String sql = createQuery(prefix, inClauseLiteralsCount);
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.LONG);
getPlan(sql, blackhole);
}
@ -250,7 +253,40 @@ public class InPlanningBenchmark
{
String prefix =
"explain plan for select long1 from foo where string1 = '7' or string1 = '8' or long1 in ";
final String sql = createQuery(prefix, inClauseLiteralsCount);
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.LONG);
getPlan(sql, blackhole);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryStringFunctionInSql(Blackhole blackhole)
{
String prefix =
"explain plan for select count(*) from foo where long1 = 8 or lower(string1) in ";
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.STRING);
getPlan(sql, blackhole);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryStringFunctionIsNotNullAndNotInSql(Blackhole blackhole)
{
String prefix =
"explain plan for select count(*) from foo where long1 = 8 and lower(string1) is not null and lower(string1) not in ";
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.STRING);
getPlan(sql, blackhole);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryStringFunctionIsNullOrInSql(Blackhole blackhole)
{
String prefix =
"explain plan for select count(*) from foo where long1 = 8 and (lower(string1) is null or lower(string1) in ";
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.STRING) + ')';
getPlan(sql, blackhole);
}
@ -259,19 +295,32 @@ public class InPlanningBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryJoinEqualOrInSql(Blackhole blackhole)
{
String prefix =
"explain plan for select foo.long1, fooright.string1 from foo inner join foo as fooright on foo.string1 = fooright.string1 where fooright.string1 = '7' or foo.long1 in ";
final String sql = createQuery(prefix, inClauseLiteralsCount);
final String sql = createQuery(prefix, inClauseLiteralsCount, ValueType.LONG);
getPlan(sql, blackhole);
}
private String createQuery(String prefix, int inClauseLiteralsCount)
private String createQuery(String prefix, int inClauseLiteralsCount, ValueType type)
{
StringBuilder sqlBuilder = new StringBuilder();
sqlBuilder.append(prefix).append('(');
IntStream.range(1, inClauseLiteralsCount - 1).forEach(i -> sqlBuilder.append(i).append(","));
sqlBuilder.append(inClauseLiteralsCount).append(")");
IntStream.range(1, inClauseLiteralsCount + 1).forEach(
i -> {
if (i > 1) {
sqlBuilder.append(',');
}
if (type == ValueType.LONG) {
sqlBuilder.append(i);
} else if (type == ValueType.STRING) {
sqlBuilder.append("'").append(i).append("'");
} else {
throw new ISE("Cannot generate IN with type[%s]", type);
}
}
);
sqlBuilder.append(")");
return sqlBuilder.toString();
}

View File

@ -557,7 +557,7 @@ public class SqlBenchmark
schemaInfo,
DimensionsSpec.builder().setDimensions(columnSchemas).build(),
TransformSpec.NONE,
IndexSpec.DEFAULT,
IndexSpec.builder().withStringDictionaryEncoding(getStringEncodingStrategy()).build(),
Granularities.NONE,
rowsPerSegment
);

View File

@ -3550,6 +3550,53 @@ Host: http://ROUTER_IP:ROUTER_PORT
```
</details>
### Handoff task groups for a supervisor early
Trigger handoff for specified task groups of a supervisor early. This is a best effort API and makes no guarantees of handoff execution
#### URL
`POST` `/druid/indexer/v1/supervisor/{supervisorId}/taskGroups/handoff`
#### Sample request
The following example shows how to handoff task groups for a Kafka supervisor with the name `social_media` and has the task groups: `1,2,3`.
<Tabs>
<TabItem value="3" label="cURL">
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/taskGroups/handoff"
--header 'Content-Type: application/json'
--data-raw '{"taskGroupIds": [1, 2, 3]}'
```
</TabItem>
<TabItem value="4" label="HTTP">
```HTTP
POST /druid/indexer/v1/supervisor/social_media/taskGroups/handoff HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
Content-Type: application/json
{
"taskGroupIds": [1, 2, 3],
}
```
</TabItem>
</Tabs>
#### Sample response
<details>
<summary>View the response</summary>
(empty response)
</details>
### Shut down a supervisor
Shuts down a supervisor. This endpoint is deprecated and will be removed in future releases. Use the equivalent [terminate](#terminate-a-supervisor) endpoint instead.

View File

@ -34,7 +34,7 @@ Compressed big decimal is an absolute number based complex type based on big dec
2. Accuracy: Provides greater level of accuracy in decimal arithmetic
## Operations
To use this extension, make sure to [load](../../configuration/extensions.md#loading-extensions) `compressed-big-decimal` to your config file.
To use this extension, make sure to [load](../../configuration/extensions.md#loading-extensions) `druid-compressed-bigdecimal` to your config file.
## Configuration
There are currently no configuration properties specific to Compressed Big Decimal

View File

@ -67,12 +67,12 @@ Depending on the size of the results you're expecting, saving the final results
By default, Druid saves the final results for queries from deep storage to task reports. Generally, this is acceptable for smaller result sets but may lead to timeouts for larger result sets.
When you run a query, include the context parameter `selectDestination` and set it to `DURABLESTORAGE`:
When you run a query, include the context parameter `selectDestination` and set it to `durableStorage`:
```json
"context":{
...
"selectDestination": "DURABLESTORAGE"
"selectDestination": "durableStorage"
}
```

View File

@ -52,7 +52,9 @@ Configure Druid SQL query planning using the parameters in the table below.
|`sqlPullUpLookup`|Whether to consider the [pull-up rewrite](lookups.md#pull-up) of the `LOOKUP` function during SQL planning.|true|
|`enableJoinLeftTableScanDirect`|`false`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.|
|`maxNumericInFilters`|`-1`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (123, 456)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.|
|`inSubQueryThreshold`|`2147483647`| Threshold for minimum number of values in an IN clause to convert the query to a JOIN operation on an inlined table rather than a predicate. A threshold of 0 forces usage of an inline table in all cases; a threshold of [Integer.MAX_VALUE] forces usage of OR in all cases. |
|`inFunctionThreshold`|`100`| At or beyond this threshold number of values, SQL `IN` is converted to [`SCALAR_IN_ARRAY`](sql-functions.md#scalar_in_array). A threshold of 0 forces this conversion in all cases. A threshold of [Integer.MAX_VALUE] disables this conversion. The converted function is eligible for fewer planning-time optimizations, which speeds up planning, but may prevent certain planning-time optimizations.|
|`inFunctionExprThreshold`|`2`| At or beyond this threshold number of values, SQL `IN` is eligible for execution using the native function `scalar_in_array` rather than an <code>&#124;&#124;</code> of `==`, even if the number of values is below `inFunctionThreshold`. This property only affects translation of SQL `IN` to a [native expression](math-expr.md). It does not affect translation of SQL `IN` to a [native filter](filters.md). This property is provided for backwards compatibility purposes, and may be removed in a future release.|
|`inSubQueryThreshold`|`2147483647`| At or beyond this threshold number of values, SQL `IN` is converted to `JOIN` on an inline table. `inFunctionThreshold` takes priority over this setting. A threshold of 0 forces usage of an inline table in all cases where the size of a SQL `IN` is larger than `inFunctionThreshold`. A threshold of `2147483647` disables the rewrite of SQL `IN` to `JOIN`. |
## Setting the query context
The query context parameters can be specified as a "context" object in the [JSON API](../api-reference/sql-api.md) or as a [JDBC connection properties object](../api-reference/sql-jdbc.md).

View File

@ -277,6 +277,6 @@ The [DataSketches extension](../development/extensions-core/datasketches-extensi
|`CASE WHEN boolean_expr1 THEN result1 \[ WHEN boolean_expr2 THEN result2 ... \] \[ ELSE resultN \] END`|Searched CASE.|
|`CAST(value AS TYPE)`|Cast value to another type. See [Data types](sql-data-types.md) for details about how Druid SQL handles CAST.|
|`COALESCE(value1, value2, ...)`|Returns the first value that is neither NULL nor empty string.|
|`DECODE_BASE64_COMPLEX(dataType, expr)`| Decodes a Base64-encoded string into a complex data type, where `dataType` is the complex data type and `expr` is the Base64-encoded string to decode. The `hyperUnique` and `serializablePairLongString` data types are supported by default. You can enable support for the following complex data types by loading their extensions:<br/><ul><li>`druid-bloom-filter`: `bloom`</li><li>`druid-datasketches`: `arrayOfDoublesSketch`, `HLLSketch`, `KllDoublesSketch`, `KllFloatsSketch`, `quantilesDoublesSketch`, `thetaSketch`</li><li>`druid-histogram`: `approximateHistogram`, `fixedBucketsHistogram`</li><li>`druid-stats`: `variance`</li><li>`druid-compressed-big-decimal`: `compressedBigDecimal`</li><li>`druid-momentsketch`: `momentSketch`</li><li>`druid-tdigestsketch`: `tDigestSketch`</li></ul>|
|`DECODE_BASE64_COMPLEX(dataType, expr)`| Decodes a Base64-encoded string into a complex data type, where `dataType` is the complex data type and `expr` is the Base64-encoded string to decode. The `hyperUnique` and `serializablePairLongString` data types are supported by default. You can enable support for the following complex data types by loading their extensions:<br/><ul><li>`druid-bloom-filter`: `bloom`</li><li>`druid-datasketches`: `arrayOfDoublesSketch`, `HLLSketch`, `KllDoublesSketch`, `KllFloatsSketch`, `quantilesDoublesSketch`, `thetaSketch`</li><li>`druid-histogram`: `approximateHistogram`, `fixedBucketsHistogram`</li><li>`druid-stats`: `variance`</li><li>`druid-compressed-bigdecimal`: `compressedBigDecimal`</li><li>`druid-momentsketch`: `momentSketch`</li><li>`druid-tdigestsketch`: `tDigestSketch`</li></ul>|
|`NULLIF(value1, value2)`|Returns NULL if `value1` and `value2` match, else returns `value1`.|
|`NVL(value1, value2)`|Returns `value1` if `value1` is not null, otherwise `value2`.|

View File

@ -55,7 +55,7 @@ import org.apache.druid.rpc.ServiceClientFactory;
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.server.DruidNode;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
@ -271,16 +271,16 @@ public class IndexerControllerContext implements ControllerContext
.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, queryKernelConfig.getMaxConcurrentStages());
// Put the lookup loading info in the task context to facilitate selective loading of lookups.
if (controllerTaskContext.get(PlannerContext.CTX_LOOKUP_LOADING_MODE) != null) {
if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) != null) {
taskContextOverridesBuilder.put(
PlannerContext.CTX_LOOKUP_LOADING_MODE,
controllerTaskContext.get(PlannerContext.CTX_LOOKUP_LOADING_MODE)
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE,
controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE)
);
}
if (controllerTaskContext.get(PlannerContext.CTX_LOOKUPS_TO_LOAD) != null) {
if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD) != null) {
taskContextOverridesBuilder.put(
PlannerContext.CTX_LOOKUPS_TO_LOAD,
controllerTaskContext.get(PlannerContext.CTX_LOOKUPS_TO_LOAD)
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD,
controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD)
);
}

View File

@ -27,7 +27,6 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Injector;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
@ -38,13 +37,9 @@ import org.apache.druid.msq.exec.MSQTasks;
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerContext;
import org.apache.druid.msq.exec.WorkerImpl;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nonnull;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
@ -190,26 +185,4 @@ public class MSQWorkerTask extends AbstractTask
{
return Objects.hash(super.hashCode(), controllerTaskId, workerNumber, retryCount, worker);
}
@Override
public LookupLoadingSpec getLookupLoadingSpec()
{
final Object lookupModeValue = getContext().get(PlannerContext.CTX_LOOKUP_LOADING_MODE);
if (lookupModeValue == null) {
return LookupLoadingSpec.ALL;
}
final LookupLoadingSpec.Mode lookupLoadingMode = LookupLoadingSpec.Mode.valueOf(lookupModeValue.toString());
if (lookupLoadingMode == LookupLoadingSpec.Mode.NONE) {
return LookupLoadingSpec.NONE;
} else if (lookupLoadingMode == LookupLoadingSpec.Mode.ONLY_REQUIRED) {
Collection<String> lookupsToLoad = (Collection<String>) getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
if (lookupsToLoad == null || lookupsToLoad.isEmpty()) {
throw InvalidInput.exception("Set of lookups to load cannot be %s for mode[ONLY_REQUIRED].", lookupsToLoad);
}
return LookupLoadingSpec.loadOnly(new HashSet<>(lookupsToLoad));
} else {
return LookupLoadingSpec.ALL;
}
}
}

View File

@ -285,9 +285,9 @@ public class MSQTaskQueryMaker implements QueryMaker
MSQTaskQueryMakerUtils.validateRealtimeReindex(querySpec);
final Map<String, Object> context = new HashMap<>();
context.put(PlannerContext.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode());
context.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode());
if (plannerContext.getLookupLoadingSpec().getMode() == LookupLoadingSpec.Mode.ONLY_REQUIRED) {
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad());
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad());
}
final MSQControllerTask controllerTask = new MSQControllerTask(

View File

@ -38,7 +38,6 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.sql.calcite.planner.ColumnMapping;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
@ -114,8 +113,8 @@ public class MSQControllerTaskTest
.dataSource("target")
.context(
ImmutableMap.of(
PlannerContext.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED)
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED)
)
.build()
)

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.error.DruidException;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.junit.Assert;
import org.junit.Test;
@ -125,7 +124,7 @@ public class MSQWorkerTaskTest
@Test
public void testGetLookupLoadingWithModeNoneInContext()
{
final ImmutableMap<String, Object> context = ImmutableMap.of(PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE);
final ImmutableMap<String, Object> context = ImmutableMap.of(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE);
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertEquals(LookupLoadingSpec.NONE, msqWorkerTask.getLookupLoadingSpec());
}
@ -134,8 +133,8 @@ public class MSQWorkerTaskTest
public void testGetLookupLoadingSpecWithLookupListInContext()
{
final ImmutableMap<String, Object> context = ImmutableMap.of(
PlannerContext.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertEquals(LookupLoadingSpec.Mode.ONLY_REQUIRED, msqWorkerTask.getLookupLoadingSpec().getMode());
Assert.assertEquals(ImmutableSet.of("lookupName1", "lookupName2"), msqWorkerTask.getLookupLoadingSpec().getLookupsToLoad());
@ -145,10 +144,10 @@ public class MSQWorkerTaskTest
public void testGetLookupLoadingSpecWithInvalidInput()
{
final HashMap<String, Object> context = new HashMap<>();
context.put(PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
context.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
// Setting CTX_LOOKUPS_TO_LOAD as null
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, null);
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, null);
MSQWorkerTask taskWithNullLookups = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
DruidException exception = Assert.assertThrows(
@ -160,7 +159,7 @@ public class MSQWorkerTaskTest
exception.getMessage());
// Setting CTX_LOOKUPS_TO_LOAD as empty list
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, Collections.emptyList());
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Collections.emptyList());
MSQWorkerTask taskWithEmptyLookups = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
exception = Assert.assertThrows(

View File

@ -855,7 +855,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
protected CompactionState expectedLastCompactionState = null;
protected Set<Interval> expectedTombstoneIntervals = null;
protected List<Object[]> expectedResultRows = null;
protected LookupLoadingSpec expectedLookupLoadingSpec = null;
protected LookupLoadingSpec expectedLookupLoadingSpec = LookupLoadingSpec.NONE;
protected Matcher<Throwable> expectedValidationErrorMatcher = null;
protected List<Pair<Predicate<MSQTaskReportPayload>, String>> adhocReportAssertionAndReasons = new ArrayList<>();
protected Matcher<Throwable> expectedExecutionErrorMatcher = null;
@ -1021,19 +1021,8 @@ public class MSQTestBase extends BaseCalciteQueryTest
protected void verifyLookupLoadingInfoInTaskContext(Map<String, Object> context)
{
String lookupLoadingMode = context.get(PlannerContext.CTX_LOOKUP_LOADING_MODE).toString();
List<String> lookupsToLoad = (List<String>) context.get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
if (expectedLookupLoadingSpec != null) {
Assert.assertEquals(expectedLookupLoadingSpec.getMode().toString(), lookupLoadingMode);
if (expectedLookupLoadingSpec.getMode().equals(LookupLoadingSpec.Mode.ONLY_REQUIRED)) {
Assert.assertEquals(new ArrayList<>(expectedLookupLoadingSpec.getLookupsToLoad()), lookupsToLoad);
} else {
Assert.assertNull(lookupsToLoad);
}
} else {
Assert.assertEquals(LookupLoadingSpec.Mode.NONE.toString(), lookupLoadingMode);
Assert.assertNull(lookupsToLoad);
}
LookupLoadingSpec specFromContext = LookupLoadingSpec.createFromContext(context, LookupLoadingSpec.ALL);
Assert.assertEquals(expectedLookupLoadingSpec, specFromContext);
}
protected void verifyWorkerCount(CounterSnapshotsTree counterSnapshotsTree)

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.indexer.HadoopDruidIndexerConfig;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@ -54,9 +55,8 @@ public class DecimalParquetInputTest extends BaseParquetInputTest
public void testReadParquetDecimalFixedLen() throws IOException, InterruptedException
{
// parquet-avro does not correctly convert decimal types
if (parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE)) {
return;
}
Assume.assumeFalse(parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE));
HadoopDruidIndexerConfig config = transformHadoopDruidIndexerConfig(
"example/decimals/dec_in_fix_len.json",
parserType,
@ -101,9 +101,8 @@ public class DecimalParquetInputTest extends BaseParquetInputTest
public void testReadParquetDecimali32() throws IOException, InterruptedException
{
// parquet-avro does not correctly convert decimal types
if (parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE)) {
return;
}
Assume.assumeFalse(parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE));
HadoopDruidIndexerConfig config = transformHadoopDruidIndexerConfig(
"example/decimals/dec_in_i32.json",
parserType,
@ -148,9 +147,8 @@ public class DecimalParquetInputTest extends BaseParquetInputTest
public void testReadParquetDecimali64() throws IOException, InterruptedException
{
// parquet-avro does not correctly convert decimal types
if (parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE)) {
return;
}
Assume.assumeFalse(parserType.equals(ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE));
HadoopDruidIndexerConfig config = transformHadoopDruidIndexerConfig(
"example/decimals/dec_in_i64.json",
parserType,

View File

@ -92,6 +92,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.server.coordinator.duty.CompactSegments;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
@ -249,6 +250,14 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg
this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec());
this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig);
this.segmentCacheManagerFactory = segmentCacheManagerFactory;
// Do not load any lookups in sub-tasks launched by compaction task, unless transformSpec is present.
// If transformSpec is present, we will not modify the context so that the sub-tasks can make the
// decision based on context values, loading all lookups by default.
// This is done to ensure backward compatibility since transformSpec can reference lookups.
if (transformSpec == null) {
addToContextIfAbsent(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE.toString());
}
}
@VisibleForTesting

View File

@ -334,9 +334,14 @@ public interface Task
);
}
/**
* Specifies the list of lookups to load for this task. Tasks load ALL lookups by default.
* This behaviour can be overridden by passing parameters {@link LookupLoadingSpec#CTX_LOOKUP_LOADING_MODE}
* and {@link LookupLoadingSpec#CTX_LOOKUPS_TO_LOAD} in the task context.
*/
@Nullable
default LookupLoadingSpec getLookupLoadingSpec()
{
return LookupLoadingSpec.ALL;
return LookupLoadingSpec.createFromContext(getContext(), LookupLoadingSpec.ALL);
}
}

View File

@ -137,6 +137,16 @@ public class SupervisorManager
return supervisor == null ? Optional.absent() : Optional.fromNullable(supervisor.lhs.getState());
}
public boolean handoffTaskGroupsEarly(String id, List<Integer> taskGroupIds)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
if (supervisor == null || supervisor.lhs == null) {
return false;
}
supervisor.lhs.handoffTaskGroupsEarly(taskGroupIds);
return true;
}
public boolean createOrUpdateAndStartSupervisor(SupervisorSpec spec)
{
Preconditions.checkState(started, "SupervisorManager not started");

View File

@ -19,6 +19,8 @@
package org.apache.druid.indexing.overlord.supervisor;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
@ -30,6 +32,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.druid.audit.AuditEntry;
import org.apache.druid.audit.AuditManager;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
@ -48,6 +51,7 @@ import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
@ -395,6 +399,45 @@ public class SupervisorResource
return terminate(id);
}
/**
* This method will immediately try to handoff the list of task group ids for the given supervisor.
* This is a best effort API and makes no guarantees of execution, e.g. if a non-existent task group id
* is passed to it, the API call will still suceced.
*/
@POST
@Path("/{id}/taskGroups/handoff")
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(SupervisorResourceFilter.class)
public Response handoffTaskGroups(@PathParam("id") final String id, @Nonnull final HandoffTaskGroupsRequest handoffTaskGroupsRequest)
{
List<Integer> taskGroupIds = handoffTaskGroupsRequest.getTaskGroupIds();
if (taskGroupIds == null || taskGroupIds.isEmpty()) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(ImmutableMap.of("error", "List of task groups to handoff can't be empty"))
.build();
}
return asLeaderWithSupervisorManager(
manager -> {
try {
if (manager.handoffTaskGroupsEarly(id, taskGroupIds)) {
return Response.ok().build();
} else {
return Response.status(Response.Status.NOT_FOUND)
.entity(ImmutableMap.of("error", StringUtils.format("Supervisor was not found [%s]", id)))
.build();
}
}
catch (NotImplementedException e) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(ImmutableMap.of("error", StringUtils.format("Supervisor [%s] does not support early handoff", id)))
.build();
}
}
);
}
@POST
@Path("/{id}/terminate")
@Produces(MediaType.APPLICATION_JSON)
@ -631,4 +674,22 @@ public class SupervisorResource
}
);
}
public static class HandoffTaskGroupsRequest
{
private final List<Integer> taskGroupIds;
@JsonCreator
public HandoffTaskGroupsRequest(@JsonProperty("taskGroupIds") List<Integer> taskGroupIds)
{
this.taskGroupIds = taskGroupIds;
}
@JsonProperty
public List<Integer> getTaskGroupIds()
{
return taskGroupIds;
}
}
}

View File

@ -203,6 +203,8 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
final String baseSequenceName;
DateTime completionTimeout; // is set after signalTasksToFinish(); if not done by timeout, take corrective action
boolean handoffEarly = false; // set by SupervisorManager.stopTaskGroupEarly
TaskGroup(
int groupId,
ImmutableMap<PartitionIdType, SequenceOffsetType> startingSequences,
@ -266,6 +268,16 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return tasks.keySet();
}
void setHandoffEarly()
{
handoffEarly = true;
}
Boolean getHandoffEarly()
{
return handoffEarly;
}
@VisibleForTesting
public String getBaseSequenceName()
{
@ -657,6 +669,39 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
}
}
private class HandoffTaskGroupsNotice implements Notice
{
final List<Integer> taskGroupIds;
private static final String TYPE = "handoff_task_group_notice";
HandoffTaskGroupsNotice(
@Nonnull final List<Integer> taskGroupIds
)
{
this.taskGroupIds = taskGroupIds;
}
@Override
public void handle()
{
for (Integer taskGroupId : taskGroupIds) {
TaskGroup taskGroup = activelyReadingTaskGroups.getOrDefault(taskGroupId, null);
if (taskGroup == null) {
log.info("Tried to stop task group [%d] for supervisor [%s] that wasn't actively reading.", taskGroupId, supervisorId);
continue;
}
log.info("Task group [%d] for supervisor [%s] will handoff early.", taskGroupId, supervisorId);
taskGroup.setHandoffEarly();
}
}
@Override
public String getType()
{
return TYPE;
}
}
protected class CheckpointNotice implements Notice
{
private final int taskGroupId;
@ -1932,6 +1977,12 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return false;
}
@Override
public void handoffTaskGroupsEarly(List<Integer> taskGroupIds)
{
addNotice(new HandoffTaskGroupsNotice(taskGroupIds));
}
private void discoverTasks() throws ExecutionException, InterruptedException
{
int taskCount = 0;
@ -3143,14 +3194,15 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
} else {
DateTime earliestTaskStart = computeEarliestTaskStartTime(group);
if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow() || group.getHandoffEarly()) {
// if this task has run longer than the configured duration
// as long as the pending task groups are less than the configured stop task count.
// If shutdownEarly has been set, ignore stopTaskCount since this is a manual operator action.
if (pendingCompletionTaskGroups.values()
.stream()
.mapToInt(CopyOnWriteArrayList::size)
.sum() + stoppedTasks.get()
< ioConfig.getMaxAllowedStops()) {
< ioConfig.getMaxAllowedStops() || group.getHandoffEarly()) {
log.info(
"Task group [%d] has run for [%s]. Stopping.",
groupId,

View File

@ -63,6 +63,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthorizerMapper;
import org.joda.time.Duration;
@ -71,6 +72,7 @@ import org.junit.Test;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
public class ClientCompactionTaskQuerySerdeTest
{
@ -78,61 +80,116 @@ public class ClientCompactionTaskQuerySerdeTest
new TestUtils().getRowIngestionMetersFactory();
private static final CoordinatorClient COORDINATOR_CLIENT = new NoopCoordinatorClient();
private static final AppenderatorsManager APPENDERATORS_MANAGER = new TestAppenderatorsManager();
private static final ObjectMapper MAPPER = setupInjectablesInObjectMapper(new DefaultObjectMapper());
private static final IndexSpec INDEX_SPEC = IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.LZF)
.withLongEncoding(LongEncodingStrategy.LONGS)
.build();
private static final IndexSpec INDEX_SPEC_FOR_INTERMEDIATE_PERSISTS = IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.UNCOMPRESSED)
.withLongEncoding(LongEncodingStrategy.AUTO)
.build();
private static final ClientCompactionTaskGranularitySpec CLIENT_COMPACTION_TASK_GRANULARITY_SPEC =
new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true);
private static final AggregatorFactory[] METRICS_SPEC = new AggregatorFactory[] {new CountAggregatorFactory("cnt")};
private static final ClientCompactionTaskTransformSpec CLIENT_COMPACTION_TASK_TRANSFORM_SPEC =
new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null));
private static final DynamicPartitionsSpec DYNAMIC_PARTITIONS_SPEC = new DynamicPartitionsSpec(100, 30000L);
private static final SegmentsSplitHintSpec SEGMENTS_SPLIT_HINT_SPEC = new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10);
@Test
public void testClientCompactionTaskQueryToCompactionTask() throws IOException
{
final ObjectMapper mapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
final ClientCompactionTaskQuery query = new ClientCompactionTaskQuery(
"id",
"datasource",
new ClientCompactionIOConfig(
new ClientCompactionIntervalSpec(
Intervals.of("2019/2020"),
"testSha256OfSortedSegmentIds"
),
true
final ClientCompactionTaskQuery query = createCompactionTaskQuery("id", CLIENT_COMPACTION_TASK_TRANSFORM_SPEC);
final byte[] json = MAPPER.writeValueAsBytes(query);
final CompactionTask task = (CompactionTask) MAPPER.readValue(json, Task.class);
assertQueryToTask(query, task);
}
@Test
public void testClientCompactionTaskQueryToCompactionTaskWithoutTransformSpec() throws IOException
{
final ClientCompactionTaskQuery query = createCompactionTaskQuery("id", null);
final byte[] json = MAPPER.writeValueAsBytes(query);
final CompactionTask task = (CompactionTask) MAPPER.readValue(json, Task.class);
// Verify that CompactionTask has added new parameters into the context because transformSpec was null.
Assert.assertNotEquals(query.getContext(), task.getContext());
query.getContext().put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE.toString());
assertQueryToTask(query, task);
}
@Test
public void testCompactionTaskToClientCompactionTaskQuery() throws IOException
{
final CompactionTask task = createCompactionTask(CLIENT_COMPACTION_TASK_TRANSFORM_SPEC);
final ClientCompactionTaskQuery expected = createCompactionTaskQuery(task.getId(), CLIENT_COMPACTION_TASK_TRANSFORM_SPEC);
final byte[] json = MAPPER.writeValueAsBytes(task);
final ClientCompactionTaskQuery actual = (ClientCompactionTaskQuery) MAPPER.readValue(json, ClientTaskQuery.class);
Assert.assertEquals(expected, actual);
}
@Test
public void testCompactionTaskToClientCompactionTaskQueryWithoutTransformSpec() throws IOException
{
final CompactionTask task = createCompactionTask(null);
final ClientCompactionTaskQuery expected = createCompactionTaskQuery(task.getId(), null);
final byte[] json = MAPPER.writeValueAsBytes(task);
final ClientCompactionTaskQuery actual = (ClientCompactionTaskQuery) MAPPER.readValue(json, ClientTaskQuery.class);
// Verify that CompactionTask has added new parameters into the context
Assert.assertNotEquals(expected, actual);
expected.getContext().put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE.toString());
Assert.assertEquals(expected, actual);
}
private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper)
{
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
objectMapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getSerializationConfig().getAnnotationIntrospector()
),
new ClientCompactionTaskQueryTuningConfig(
null,
null,
40000,
2000L,
null,
new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10),
new DynamicPartitionsSpec(100, 30000L),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.LZF)
.withLongEncoding(LongEncodingStrategy.LONGS)
.build(),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.UNCOMPRESSED)
.withLongEncoding(LongEncodingStrategy.AUTO)
.build(),
2,
1000L,
TmpFileSegmentWriteOutMediumFactory.instance(),
100,
5,
1000L,
new Duration(3000L),
7,
1000,
100,
2
),
new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true),
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)),
ImmutableMap.of("key", "value")
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
GuiceInjectableValues injectableValues = new GuiceInjectableValues(
GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.of(
binder -> {
binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER);
binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider());
binder.bind(RowIngestionMetersFactory.class).toInstance(ROW_INGESTION_METERS_FACTORY);
binder.bind(CoordinatorClient.class).toInstance(COORDINATOR_CLIENT);
binder.bind(SegmentCacheManagerFactory.class).toInstance(new SegmentCacheManagerFactory(objectMapper));
binder.bind(AppenderatorsManager.class).toInstance(APPENDERATORS_MANAGER);
binder.bind(OverlordClient.class).toInstance(new NoopOverlordClient());
}
)
)
);
objectMapper.setInjectableValues(injectableValues);
objectMapper.registerSubtypes(new NamedType(ParallelIndexTuningConfig.class, "index_parallel"));
return objectMapper;
}
final byte[] json = mapper.writeValueAsBytes(query);
final CompactionTask task = (CompactionTask) mapper.readValue(json, Task.class);
private void assertQueryToTask(ClientCompactionTaskQuery query, CompactionTask task)
{
Assert.assertEquals(query.getId(), task.getId());
Assert.assertEquals(query.getDataSource(), task.getDataSource());
Assert.assertTrue(task.getIoConfig().getInputSpec() instanceof CompactionIntervalSpec);
@ -226,8 +283,8 @@ public class ClientCompactionTaskQuerySerdeTest
task.getDimensionsSpec().getDimensions()
);
Assert.assertEquals(
query.getTransformSpec().getFilter(),
task.getTransformSpec().getFilter()
query.getTransformSpec(),
task.getTransformSpec()
);
Assert.assertArrayEquals(
query.getMetricsSpec(),
@ -235,16 +292,53 @@ public class ClientCompactionTaskQuerySerdeTest
);
}
@Test
public void testCompactionTaskToClientCompactionTaskQuery() throws IOException
private ClientCompactionTaskQuery createCompactionTaskQuery(String id, ClientCompactionTaskTransformSpec transformSpec)
{
final ObjectMapper mapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
final CompactionTask.Builder builder = new CompactionTask.Builder(
Map<String, Object> context = new HashMap<>();
context.put("key", "value");
return new ClientCompactionTaskQuery(
id,
"datasource",
new SegmentCacheManagerFactory(mapper),
new RetryPolicyFactory(new RetryPolicyConfig())
new ClientCompactionIOConfig(
new ClientCompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true
),
new ClientCompactionTaskQueryTuningConfig(
100,
new OnheapIncrementalIndex.Spec(true),
40000,
2000L,
30000L,
SEGMENTS_SPLIT_HINT_SPEC,
DYNAMIC_PARTITIONS_SPEC,
INDEX_SPEC,
INDEX_SPEC_FOR_INTERMEDIATE_PERSISTS,
2,
1000L,
TmpFileSegmentWriteOutMediumFactory.instance(),
100,
5,
1000L,
new Duration(3000L),
7,
1000,
100,
2
),
CLIENT_COMPACTION_TASK_GRANULARITY_SPEC,
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
METRICS_SPEC,
transformSpec,
context
);
final CompactionTask task = builder
}
private CompactionTask createCompactionTask(ClientCompactionTaskTransformSpec transformSpec)
{
CompactionTask.Builder compactionTaskBuilder = new CompactionTask.Builder(
"datasource",
new SegmentCacheManagerFactory(MAPPER),
new RetryPolicyFactory(new RetryPolicyConfig())
)
.inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds"), true)
.tuningConfig(
new ParallelIndexTuningConfig(
@ -256,18 +350,10 @@ public class ClientCompactionTaskQuerySerdeTest
null,
null,
null,
new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10),
new DynamicPartitionsSpec(100, 30000L),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.LZF)
.withLongEncoding(LongEncodingStrategy.LONGS)
.build(),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.UNCOMPRESSED)
.withLongEncoding(LongEncodingStrategy.AUTO)
.build(),
SEGMENTS_SPLIT_HINT_SPEC,
DYNAMIC_PARTITIONS_SPEC,
INDEX_SPEC,
INDEX_SPEC_FOR_INTERMEDIATE_PERSISTS,
2,
null,
null,
@ -290,100 +376,17 @@ public class ClientCompactionTaskQuerySerdeTest
null
)
)
.granularitySpec(new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true))
.granularitySpec(CLIENT_COMPACTION_TASK_GRANULARITY_SPEC)
.dimensionsSpec(
DimensionsSpec.builder()
.setDimensions(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim")))
.setDimensionExclusions(ImmutableList.of("__time", "val"))
.build()
)
.metricsSpec(new AggregatorFactory[] {new CountAggregatorFactory("cnt")})
.transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)))
.build();
.metricsSpec(METRICS_SPEC)
.transformSpec(transformSpec)
.context(ImmutableMap.of("key", "value"));
final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery(
task.getId(),
"datasource",
new ClientCompactionIOConfig(
new ClientCompactionIntervalSpec(
Intervals.of("2019/2020"),
"testSha256OfSortedSegmentIds"
),
true
),
new ClientCompactionTaskQueryTuningConfig(
100,
new OnheapIncrementalIndex.Spec(true),
40000,
2000L,
30000L,
new SegmentsSplitHintSpec(new HumanReadableBytes(100000L), 10),
new DynamicPartitionsSpec(100, 30000L),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.LZF)
.withLongEncoding(LongEncodingStrategy.LONGS)
.build(),
IndexSpec.builder()
.withDimensionCompression(CompressionStrategy.LZ4)
.withMetricCompression(CompressionStrategy.UNCOMPRESSED)
.withLongEncoding(LongEncodingStrategy.AUTO)
.build(),
2,
1000L,
TmpFileSegmentWriteOutMediumFactory.instance(),
100,
5,
1000L,
new Duration(3000L),
7,
1000,
100,
2
),
new ClientCompactionTaskGranularitySpec(Granularities.DAY, Granularities.HOUR, true),
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)),
new HashMap<>()
);
final byte[] json = mapper.writeValueAsBytes(task);
final ClientCompactionTaskQuery actual = (ClientCompactionTaskQuery) mapper.readValue(json, ClientTaskQuery.class);
Assert.assertEquals(expected, actual);
}
private static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper)
{
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
objectMapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
GuiceInjectableValues injectableValues = new GuiceInjectableValues(
GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.of(
binder -> {
binder.bind(AuthorizerMapper.class).toInstance(AuthTestUtils.TEST_AUTHORIZER_MAPPER);
binder.bind(ChatHandlerProvider.class).toInstance(new NoopChatHandlerProvider());
binder.bind(RowIngestionMetersFactory.class).toInstance(ROW_INGESTION_METERS_FACTORY);
binder.bind(CoordinatorClient.class).toInstance(COORDINATOR_CLIENT);
binder.bind(SegmentCacheManagerFactory.class).toInstance(new SegmentCacheManagerFactory(objectMapper));
binder.bind(AppenderatorsManager.class).toInstance(APPENDERATORS_MANAGER);
binder.bind(OverlordClient.class).toInstance(new NoopOverlordClient());
}
)
)
);
objectMapper.setInjectableValues(injectableValues);
objectMapper.registerSubtypes(new NamedType(ParallelIndexTuningConfig.class, "index_parallel"));
return objectMapper;
return compactionTaskBuilder.build();
}
}

View File

@ -133,6 +133,7 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.segment.selector.settable.SettableColumnValueSelector;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.ResourceAction;
@ -1738,6 +1739,35 @@ public class CompactionTaskTest
Assert.assertNull(chooseFinestGranularityHelper(input));
}
@Test
public void testGetDefaultLookupLoadingSpec()
{
final Builder builder = new Builder(
DATA_SOURCE,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
);
final CompactionTask task = builder
.interval(Intervals.of("2000-01-01/2000-01-02"))
.build();
Assert.assertEquals(LookupLoadingSpec.NONE, task.getLookupLoadingSpec());
}
@Test
public void testGetDefaultLookupLoadingSpecWithTransformSpec()
{
final Builder builder = new Builder(
DATA_SOURCE,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
);
final CompactionTask task = builder
.interval(Intervals.of("2000-01-01/2000-01-02"))
.transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)))
.build();
Assert.assertEquals(LookupLoadingSpec.ALL, task.getLookupLoadingSpec());
}
private Granularity chooseFinestGranularityHelper(List<Granularity> granularities)
{
SettableSupplier<Granularity> queryGranularity = new SettableSupplier<>();

View File

@ -26,6 +26,7 @@ import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.junit.Assert;
import org.junit.Test;
@ -128,4 +129,10 @@ public class TaskTest
TASK::getInputSourceResources
);
}
@Test
public void testGetLookupLoadingSpec()
{
Assert.assertEquals(LookupLoadingSpec.ALL, TASK.getLookupLoadingSpec());
}
}

View File

@ -240,6 +240,27 @@ public class SupervisorManagerTest extends EasyMockSupport
verifyAll();
}
@Test
public void testHandoffTaskGroupsEarly()
{
Map<String, SupervisorSpec> existingSpecs = ImmutableMap.of(
"id1", new TestSupervisorSpec("id1", supervisor1)
);
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
supervisor1.start();
supervisor1.handoffTaskGroupsEarly(ImmutableList.of(1));
replayAll();
manager.start();
Assert.assertTrue(manager.handoffTaskGroupsEarly("id1", ImmutableList.of(1)));
Assert.assertFalse(manager.handoffTaskGroupsEarly("id2", ImmutableList.of(1)));
verifyAll();
}
@Test
public void testStartAlreadyStarted()
{

View File

@ -1209,6 +1209,133 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
Assert.assertTrue(supervisor.getNoticesQueueSize() == 0);
}
@Test(timeout = 10_000L)
public void testSupervisorStopTaskGroupEarly() throws JsonProcessingException, InterruptedException
{
DateTime startTime = DateTimes.nowUtc();
SeekableStreamSupervisorIOConfig ioConfig = new SeekableStreamSupervisorIOConfig(
STREAM,
new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false),
1,
1,
new Period("PT1H"),
new Period("PT1S"),
new Period("PT30S"),
false,
new Period("PT30M"),
null,
null,
null,
null,
new IdleConfig(true, 200L),
null
)
{
};
EasyMock.reset(spec);
EasyMock.expect(spec.isSuspended()).andReturn(false).anyTimes();
EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes();
EasyMock.expect(spec.getIoConfig()).andReturn(ioConfig).anyTimes();
EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes();
EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes();
EasyMock.expect(spec.getMonitorSchedulerConfig()).andReturn(new DruidMonitorSchedulerConfig()
{
@Override
public Duration getEmissionDuration()
{
return new Period("PT2S").toStandardDuration();
}
}).anyTimes();
EasyMock.expect(spec.getType()).andReturn("stream").anyTimes();
EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes();
EasyMock.expect(spec.getContextValue("tags")).andReturn("").anyTimes();
EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.anyObject())).andReturn(true).anyTimes();
SeekableStreamIndexTaskTuningConfig taskTuningConfig = getTuningConfig().convertToTaskTuningConfig();
TreeMap<Integer, Map<String, Long>> sequenceOffsets = new TreeMap<>();
sequenceOffsets.put(0, ImmutableMap.of("0", 10L, "1", 20L));
Map<String, Object> context = new HashMap<>();
context.put("checkpoints", new ObjectMapper().writeValueAsString(sequenceOffsets));
TestSeekableStreamIndexTask id1 = new TestSeekableStreamIndexTask(
"id1",
null,
getDataSchema(),
taskTuningConfig,
createTaskIoConfigExt(
0,
Collections.singletonMap("0", "10"),
Collections.singletonMap("0", "20"),
"test",
startTime,
null,
Collections.emptySet(),
ioConfig
),
context,
"0"
);
final TaskLocation location1 = TaskLocation.create("testHost", 1234, -1);
Collection workItems = new ArrayList<>();
workItems.add(new TestTaskRunnerWorkItem(id1, null, location1));
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(workItems).anyTimes();
EasyMock.expect(taskRunner.getTaskLocation(id1.getId())).andReturn(location1).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE))
.andReturn(ImmutableList.of(id1))
.anyTimes();
EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.reset(indexerMetadataStorageCoordinator);
EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
.andReturn(new TestSeekableStreamDataSourceMetadata(null)).anyTimes();
EasyMock.expect(indexTaskClient.getStatusAsync("id1"))
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING))
.anyTimes();
EasyMock.expect(indexTaskClient.getStartTimeAsync("id1"))
.andReturn(Futures.immediateFuture(startTime.plusSeconds(1)))
.anyTimes();
ImmutableMap<String, String> partitionOffset = ImmutableMap.of("0", "10");
final TreeMap<Integer, Map<String, String>> checkpoints = new TreeMap<>();
checkpoints.put(0, partitionOffset);
EasyMock.expect(indexTaskClient.getCheckpointsAsync(EasyMock.contains("id1"), EasyMock.anyBoolean()))
.andReturn(Futures.immediateFuture(checkpoints))
.anyTimes();
// The task should only be pause/resumed in one of the runInternal commands, after stopTaskGroupEarly has been called.
EasyMock.expect(indexTaskClient.resumeAsync("id1"))
.andReturn(Futures.immediateFuture(true))
.once();
EasyMock.expect(indexTaskClient.pauseAsync("id1"))
.andReturn(Futures.immediateFuture(true))
.once();
taskQueue.shutdown("id1", "All tasks in group[%s] failed to transition to publishing state", 0);
replayAll();
SeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor();
supervisor.start();
supervisor.runInternal();
supervisor.handoffTaskGroupsEarly(ImmutableList.of(0));
while (supervisor.getNoticesQueueSize() > 0) {
Thread.sleep(100);
}
supervisor.runInternal();
verifyAll();
}
@Test
public void testEmitBothLag() throws Exception
{

View File

@ -0,0 +1,623 @@
/*
* 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.testsEx.catalog;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.ClusterKeySpec;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.msq.sql.SqlTaskStatus;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.sql.http.SqlQuery;
import org.apache.druid.testing.utils.DataLoaderHelper;
import org.apache.druid.testing.utils.MsqTestQueryHelper;
import org.apache.druid.testsEx.cluster.CatalogClient;
import org.apache.druid.testsEx.cluster.DruidClusterClient;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.Map;
import static org.junit.Assert.assertTrue;
/**
* Tests that expect succesfully ingestng data into catalog defined tables and querying the data
* gives expected results.
*/
public abstract class ITCatalogIngestAndQueryTest
{
public static final Logger LOG = new Logger(ITCatalogIngestAndQueryTest.class);
@Inject
private MsqTestQueryHelper msqHelper;
@Inject
private DataLoaderHelper dataLoaderHelper;
@Inject
private DruidClusterClient clusterClient;
private CatalogClient client;
private final String operationName;
private final String dmlPrefixPattern;
public ITCatalogIngestAndQueryTest()
{
this.operationName = getOperationName();
this.dmlPrefixPattern = getDmlPrefixPattern();
}
public abstract String getOperationName();
public abstract String getDmlPrefixPattern();
@Before
public void initializeClient()
{
client = new CatalogClient(clusterClient);
}
/**
* Create table with columns:
* <p>
* __time LONG
* double_col1 DOUBLE
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* <p>
* When querying the table with query: 'SELECT * from ##tableName', the BIGINT type column should
* be implicitly coherced into type DOUBLE when inserted into the table, since the column being
* written into is type DOUBLE.
* <p>
* __time, bigint_col1
* 2022-12-26T12:34:56,8.0
* 2022-12-26T12:34:56,8.0
* 2022-12-26T12:34:56,9.0
* 2022-12-26T12:34:56,10.0
*
*/
@Test
public void testInsertImplicitCast() throws Exception
{
String queryFile = "/catalog/implicitCast_select.sql";
String tableName = "testImplicitCast" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("double_col1", "DOUBLE")
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " c AS double_col1\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\\n2022-12-26T12:34:56,extra,9,\\\"30\\\",2.0,foo\\n2022-12-26T12:34:56,extra,8,\\\"40\\\",2.0,foq\\n2022-12-26T12:34:56,extra,8,\\\"50\\\",2.0,fop\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n";
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryInline);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Create table with columns:
* <p>
* __time LONG
* double_col1 DOUBLE
* <p>
* and clustering columns defined in catalog as
* <p>
* bigInt_col1
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* <p>
* When querying the table with query: 'SELECT * from ##tableName', because of the clustering
* defined on the table, the data should be reordered to:
* <p>
* __time, bigint_col1
* 2022-12-26T12:34:56,8
* 2022-12-26T12:34:56,8
* 2022-12-26T12:34:56,9
* 2022-12-26T12:34:56,10
*
*/
@Test
public void testInsertWithClusteringFromCatalog() throws Exception
{
String queryFile = "/catalog/clustering_select.sql";
String tableName = "testWithClusteringFromCatalog" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("bigint_col1", "BIGINT")
.property(
DatasourceDefn.CLUSTER_KEYS_PROPERTY,
ImmutableList.of(new ClusterKeySpec("bigint_col1", false))
)
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " c AS bigint_col1\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\\n2022-12-26T12:34:56,extra,9,\\\"30\\\",2.0,foo\\n2022-12-26T12:34:56,extra,8,\\\"40\\\",2.0,foq\\n2022-12-26T12:34:56,extra,8,\\\"50\\\",2.0,fop\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n";
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryInline);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Create table with columns:
* <p>
* __time LONG
* double_col1 DOUBLE
* <p>
* and clustering columns defined in query as
* <p>
* bigInt_col1
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* <p>
* When querying the table with query: 'SELECT * from ##tableName', because of the clustering
* defined on the table, the data should be reordered to:
* <p>
* __time, bigint_col1
* 2022-12-26T12:34:56,8
* 2022-12-26T12:34:56,8
* 2022-12-26T12:34:56,9
* 2022-12-26T12:34:56,10
*
*/
@Test
public void testInsertWithClusteringFromQuery() throws Exception
{
String queryFile = "/catalog/clustering_select.sql";
String tableName = "testWithClusteringFromQuery" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("bigint_col1", "BIGINT")
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " c AS bigint_col1\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\\n2022-12-26T12:34:56,extra,9,\\\"30\\\",2.0,foo\\n2022-12-26T12:34:56,extra,8,\\\"40\\\",2.0,foq\\n2022-12-26T12:34:56,extra,8,\\\"50\\\",2.0,fop\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "CLUSTERED BY \"bigint_col1\"\n";
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryInline);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Create table with columns:
* <p>
* __time LONG
* varchar_col1 VARCHAR
* bigint_col1 BIGINT
* float_col1 FLOAT
* varchar_col2 VARCHAR
* <p>
* and multiple clustering columns defined in catalog as
* <p>
* bigInt_col1, varchar_col2
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* <p>
* When querying the table with query: 'SELECT * from ##tableName', because of the clustering
* defined on the table, the data should be reordered to:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
*
*/
@Test
public void testInsertWithMultiClusteringFromCatalog() throws Exception
{
String queryFile = "/catalog/multiClustering_select.sql";
String tableName = "testWithMultiClusteringFromCatalog" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col1", "VARCHAR")
.column("bigint_col1", "BIGINT")
.column("float_col1", "FLOAT")
.column("varchar_col2", "VARCHAR")
.property(
DatasourceDefn.CLUSTER_KEYS_PROPERTY,
ImmutableList.of(new ClusterKeySpec("bigint_col1", false), new ClusterKeySpec("varchar_col2", false))
)
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS varchar_col1,\n"
+ " c AS bigint_col1,\n"
+ " e AS float_col1,\n"
+ " f AS varchar_col2\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\\n2022-12-26T12:34:56,extra,9,\\\"30\\\",2.0,foo\\n2022-12-26T12:34:56,extra,8,\\\"40\\\",2.0,foq\\n2022-12-26T12:34:56,extra,8,\\\"50\\\",2.0,fop\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n";
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryInline);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Create table with columns:
* <p>
* __time LONG
* varchar_col1 VARCHAR
* bigint_col1 BIGINT
* float_col1 FLOAT
* varchar_col2 VARCHAR
* <p>
* and multiple clustering columns defined in query as
* <p>
* bigInt_col1, varchar_col2
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* <p>
* When querying the table with query: 'SELECT * from ##tableName', because of the clustering
* defined on the query, the data should be reordered to:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,8,"50",2.0,fop
* 2022-12-26T12:34:56,extra,8,"40",2.0,foq
* 2022-12-26T12:34:56,extra,9,"30",2.0,foo
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
*
*/
@Test
public void testInsertWithMultiClusteringFromQuery() throws Exception
{
String queryFile = "/catalog/multiClustering_select.sql";
String tableName = "testWithMultiClusteringFromQuery" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col1", "VARCHAR")
.column("bigint_col1", "BIGINT")
.column("float_col1", "FLOAT")
.column("varchar_col2", "VARCHAR")
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS varchar_col1,\n"
+ " c AS bigint_col1,\n"
+ " e AS float_col1,\n"
+ " f AS varchar_col2\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\\n2022-12-26T12:34:56,extra,9,\\\"30\\\",2.0,foo\\n2022-12-26T12:34:56,extra,8,\\\"40\\\",2.0,foq\\n2022-12-26T12:34:56,extra,8,\\\"50\\\",2.0,fop\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "CLUSTERED BY \"bigint_col1\", \"varchar_col2\"\n";
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(queryInline);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Adding a new column during ingestion that is not defined in a sealed table, should fail with
* proper validation error. Disabling catalog validation, through context parameter, and issuing ingest
* query again, should succeed.
*/
@Test
public void testInsertNonDefinedColumnIntoSealedCatalogTableWithValidationDisabled() throws Exception
{
String queryFile = "/catalog/sealedWithValidationDisabled_select.sql";
String tableName = "testInsertNonDefinedColumnIntoSealedCatalogTableWithValidationDisabled" + operationName;
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.property(DatasourceDefn.SEALED_PROPERTY, true)
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(dmlPrefixPattern, tableName) + "\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " f AS extra\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "PARTITIONED BY DAY\n";
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(
sqlQueryFromString(
queryInline,
ImmutableMap.of()
),
null,
null,
HttpResponseStatus.BAD_REQUEST
);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
StringUtils.format("Column [extra] is not defined in the target table [druid.%s] strict schema", tableName))
);
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(
queryInline,
ImmutableMap.of(QueryContexts.CATALOG_VALIDATION_ENABLED, false)
);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
/**
* Assigning a column during ingestion, to an input type that is not compatible with the defined type of the
* column, should result in a proper validation error. Disabling catalog validation, through context parameter, and
* issuing ingest query again, should succeed.
*
* In this test we define the table as
* <p>
* __time LONG
* double_col DOUBLE
* <p>
* And insert the following data:
* <p>
* __time, varchar_col1, bigint_col1, float_col1, varchar_col2
* 2022-12-26T12:34:56,extra,10,"20",2.0,foo
* <p>
* even though the data is written
* as
* <p>
* 2022-12-26T12:34:56,extra
* <p>
* When querying the table with query: 'SELECT * from ##tableName', the data is returned as:
* <p>
* __time, double_col
* 2022-12-26T12:34:56,0.0
* <p>
* because the broker knows the double_col column to be a DOUBLE, and so converts to null (0.0) at query time.
*/
@Test
public void testInsertWithIncompatibleTypeAssignmentWithValidationDisabled() throws Exception
{
String tableName = "testInsertWithIncompatibleTypeAssignmentWithValidationDisabled" + operationName;
String queryFile = "/catalog/incompatibleTypeAssignmentWithValidationDisabled_select.sql";
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("double_col", "DOUBLE")
.property(DatasourceDefn.SEALED_PROPERTY, true)
.build();
client.createTable(table, true);
LOG.info("table created:\n%s", client.readTable(table.id()));
String queryInline =
StringUtils.format(
"INSERT INTO %s\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS double_col\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "PARTITIONED BY DAY\n",
tableName
);
LOG.info("Running query:\n%s", queryInline);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(
sqlQueryFromString(
queryInline,
ImmutableMap.of()
),
null,
null,
HttpResponseStatus.BAD_REQUEST
);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
"Cannot assign to target field 'double_col' of type DOUBLE from source field 'double_col' of type VARCHAR (line [4], column [3])")
);
// Submit the task and wait for the datasource to get loaded
LOG.info("Running query:\n%s", queryInline);
sqlTaskStatus = msqHelper.submitMsqTaskSuccesfully(
queryInline,
ImmutableMap.of(QueryContexts.CATALOG_VALIDATION_ENABLED, false)
);
if (sqlTaskStatus.getState().isFailure()) {
Assert.fail(StringUtils.format(
"Unable to start the task successfully.\nPossible exception: %s",
sqlTaskStatus.getError()
));
}
msqHelper.pollTaskIdForSuccess(sqlTaskStatus.getTaskId());
dataLoaderHelper.waitUntilDatasourceIsReady(tableName);
msqHelper.testQueriesFromFile(queryFile, tableName);
}
private static SqlQuery sqlQueryFromString(String queryString, Map<String, Object> context)
{
return new SqlQuery(queryString, null, false, false, false, context, null);
}
}

View File

@ -0,0 +1,251 @@
/*
* 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.testsEx.catalog;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.msq.sql.SqlTaskStatus;
import org.apache.druid.sql.http.SqlQuery;
import org.apache.druid.testing.utils.MsqTestQueryHelper;
import org.apache.druid.testsEx.categories.Catalog;
import org.apache.druid.testsEx.cluster.CatalogClient;
import org.apache.druid.testsEx.cluster.DruidClusterClient;
import org.apache.druid.testsEx.config.DruidTestRunner;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import java.util.concurrent.ExecutionException;
import static org.junit.Assert.assertTrue;
/**
* Tests that expect failures when ingestng data into catalog defined tables.
*/
@RunWith(DruidTestRunner.class)
@Category(Catalog.class)
public class ITCatalogIngestErrorTest
{
@Inject
private MsqTestQueryHelper msqHelper;
@Inject
private DruidClusterClient clusterClient;
private CatalogClient client;
@Before
public void initializeClient()
{
client = new CatalogClient(clusterClient);
}
/**
* If the segment grain is absent in the catalog and absent in the PARTITIONED BY clause in the query, then
* validation error.
*/
@Test
public void testInsertNoPartitonedByFromCatalogOrQuery() throws ExecutionException, InterruptedException
{
String tableName = "testInsertNoPartitonedByFromCatalogOrQuery";
TableMetadata table = new TableBuilder(TableId.datasource(tableName), DatasourceDefn.TABLE_TYPE)
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col", "VARCHAR")
.column("bigint_col", "BIGINT")
.column("float_col", "FLOAT")
.build();
client.createTable(table, true);
String queryInline =
StringUtils.format(
"INSERT INTO %s\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS varchar_col,\n"
+ " c AS bigint_col,\n"
+ " e AS float_col\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n",
tableName
);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(sqlQueryFromString(queryInline), null, null, HttpResponseStatus.BAD_REQUEST);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
"Operation [INSERT] requires a PARTITIONED BY to be explicitly defined, but none was found.")
);
}
/**
* Adding a new column during ingestion that is not defined in a sealed table should fail with
* proper validation error.
*/
@Test
public void testInsertNonDefinedColumnIntoSealedCatalogTable() throws ExecutionException, InterruptedException
{
String tableName = "testInsertNonDefinedColumnIntoSealedCatalogTable";
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col", "VARCHAR")
.column("bigint_col", "BIGINT")
.column("float_col", "FLOAT")
.property(DatasourceDefn.SEALED_PROPERTY, true)
.build();
client.createTable(table, true);
String queryInline =
StringUtils.format(
"INSERT INTO %s\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS varchar_col,\n"
+ " c AS bigint_col,\n"
+ " e AS float_col,\n"
+ " c AS extra\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "PARTITIONED BY DAY\n",
tableName
);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(sqlQueryFromString(queryInline), null, null, HttpResponseStatus.BAD_REQUEST);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
"Column [extra] is not defined in the target table [druid.testInsertNonDefinedColumnIntoSealedCatalogTable] strict schema")
);
}
/**
* Assigning a column during ingestion, to an input type that is not compatible with the defined type of the
* column, should result in a proper validation error.
*/
@Test
public void testInsertWithIncompatibleTypeAssignment() throws ExecutionException, InterruptedException
{
String tableName = "testInsertWithIncompatibleTypeAssignment";
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col", "VARCHAR")
.column("bigint_col", "BIGINT")
.column("float_col", "FLOAT")
.property(DatasourceDefn.SEALED_PROPERTY, true)
.build();
client.createTable(table, true);
String queryInline =
StringUtils.format(
"INSERT INTO %s\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " ARRAY[b] AS varchar_col,\n"
+ " c AS bigint_col,\n"
+ " e AS float_col\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "PARTITIONED BY DAY\n",
tableName
);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(sqlQueryFromString(queryInline), null, null, HttpResponseStatus.BAD_REQUEST);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
"Cannot assign to target field 'varchar_col' of type VARCHAR from source field 'varchar_col' of type VARCHAR ARRAY (line [4], column [3])")
);
}
/**
* Assigning a complex type column during ingestion, to an input type that is not compatible with the defined type of
* the column, should result in a proper validation error.
*/
@Test
public void testInsertGroupByWithIncompatibleTypeAssignment() throws ExecutionException, InterruptedException
{
String tableName = "testInsertGroupByWithIncompatibleTypeAssignment";
TableMetadata table = TableBuilder.datasource(tableName, "P1D")
.column(Columns.TIME_COLUMN, Columns.LONG)
.column("varchar_col", "VARCHAR")
.column("bigint_col", "BIGINT")
.column("float_col", "FLOAT")
.column("hll_col", "COMPLEX<hyperUnique>")
.property(DatasourceDefn.SEALED_PROPERTY, true)
.build();
client.createTable(table, true);
String queryInline =
StringUtils.format(
"INSERT INTO %s\n"
+ "SELECT\n"
+ " TIME_PARSE(a) AS __time,\n"
+ " b AS varchar_col,\n"
+ " c AS bigint_col,\n"
+ " e AS float_col,\n"
+ " ARRAY[b] AS hll_col\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{\"type\":\"inline\",\"data\":\"2022-12-26T12:34:56,extra,10,\\\"20\\\",2.0,foo\"}',\n"
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":false,\"columns\":[\"a\",\"b\",\"c\",\"d\",\"e\",\"f\"]}'\n"
+ " )\n"
+ ") "
+ " EXTEND (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e FLOAT, f VARCHAR)\n"
+ "PARTITIONED BY DAY\n",
tableName
);
SqlTaskStatus sqlTaskStatus = msqHelper.submitMsqTaskWithExpectedStatusCode(sqlQueryFromString(queryInline), null, null, HttpResponseStatus.BAD_REQUEST);
assertTrue(sqlTaskStatus.getError() != null && sqlTaskStatus.getError()
.getUnderlyingException()
.getMessage()
.equals(
"Cannot assign to target field 'hll_col' of type COMPLEX<hyperUnique> from source field 'hll_col' of type VARCHAR ARRAY (line [7], column [3])")
);
}
private static SqlQuery sqlQueryFromString(String queryString)
{
return new SqlQuery(queryString, null, false, false, false, ImmutableMap.of(), null);
}
}

View File

@ -0,0 +1,42 @@
/*
* 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.testsEx.catalog;
import org.apache.druid.testsEx.categories.Catalog;
import org.apache.druid.testsEx.config.DruidTestRunner;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
@RunWith(DruidTestRunner.class)
@Category(Catalog.class)
public class ITCatalogInsertAndQueryTest extends ITCatalogIngestAndQueryTest
{
@Override
public String getOperationName()
{
return "INSERT";
}
@Override
public String getDmlPrefixPattern()
{
return "INSERT INTO \"%s\"";
}
}

View File

@ -0,0 +1,42 @@
/*
* 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.testsEx.catalog;
import org.apache.druid.testsEx.categories.Catalog;
import org.apache.druid.testsEx.config.DruidTestRunner;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
@RunWith(DruidTestRunner.class)
@Category(Catalog.class)
public class ITCatalogReplaceAndQueryTest extends ITCatalogIngestAndQueryTest
{
@Override
public String getOperationName()
{
return "REPLACE";
}
@Override
public String getDmlPrefixPattern()
{
return "REPLACE INTO \"%s\" OVERWRITE ALL";
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.testsEx.catalog;
import com.google.common.collect.ImmutableList;
import com.google.inject.Inject;
import org.apache.druid.catalog.http.TableEditRequest.DropColumns;
import org.apache.druid.catalog.http.TableEditRequest.HideColumns;
@ -28,6 +29,7 @@ import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.ClusterKeySpec;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.java.util.common.ISE;
@ -106,6 +108,17 @@ public class ITCatalogRestTest
() -> client.createTable(table, false)
);
}
// DESC cluster keys not supported
{
final TableMetadata table = TableBuilder.datasource("foo", "P1D")
.property(DatasourceDefn.CLUSTER_KEYS_PROPERTY, ImmutableList.of(new ClusterKeySpec("clusterKeyA", true)))
.build();
assertThrows(
Exception.class,
() -> client.createTable(table, false)
);
}
}
/**

View File

@ -0,0 +1,23 @@
[
{
"query": "SELECT * FROM %%DATASOURCE%%",
"expectedResults": [
{
"__time": 1672058096000,
"bigint_col1": 8
},
{
"__time": 1672058096000,
"bigint_col1": 8
},
{
"__time": 1672058096000,
"bigint_col1": 9
},
{
"__time": 1672058096000,
"bigint_col1": 10
}
]
}
]

View File

@ -0,0 +1,23 @@
[
{
"query": "SELECT * FROM %%DATASOURCE%%",
"expectedResults": [
{
"__time": 1672058096000,
"double_col1": 8.0
},
{
"__time": 1672058096000,
"double_col1": 8.0
},
{
"__time": 1672058096000,
"double_col1": 9.0
},
{
"__time": 1672058096000,
"double_col1": 10.0
}
]
}
]

View File

@ -0,0 +1,11 @@
[
{
"query": "SELECT * FROM %%DATASOURCE%%",
"expectedResults": [
{
"__time": 1672058096000,
"double_col": 0.0
}
]
}
]

View File

@ -0,0 +1,35 @@
[
{
"query": "SELECT * FROM %%DATASOURCE%%",
"expectedResults": [
{
"__time": 1672058096000,
"bigint_col1": 8,
"varchar_col2": "fop",
"varchar_col1": "extra",
"float_col1": 2.0
},
{
"__time": 1672058096000,
"bigint_col1": 8,
"varchar_col2": "foq",
"varchar_col1": "extra",
"float_col1": 2.0
},
{
"__time": 1672058096000,
"bigint_col1": 9,
"varchar_col2": "foo",
"varchar_col1": "extra",
"float_col1": 2.0
},
{
"__time": 1672058096000,
"bigint_col1": 10,
"varchar_col2": "foo",
"varchar_col1": "extra",
"float_col1": 2.0
}
]
}
]

View File

@ -0,0 +1,11 @@
[
{
"query": "SELECT * FROM %%DATASOURCE%%",
"expectedResults": [
{
"__time": 1672058096000,
"extra": "foo"
}
]
}
]

View File

@ -20,7 +20,7 @@
AWS_REGION=us-east-1
# If you are making a change in load list below, make the necessary changes in github actions too
druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-integration-tests"]
druid_extensions_loadList=["mysql-metadata-storage","druid-s3-extensions","druid-basic-security","simple-client-sslcontext","druid-testing-tools","druid-lookups-cached-global","druid-histogram","druid-datasketches","druid-integration-tests","druid-parquet-extensions","druid-avro-extensions","druid-protobuf-extensions","druid-orc-extensions","druid-kafka-indexing-service"]
# Setting s3 credentials and region to use pre-populated data for testing.
druid_s3_accessKey=AKIAT2GGLKKJQCMG64V4

View File

@ -121,14 +121,29 @@ public class MsqTestQueryHelper extends AbstractTestQueryHelper<MsqQueryWithResu
* and returns the status associated with the submitted task
*/
public SqlTaskStatus submitMsqTaskSuccesfully(SqlQuery sqlQuery, String username, String password) throws ExecutionException, InterruptedException
{
return submitMsqTaskWithExpectedStatusCode(sqlQuery, username, password, HttpResponseStatus.ACCEPTED);
}
/**
* Submits a {@link SqlQuery} to the MSQ API for execution. This method waits for the task to be accepted by the cluster
* and returns the status associated with the submitted task
*/
public SqlTaskStatus submitMsqTaskWithExpectedStatusCode(
SqlQuery sqlQuery,
String username,
String password,
HttpResponseStatus expectedResponseStatus
) throws ExecutionException, InterruptedException
{
StatusResponseHolder statusResponseHolder = submitMsqTask(sqlQuery, username, password);
// Check if the task has been accepted successfully
HttpResponseStatus httpResponseStatus = statusResponseHolder.getStatus();
if (!httpResponseStatus.equals(HttpResponseStatus.ACCEPTED)) {
if (!httpResponseStatus.equals(expectedResponseStatus)) {
throw new ISE(
StringUtils.format(
"Unable to submit the task successfully. Received response status code [%d], and response content:\n[%s]",
"Expected response status code [%d] when submitting task. Received response status code [%d], and response content:\n[%s]",
expectedResponseStatus.getCode(),
httpResponseStatus.getCode(),
statusResponseHolder.getContent()
)

View File

@ -5040,7 +5040,7 @@ license_category: binary
module: web-console
license_name: Apache License version 2.0
copyright: Palantir Technologies
version: 4.20.1
version: 4.20.2
---
@ -5076,7 +5076,7 @@ license_category: binary
module: web-console
license_name: Apache License version 2.0
copyright: Palantir Technologies
version: 1.14.9
version: 1.14.11
---
@ -5085,7 +5085,7 @@ license_category: binary
module: web-console
license_name: Apache License version 2.0
copyright: Palantir Technologies
version: 4.9.22
version: 4.9.24
---

View File

@ -1371,7 +1371,7 @@
<plugin>
<groupId>org.openrewrite.maven</groupId>
<artifactId>rewrite-maven-plugin</artifactId>
<version>5.27.0</version>
<version>5.31.0</version>
<configuration>
<activeRecipes>
<recipe>org.apache.druid.RewriteRules</recipe>

View File

@ -70,10 +70,7 @@ public class ImmutableFloatNode implements ImmutableNode<float[]>
this.numChildren = (short) (header & 0x7FFF);
final int sizePosition = initialOffset + offsetFromInitial + HEADER_NUM_BYTES + 2 * numDims * Float.BYTES;
int bitmapSize = data.getInt(sizePosition);
this.childrenOffset = initialOffset
+ offsetFromInitial
+ HEADER_NUM_BYTES
+ 2 * numDims * Float.BYTES
this.childrenOffset = sizePosition
+ Integer.BYTES
+ bitmapSize;
@ -98,10 +95,7 @@ public class ImmutableFloatNode implements ImmutableNode<float[]>
this.isLeaf = leaf;
final int sizePosition = initialOffset + offsetFromInitial + HEADER_NUM_BYTES + 2 * numDims * Float.BYTES;
int bitmapSize = data.getInt(sizePosition);
this.childrenOffset = initialOffset
+ offsetFromInitial
+ HEADER_NUM_BYTES
+ 2 * numDims * Float.BYTES
this.childrenOffset = sizePosition
+ Integer.BYTES
+ bitmapSize;

View File

@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
import org.apache.druid.collections.spatial.ImmutableNode;
import javax.annotation.Nullable;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@ -43,6 +45,14 @@ public interface Bound<TCoordinateArray, TPoint extends ImmutableNode<TCoordinat
boolean contains(TCoordinateArray coords);
/***
* containsObj is mainly used to create object matechers on top custom/extensible spatial column,
* it receives it as object and corresponding implementations need to logic to unpack the objects and invoke contains
* @param input Takes an object spatial column as input
* @return boolean value if it falls within given bound
*/
boolean containsObj(@Nullable Object input);
Iterable<TPoint> filter(Iterable<TPoint> points);
byte[] getCacheKey();

View File

@ -26,12 +26,15 @@ import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import org.apache.druid.collections.spatial.ImmutableFloatPoint;
import org.apache.druid.collections.spatial.ImmutableNode;
import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Objects;
/**
*
*/
public class RectangularBound implements Bound<float[], ImmutableFloatPoint>
{
@ -118,6 +121,19 @@ public class RectangularBound implements Bound<float[], ImmutableFloatPoint>
return true;
}
@Override
public boolean containsObj(@Nullable Object input)
{
if (input instanceof String) {
final float[] coordinate = SpatialDimensionRowTransformer.decode((String) input);
if (coordinate == null) {
return false;
}
return contains(coordinate);
}
return false;
}
@Override
public Iterable<ImmutableFloatPoint> filter(Iterable<ImmutableFloatPoint> points)
{

View File

@ -20,6 +20,7 @@
package org.apache.druid.math.expr;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.filter.DruidDoublePredicate;
import org.apache.druid.query.filter.DruidFloatPredicate;
import org.apache.druid.query.filter.DruidLongPredicate;
@ -75,8 +76,32 @@ public class ExpressionPredicateIndexSupplier implements ColumnIndexSupplier
@Override
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
final java.util.function.Function<Object, ExprEval<?>> evalFunction =
inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue));
final java.util.function.Function<Object, ExprEval<?>> evalFunction;
if (NullHandling.sqlCompatible()) {
evalFunction =
inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue));
} else {
switch (inputType.getType()) {
case LONG:
evalFunction =
inputValue -> expr.eval(
InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue == null ? 0L : inputValue)
);
break;
case DOUBLE:
evalFunction =
inputValue -> expr.eval(
InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue == null ? 0.0 : inputValue)
);
break;
default:
evalFunction =
inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue));
}
}
return new DictionaryScanningBitmapIndex(inputColumnIndexes.getCardinality())
{

View File

@ -24,10 +24,11 @@ import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.QueryContexts.Vectorize;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.TypedInFilter;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
@ -575,6 +576,35 @@ public class QueryContext
);
}
/**
* At or above this threshold number of values, when planning SQL queries, use the SQL SCALAR_IN_ARRAY operator rather
* than a stack of SQL ORs. This speeds up planning for large sets of points because it is opaque to various
* expensive optimizations. But, because this does bypass certain optimizations, we only do the transformation above
* a certain threshold. The SCALAR_IN_ARRAY operator is still able to convert to {@link InDimFilter} or
* {@link TypedInFilter}.
*/
public int getInFunctionThreshold()
{
return getInt(
QueryContexts.IN_FUNCTION_THRESHOLD,
QueryContexts.DEFAULT_IN_FUNCTION_THRESHOLD
);
}
/**
* At or above this threshold, when converting the SEARCH operator to a native expression, use the "scalar_in_array"
* function rather than a sequence of equals (==) separated by or (||). This is typically a lower threshold
* than {@link #getInFunctionThreshold()}, because it does not prevent any SQL planning optimizations, and it
* speeds up query execution.
*/
public int getInFunctionExprThreshold()
{
return getInt(
QueryContexts.IN_FUNCTION_EXPR_THRESHOLD,
QueryContexts.DEFAULT_IN_FUNCTION_EXPR_THRESHOLD
);
}
public boolean isTimeBoundaryPlanningEnabled()
{
return getBoolean(
@ -591,6 +621,14 @@ public class QueryContext
);
}
public boolean isCatalogValidationEnabled()
{
return getBoolean(
QueryContexts.CATALOG_VALIDATION_ENABLED,
QueryContexts.DEFAULT_CATALOG_VALIDATION_ENABLED
);
}
public QueryResourceId getQueryResourceId()
{
return new QueryResourceId(getString(QueryContexts.QUERY_RESOURCE_ID));

View File

@ -77,6 +77,8 @@ public class QueryContexts
public static final String BY_SEGMENT_KEY = "bySegment";
public static final String BROKER_SERVICE_NAME = "brokerService";
public static final String IN_SUB_QUERY_THRESHOLD_KEY = "inSubQueryThreshold";
public static final String IN_FUNCTION_THRESHOLD = "inFunctionThreshold";
public static final String IN_FUNCTION_EXPR_THRESHOLD = "inFunctionExprThreshold";
public static final String TIME_BOUNDARY_PLANNING_KEY = "enableTimeBoundaryPlanning";
public static final String POPULATE_CACHE_KEY = "populateCache";
public static final String POPULATE_RESULT_LEVEL_CACHE_KEY = "populateResultLevelCache";
@ -86,6 +88,7 @@ public class QueryContexts
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
public static final String WINDOWING_STRICT_VALIDATION = "windowingStrictValidation";
public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled";
// Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the
// query's runtime
public static final String QUERY_RESOURCE_ID = "queryResourceId";
@ -120,8 +123,11 @@ public class QueryContexts
public static final boolean DEFAULT_SECONDARY_PARTITION_PRUNING = true;
public static final boolean DEFAULT_ENABLE_DEBUG = false;
public static final int DEFAULT_IN_SUB_QUERY_THRESHOLD = Integer.MAX_VALUE;
public static final int DEFAULT_IN_FUNCTION_THRESHOLD = 100;
public static final int DEFAULT_IN_FUNCTION_EXPR_THRESHOLD = 2;
public static final boolean DEFAULT_ENABLE_TIME_BOUNDARY_PLANNING = false;
public static final boolean DEFAULT_WINDOWING_STRICT_VALIDATION = true;
public static final boolean DEFAULT_CATALOG_VALIDATION_ENABLED = true;
@SuppressWarnings("unused") // Used by Jackson serialization
public enum Vectorize

View File

@ -28,7 +28,7 @@ import java.util.Comparator;
*/
public class FloatSumAggregator implements Aggregator
{
static final Comparator COMPARATOR = new Ordering()
public static final Comparator COMPARATOR = new Ordering()
{
@Override
public int compare(Object o, Object o1)

View File

@ -29,7 +29,7 @@ import java.util.Comparator;
*/
public class LongSumAggregator implements Aggregator
{
static final Comparator COMPARATOR = new Ordering()
public static final Comparator COMPARATOR = new Ordering()
{
@Override
public int compare(Object o, Object o1)

View File

@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.DoubleSumAggregator;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
@ -48,8 +49,6 @@ import java.util.Objects;
public class DoubleAnyAggregatorFactory extends AggregatorFactory
{
private static final Comparator<Double> VALUE_COMPARATOR = Comparator.nullsFirst(Double::compare);
private static final Aggregator NIL_AGGREGATOR = new DoubleAnyAggregator(
NilColumnValueSelector.instance()
)
@ -136,7 +135,7 @@ public class DoubleAnyAggregatorFactory extends AggregatorFactory
@Override
public Comparator getComparator()
{
return VALUE_COMPARATOR;
return DoubleSumAggregator.COMPARATOR;
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.FloatSumAggregator;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
@ -47,8 +48,6 @@ import java.util.Objects;
public class FloatAnyAggregatorFactory extends AggregatorFactory
{
private static final Comparator<Float> VALUE_COMPARATOR = Comparator.nullsFirst(Float::compare);
private static final Aggregator NIL_AGGREGATOR = new FloatAnyAggregator(
NilColumnValueSelector.instance()
)
@ -133,7 +132,7 @@ public class FloatAnyAggregatorFactory extends AggregatorFactory
@Override
public Comparator getComparator()
{
return VALUE_COMPARATOR;
return FloatSumAggregator.COMPARATOR;
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.LongSumAggregator;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseLongColumnValueSelector;
@ -46,8 +47,6 @@ import java.util.List;
public class LongAnyAggregatorFactory extends AggregatorFactory
{
private static final Comparator<Long> VALUE_COMPARATOR = Comparator.nullsFirst(Long::compare);
private static final Aggregator NIL_AGGREGATOR = new LongAnyAggregator(
NilColumnValueSelector.instance()
)
@ -132,7 +131,7 @@ public class LongAnyAggregatorFactory extends AggregatorFactory
@Override
public Comparator getComparator()
{
return VALUE_COMPARATOR;
return LongSumAggregator.COMPARATOR;
}
@Override

View File

@ -281,7 +281,7 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
@Override
public int getCardinality()
{
return globalDictionary.getCardinality();
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
}
@Override

View File

@ -43,7 +43,6 @@ import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer;
import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex;
import org.apache.druid.segment.index.BitmapColumnIndex;
import org.apache.druid.segment.index.semantic.SpatialIndex;
@ -174,8 +173,18 @@ public class SpatialFilter implements Filter
if (input == null) {
return DruidPredicateMatch.UNKNOWN;
}
final float[] coordinate = SpatialDimensionRowTransformer.decode(input);
return DruidPredicateMatch.of(bound.contains(coordinate));
return DruidPredicateMatch.of(bound.containsObj(input));
};
}
@Override
public DruidObjectPredicate<Object> makeObjectPredicate()
{
return input -> {
if (input == null) {
return DruidPredicateMatch.UNKNOWN;
}
return DruidPredicateMatch.of(bound.containsObj(input));
};
}

View File

@ -203,6 +203,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
capabilitiesBuilder.setDictionaryValuesSorted(true);
capabilitiesBuilder.setDictionaryValuesUnique(true);
builder.setType(logicalType);
builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false);
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType));
@ -225,6 +226,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn
// interface, so do not make the claim in the ColumnCapabilities
builder.setType(logicalType);
builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false);
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType));
@ -247,6 +249,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn
// interface, so do not make the claim in the ColumnCapabilities
builder.setType(logicalType);
builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false);
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType));
@ -275,6 +278,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
capabilitiesBuilder.setDictionaryValuesUnique(true);
}
builder.setType(logicalType);
builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false);
builder.setColumnFormat(new NestedCommonFormatColumn.Format(
@ -306,6 +310,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
ColumnType simpleType = supplier.getLogicalType();
ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType;
builder.setType(logicalType);
builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier);
// in default value mode, SQL planning by default uses selector filters for things like 'is null', which does
// not work correctly for complex types (or arrays). so, only hook up this index in sql compatible mode so that

View File

@ -41,7 +41,6 @@ import org.joda.time.Interval;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -337,11 +336,35 @@ public class QueryContextTest
ImmutableMap.of(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "auto")
);
assertEquals("auto", context2.getMaxSubqueryMemoryBytes(null));
final QueryContext context3 = new QueryContext(ImmutableMap.of());
assertEquals("disabled", context3.getMaxSubqueryMemoryBytes("disabled"));
}
@Test
public void testGetInFunctionThreshold()
{
final QueryContext context1 = new QueryContext(
ImmutableMap.of(QueryContexts.IN_FUNCTION_THRESHOLD, Integer.MAX_VALUE)
);
assertEquals(Integer.MAX_VALUE, context1.getInFunctionThreshold());
final QueryContext context2 = QueryContext.empty();
assertEquals(QueryContexts.DEFAULT_IN_FUNCTION_THRESHOLD, context2.getInFunctionThreshold());
}
@Test
public void testGetInFunctionExprThreshold()
{
final QueryContext context1 = new QueryContext(
ImmutableMap.of(QueryContexts.IN_FUNCTION_EXPR_THRESHOLD, Integer.MAX_VALUE)
);
assertEquals(Integer.MAX_VALUE, context1.getInFunctionExprThreshold());
final QueryContext context2 = QueryContext.empty();
assertEquals(QueryContexts.DEFAULT_IN_FUNCTION_EXPR_THRESHOLD, context2.getInFunctionExprThreshold());
}
@Test
public void testDefaultEnableQueryDebugging()
{

View File

@ -160,6 +160,24 @@ public class QueryContextsTest
);
}
@Test
public void testCatalogValidationEnabled()
{
Assert.assertEquals(
QueryContexts.DEFAULT_CATALOG_VALIDATION_ENABLED,
QueryContext.empty().isCatalogValidationEnabled()
);
Assert.assertTrue(QueryContext.of(ImmutableMap.of(
QueryContexts.CATALOG_VALIDATION_ENABLED,
true
)).isCatalogValidationEnabled());
Assert.assertFalse(QueryContext.of(ImmutableMap.of(
QueryContexts.CATALOG_VALIDATION_ENABLED,
false
)).isCatalogValidationEnabled());
}
@Test
public void testGetEnableJoinLeftScanDirect()
{

View File

@ -117,6 +117,17 @@ public class DoubleAnyAggregationTest extends InitializedNullHandlingTest
Assert.assertEquals(-1, comparator.compare(d2, d1));
}
@Test
public void testComparatorWithTypeMismatch()
{
Long n1 = 3L;
Double n2 = 4.0;
Comparator comparator = doubleAnyAggFactory.getComparator();
Assert.assertEquals(0, comparator.compare(n1, n1));
Assert.assertEquals(-1, comparator.compare(n1, n2));
Assert.assertEquals(1, comparator.compare(n2, n1));
}
@Test
public void testDoubleAnyCombiningAggregator()
{

View File

@ -117,6 +117,17 @@ public class FloatAnyAggregationTest extends InitializedNullHandlingTest
Assert.assertEquals(-1, comparator.compare(f2, f1));
}
@Test
public void testComparatorWithTypeMismatch()
{
Long n1 = 3L;
Float n2 = 4.0f;
Comparator comparator = floatAnyAggFactory.getComparator();
Assert.assertEquals(0, comparator.compare(n1, n1));
Assert.assertEquals(-1, comparator.compare(n1, n2));
Assert.assertEquals(1, comparator.compare(n2, n1));
}
@Test
public void testFloatAnyCombiningAggregator()
{

View File

@ -118,6 +118,17 @@ public class LongAnyAggregationTest extends InitializedNullHandlingTest
Assert.assertEquals(-1, comparator.compare(l2, l1));
}
@Test
public void testComparatorWithTypeMismatch()
{
Integer n1 = 3;
Long n2 = 4L;
Comparator comparator = longAnyAggFactory.getComparator();
Assert.assertEquals(0, comparator.compare(n1, n1));
Assert.assertEquals(-1, comparator.compare(n1, n2));
Assert.assertEquals(1, comparator.compare(n2, n1));
}
@Test
public void testLongAnyCombiningAggregator()
{

View File

@ -622,6 +622,30 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
);
}
@Test
public void testGroupByRootAuto()
{
GroupByQuery groupQuery = GroupByQuery.builder()
.setDataSource("test_datasource")
.setGranularity(Granularities.ALL)
.setInterval(Intervals.ETERNITY)
.setDimensions(DefaultDimensionSpec.of("dim"))
.setAggregatorSpecs(new CountAggregatorFactory("count"))
.setContext(getContext())
.build();
runResults(
groupQuery,
ImmutableList.of(
new Object[]{"100", 2L},
new Object[]{"hello", 12L},
new Object[]{"world", 2L}
)
);
}
private void runResults(
GroupByQuery groupQuery,
List<Object[]> expectedResults

View File

@ -70,69 +70,69 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
{
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer("test", null);
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
EncodedKeyComponent<StructuredData> key;
// new raw value, new field, new dictionary entry
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
Assert.assertEquals(228, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
// adding same value only adds estimated size of value itself
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
Assert.assertEquals(112, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
// new raw value, new field, new dictionary entry
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
Assert.assertEquals(94, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 2, indexer.globalDictionary.getCardinality());
// adding same value only adds estimated size of value itself
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 2, indexer.globalDictionary.getCardinality());
// new raw value, new dictionary entry
key = indexer.processRowValsToUnsortedEncodedKeyComponent(11L, false);
Assert.assertEquals(48, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 3, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 3, indexer.globalDictionary.getCardinality());
// new raw value, new fields
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
Assert.assertEquals(168, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 6, indexer.globalDictionary.getCardinality());
// new raw value, re-use fields and dictionary
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
Assert.assertEquals(104, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 6, indexer.globalDictionary.getCardinality());
// new raw value, new fields
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
false
);
Assert.assertEquals(166, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 6, indexer.globalDictionary.getCardinality());
// new raw value
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
false
);
Assert.assertEquals(166, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 6, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false);
if (NullHandling.replaceWithDefault()) {
Assert.assertEquals(0, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 7, indexer.globalDictionary.getCardinality());
} else {
Assert.assertEquals(104, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 7, indexer.globalDictionary.getCardinality());
}
key = indexer.processRowValsToUnsortedEncodedKeyComponent(0L, false);
if (NullHandling.replaceWithDefault()) {
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 7, indexer.globalDictionary.getCardinality());
} else {
Assert.assertEquals(48, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 8, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 8, indexer.globalDictionary.getCardinality());
}
}
@ -673,14 +673,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
Assert.assertEquals(0, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
Assert.assertEquals(0, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
Assert.assertEquals(0, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
Assert.assertTrue(indexer.hasNulls);
@ -698,14 +698,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
Assert.assertEquals(166, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
Assert.assertEquals(52, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
Assert.assertEquals(52, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
Assert.assertFalse(indexer.hasNulls);
Assert.assertFalse(indexer.hasNestedData);
@ -722,14 +722,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
Assert.assertEquals(94, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
Assert.assertFalse(indexer.hasNulls);
Assert.assertFalse(indexer.hasNestedData);
@ -746,14 +746,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
Assert.assertEquals(54, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
Assert.assertEquals(8, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
Assert.assertEquals(8, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 1, indexer.globalDictionary.getCardinality());
Assert.assertFalse(indexer.hasNulls);
Assert.assertFalse(indexer.hasNestedData);
@ -770,14 +770,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
Assert.assertEquals(246, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 4, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
Assert.assertEquals(104, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 4, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
Assert.assertEquals(104, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
Assert.assertEquals(baseCardinality + 4, indexer.globalDictionary.getCardinality());
Assert.assertFalse(indexer.hasNulls);
Assert.assertFalse(indexer.hasNestedData);
@ -794,14 +794,14 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
Assert.assertEquals(16, key.getEffectiveSizeBytes());
Assert.assertEquals(baseCardinality, indexer.getCardinality());
Assert.assertEquals(baseCardinality, indexer.globalDictionary.getCardinality());
Assert.assertFalse(indexer.hasNulls);
Assert.assertTrue(indexer.hasNestedData);

View File

@ -143,6 +143,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vd0-nvl-2", "nvl(vd0, 2.0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vd0-add-sub", "d0 + (d0 - d0)", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vf0-add-sub", "f0 + (f0 - f0)", ColumnType.FLOAT, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("vl0-add-sub", "l0 + (l0 - l0)", ColumnType.LONG, TestExprMacroTable.INSTANCE),

View File

@ -816,6 +816,22 @@ public class BoundFilterTest extends BaseFilterTest
? ImmutableList.of("0", "3", "7")
: ImmutableList.of("0")
);
assertFilterMatches(
new BoundDimFilter(
"vd0-nvl-2",
"0",
null,
true,
false,
false,
null,
StringComparators.NUMERIC
),
NullHandling.replaceWithDefault()
? ImmutableList.of("1", "3", "4", "5", "6")
: ImmutableList.of("1", "2", "3", "4", "5", "6", "7")
);
}
@Test

View File

@ -229,6 +229,12 @@ public class NullFilterTests
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(NullFilter.forColumn("vd0-nvl-2"), ImmutableList.of());
assertFilterMatches(
NotDimFilter.of(NullFilter.forColumn("vd0-nvl-2")),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(NullFilter.forColumn("vf0-add-sub"), ImmutableList.of());
assertFilterMatches(
NotDimFilter.of(NullFilter.forColumn("vf0-add-sub")),
@ -274,6 +280,12 @@ public class NullFilterTests
assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of("3"));
assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vl0")), ImmutableList.of("0", "1", "2", "4", "5"));
assertFilterMatches(NullFilter.forColumn("vd0-nvl-2"), ImmutableList.of());
assertFilterMatches(
NotDimFilter.of(NullFilter.forColumn("vd0-nvl-2")),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
if (NullHandling.sqlCompatible()) {
// these fail in default value mode that cannot be tested as numeric default values becuase of type
// mismatch for subtract operation

View File

@ -895,6 +895,21 @@ public class RangeFilterTests
: ImmutableList.of("0")
);
assertFilterMatches(
new RangeFilter(
"vd0-nvl-2",
ColumnType.DOUBLE,
0.0,
null,
true,
false,
null
),
NullHandling.replaceWithDefault()
? ImmutableList.of("1", "3", "4", "5", "6")
: ImmutableList.of("1", "2", "3", "4", "5", "6", "7")
);
if (NullHandling.sqlCompatible() || canTestNumericNullsAsDefaultValues) {
// these fail in default value mode that cannot be tested as numeric default values becuase of type
// mismatch for subtract operation

View File

@ -40,12 +40,18 @@ import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.FilterTuning;
import org.apache.druid.query.filter.SpatialDimFilter;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMerger;
@ -54,6 +60,9 @@ import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.StringEncodingStrategy;
import org.apache.druid.segment.data.FrontCodedIndexed;
import org.apache.druid.segment.incremental.IncrementalIndex;
@ -62,28 +71,31 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Random;
import java.util.concurrent.ThreadLocalRandom;
/**
*
*/
@RunWith(Parameterized.class)
public class SpatialFilterTest extends InitializedNullHandlingTest
{
public static final int NUM_POINTS = 5000;
private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance());
private static IndexIO INDEX_IO = TestHelper.getTestIndexIO();
public static final int NUM_POINTS = 5000;
private static Interval DATA_INTERVAL = Intervals.of("2013-01-01/2013-01-07");
private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{
@ -92,6 +104,12 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
};
private static List<String> DIMS = Lists.newArrayList("dim", "lat", "long", "lat2", "long2");
private final Segment segment;
public SpatialFilterTest(Segment segment)
{
this.segment = segment;
}
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
@ -517,7 +535,11 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
QueryableIndex mergedRealtime = INDEX_IO.loadIndex(
INDEX_MERGER.mergeQueryableIndex(
Arrays.asList(INDEX_IO.loadIndex(firstFile), INDEX_IO.loadIndex(secondFile), INDEX_IO.loadIndex(thirdFile)),
Arrays.asList(
INDEX_IO.loadIndex(firstFile),
INDEX_IO.loadIndex(secondFile),
INDEX_IO.loadIndex(thirdFile)
),
true,
METRIC_AGGS,
mergedFile,
@ -534,13 +556,6 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
}
}
private final Segment segment;
public SpatialFilterTest(Segment segment)
{
this.segment = segment;
}
@Test
public void testSpatialQuery()
{
@ -567,9 +582,9 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
DateTimes.of("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 3L)
.put("val", 59L)
.build()
.put("rows", 3L)
.put("val", 59L)
.build()
)
)
);
@ -619,9 +634,9 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
DateTimes.of("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 13L)
.build()
.put("rows", 1L)
.put("val", 13L)
.build()
)
)
);
@ -742,4 +757,48 @@ public class SpatialFilterTest extends InitializedNullHandlingTest
{
EqualsVerifier.forClass(SpatialFilter.BoundDruidPredicateFactory.class).usingGetClass().verify();
}
@Test
public void testSpatialFilter()
{
SpatialFilter spatialFilter = new SpatialFilter(
"test",
new RadiusBound(new float[]{0, 0}, 0f, 0),
new FilterTuning(false, 1, 1)
);
// String complex
Assert.assertTrue(spatialFilter.makeMatcher(new TestSpatialSelectorFactory("0,0")).matches(true));
// Unknown complex, invokes object predicate
Assert.assertFalse(spatialFilter.makeMatcher(new TestSpatialSelectorFactory(new Date())).matches(true));
Assert.assertFalse(spatialFilter.makeMatcher(new TestSpatialSelectorFactory(new Object())).matches(true));
}
static class TestSpatialSelectorFactory implements ColumnSelectorFactory
{
Object object;
public TestSpatialSelectorFactory(Object value)
{
object = value;
}
@Override
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
{
return null;
}
@Override
public ColumnValueSelector makeColumnValueSelector(String columnName)
{
return new TestObjectColumnSelector(new Object[]{object});
}
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.UNKNOWN_COMPLEX);
}
}
}

View File

@ -93,4 +93,14 @@ public interface Supervisor
LagStats computeLagStats();
int getActiveTaskGroupsCount();
/**
* Marks the given task groups as ready for segment hand-off irrespective of the task run times.
* In the subsequent run, the supervisor initiates segment publish and hand-off for these task groups and rolls over their tasks.
* taskGroupIds that are not valid or not actively reading are simply ignored.
*/
default void handoffTaskGroupsEarly(List<Integer> taskGroupIds)
{
throw new UnsupportedOperationException("Supervisor does not have the feature to handoff task groups early implemented");
}
}

View File

@ -23,14 +23,13 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.druid.client.indexing.TaskStatusResponse;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.rpc.ServiceLocation;
@ -57,7 +56,6 @@ public class SpecificTaskServiceLocator implements ServiceLocator
private final String taskId;
private final OverlordClient overlordClient;
private final TaskLocationFetcher locationFetcher = new TaskLocationFetcher();
private final Object lock = new Object();
@GuardedBy("lock")
@ -125,42 +123,15 @@ public class SpecificTaskServiceLocator implements ServiceLocator
lastUpdateTime = System.currentTimeMillis();
final TaskStatus status = taskStatusMap.get(taskId);
if (status == null) {
// If the task status is unknown, we'll treat it as closed.
lastKnownState = null;
lastKnownLocation = null;
resolvePendingFuture(null, null);
} else if (TaskLocation.unknown().equals(status.getLocation())) {
// Do not resolve the future just yet, try the fallback API instead
fetchFallbackTaskLocation();
} else {
lastKnownState = status.getStatusCode();
final TaskLocation location;
if (TaskLocation.unknown().equals(status.getLocation())) {
location = locationFetcher.getLocation();
} else {
location = status.getLocation();
}
if (TaskLocation.unknown().equals(location)) {
lastKnownLocation = null;
} else {
lastKnownLocation = new ServiceLocation(
location.getHost(),
location.getPort(),
location.getTlsPort(),
StringUtils.format("%s/%s", BASE_PATH, StringUtils.urlEncode(taskId))
);
}
resolvePendingFuture(status.getStatusCode(), status.getLocation());
}
if (lastKnownState != TaskState.RUNNING) {
pendingFuture.set(ServiceLocations.closed());
} else if (lastKnownLocation == null) {
pendingFuture.set(ServiceLocations.forLocations(Collections.emptySet()));
} else {
pendingFuture.set(ServiceLocations.forLocation(lastKnownLocation));
}
// Clear pendingFuture once it has been set.
pendingFuture = null;
}
}
}
@ -168,17 +139,10 @@ public class SpecificTaskServiceLocator implements ServiceLocator
@Override
public void onFailure(Throwable t)
{
synchronized (lock) {
if (pendingFuture != null) {
pendingFuture.setException(t);
// Clear pendingFuture once it has been set.
pendingFuture = null;
}
}
resolvePendingFutureOnException(t);
}
},
MoreExecutors.directExecutor()
Execs.directExecutor()
);
return Futures.nonCancellationPropagating(retVal);
@ -209,18 +173,104 @@ public class SpecificTaskServiceLocator implements ServiceLocator
}
}
private class TaskLocationFetcher
private void resolvePendingFuture(TaskState state, TaskLocation location)
{
TaskLocation getLocation()
{
final TaskStatusResponse statusResponse = FutureUtils.getUnchecked(
overlordClient.taskStatus(taskId),
true
);
if (statusResponse == null || statusResponse.getStatus() == null) {
return TaskLocation.unknown();
} else {
return statusResponse.getStatus().getLocation();
synchronized (lock) {
if (pendingFuture != null) {
lastKnownState = state;
lastKnownLocation = location == null ? null : new ServiceLocation(
location.getHost(),
location.getPort(),
location.getTlsPort(),
StringUtils.format("%s/%s", BASE_PATH, StringUtils.urlEncode(taskId))
);
if (lastKnownState != TaskState.RUNNING) {
pendingFuture.set(ServiceLocations.closed());
} else if (lastKnownLocation == null) {
pendingFuture.set(ServiceLocations.forLocations(Collections.emptySet()));
} else {
pendingFuture.set(ServiceLocations.forLocation(lastKnownLocation));
}
// Clear pendingFuture once it has been set.
pendingFuture = null;
}
}
}
private void resolvePendingFutureOnException(Throwable t)
{
synchronized (lock) {
if (pendingFuture != null) {
pendingFuture.setException(t);
// Clear pendingFuture once it has been set.
pendingFuture = null;
}
}
}
/**
* Invokes the single task status API {@link OverlordClient#taskStatus} if the
* multi-task status API returns an unknown location (this can happen if the
* Overlord is running on a version older than Druid 30.0.0 (pre #15724)).
*/
private void fetchFallbackTaskLocation()
{
synchronized (lock) {
if (pendingFuture != null) {
final ListenableFuture<TaskStatusResponse> taskStatusFuture;
try {
taskStatusFuture = overlordClient.taskStatus(taskId);
}
catch (Exception e) {
resolvePendingFutureOnException(e);
return;
}
pendingFuture.addListener(
() -> {
if (!taskStatusFuture.isDone()) {
// pendingFuture may resolve without taskStatusFuture due to close().
taskStatusFuture.cancel(true);
}
},
Execs.directExecutor()
);
Futures.addCallback(
taskStatusFuture,
new FutureCallback<TaskStatusResponse>()
{
@Override
public void onSuccess(final TaskStatusResponse taskStatusResponse)
{
synchronized (lock) {
if (pendingFuture != null) {
lastUpdateTime = System.currentTimeMillis();
final TaskStatusPlus status = taskStatusResponse.getStatus();
if (status == null) {
// If the task status is unknown, we'll treat it as closed.
resolvePendingFuture(null, null);
} else if (TaskLocation.unknown().equals(status.getLocation())) {
resolvePendingFuture(status.getStatusCode(), null);
} else {
resolvePendingFuture(status.getStatusCode(), status.getLocation());
}
}
}
}
@Override
public void onFailure(Throwable t)
{
resolvePendingFutureOnException(t);
}
},
Execs.directExecutor()
);
}
}
}

View File

@ -465,6 +465,7 @@ public class StreamAppenderator implements Appenderator
MoreExecutors.directExecutor()
);
} else {
log.info("Marking ready for non-incremental async persist due to reasons[%s].", persistReasons);
isPersistRequired = true;
}
}

View File

@ -22,6 +22,11 @@ package org.apache.druid.server.lookup.cache;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.error.InvalidInput;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
/**
@ -39,6 +44,10 @@ import java.util.Set;
*/
public class LookupLoadingSpec
{
public static final String CTX_LOOKUP_LOADING_MODE = "lookupLoadingMode";
public static final String CTX_LOOKUPS_TO_LOAD = "lookupsToLoad";
public enum Mode
{
ALL, NONE, ONLY_REQUIRED
@ -80,6 +89,50 @@ public class LookupLoadingSpec
return lookupsToLoad;
}
public static LookupLoadingSpec createFromContext(Map<String, Object> context, LookupLoadingSpec defaultSpec)
{
if (context == null) {
return defaultSpec;
}
final Object lookupModeValue = context.get(CTX_LOOKUP_LOADING_MODE);
if (lookupModeValue == null) {
return defaultSpec;
}
final LookupLoadingSpec.Mode lookupLoadingMode;
try {
lookupLoadingMode = LookupLoadingSpec.Mode.valueOf(lookupModeValue.toString());
}
catch (IllegalArgumentException e) {
throw InvalidInput.exception("Invalid value of %s[%s]. Allowed values are %s",
CTX_LOOKUP_LOADING_MODE, lookupModeValue.toString(), Arrays.asList(LookupLoadingSpec.Mode.values()));
}
if (lookupLoadingMode == Mode.NONE) {
return NONE;
} else if (lookupLoadingMode == Mode.ALL) {
return ALL;
} else if (lookupLoadingMode == Mode.ONLY_REQUIRED) {
Collection<String> lookupsToLoad;
try {
lookupsToLoad = (Collection<String>) context.get(CTX_LOOKUPS_TO_LOAD);
}
catch (ClassCastException e) {
throw InvalidInput.exception("Invalid value of %s[%s]. Please provide a comma-separated list of "
+ "lookup names. For example: [\"lookupName1\", \"lookupName2\"]",
CTX_LOOKUPS_TO_LOAD, context.get(CTX_LOOKUPS_TO_LOAD));
}
if (lookupsToLoad == null || lookupsToLoad.isEmpty()) {
throw InvalidInput.exception("Set of lookups to load cannot be %s for mode[ONLY_REQUIRED].", lookupsToLoad);
}
return LookupLoadingSpec.loadOnly(new HashSet<>(lookupsToLoad));
} else {
return defaultSpec;
}
}
@Override
public String toString()
{
@ -88,4 +141,23 @@ public class LookupLoadingSpec
", lookupsToLoad=" + lookupsToLoad +
'}';
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LookupLoadingSpec that = (LookupLoadingSpec) o;
return mode == that.mode && Objects.equals(lookupsToLoad, that.lookupsToLoad);
}
@Override
public int hashCode()
{
return Objects.hash(mode, lookupsToLoad);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.indexing;
import com.google.common.collect.ImmutableList;
import org.apache.druid.indexing.overlord.ObjectMetadata;
import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
@ -89,4 +90,14 @@ public class NoopSupervisorSpecTest
Assert.assertEquals(-1, noOpSupervisor.getActiveTaskGroupsCount());
Assert.assertEquals(SupervisorStateManager.BasicState.RUNNING, noOpSupervisor.getState());
}
@Test
public void testNoppSupervisorStopTaskEarlyDoNothing()
{
NoopSupervisorSpec expectedSpec = new NoopSupervisorSpec(null, null);
Supervisor noOpSupervisor = expectedSpec.createSupervisor();
Assert.assertThrows(UnsupportedOperationException.class,
() -> noOpSupervisor.handoffTaskGroupsEarly(ImmutableList.of())
);
}
}

View File

@ -36,6 +36,7 @@ import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.segment.FrameBasedInlineSegmentWrangler;
import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.InlineSegmentWrangler;
import org.apache.druid.segment.LookupSegmentWrangler;
import org.apache.druid.segment.MapSegmentWrangler;
@ -44,6 +45,7 @@ import org.apache.druid.segment.QueryableIndexSegment;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.join.JoinableFactory;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.initialization.ServerConfig;
@ -196,6 +198,11 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
return add(descriptor, new QueryableIndexSegment(index, descriptor.getId()));
}
public SpecificSegmentsQuerySegmentWalker add(final DataSegment descriptor, final IncrementalIndex index)
{
return add(descriptor, new IncrementalIndexSegment(index, descriptor.getId()));
}
public List<DataSegment> getSegments()
{
return segments;

View File

@ -19,13 +19,20 @@
package org.apache.druid.server.lookup.cache;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import junitparams.JUnitParamsRunner;
import junitparams.Parameters;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.StringUtils;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import java.util.Arrays;
import java.util.Set;
@RunWith(JUnitParamsRunner.class)
public class LookupLoadingSpecTest
{
@Test
@ -59,4 +66,121 @@ public class LookupLoadingSpecTest
DruidException exception = Assert.assertThrows(DruidException.class, () -> LookupLoadingSpec.loadOnly(null));
Assert.assertEquals("Expected non-null set of lookups to load.", exception.getMessage());
}
@Test
public void testCreateLookupLoadingSpecFromEmptyContext()
{
// Default spec is returned in the case of context not having the lookup keys.
Assert.assertEquals(
LookupLoadingSpec.ALL,
LookupLoadingSpec.createFromContext(
ImmutableMap.of(),
LookupLoadingSpec.ALL
)
);
Assert.assertEquals(
LookupLoadingSpec.NONE,
LookupLoadingSpec.createFromContext(
ImmutableMap.of(),
LookupLoadingSpec.NONE
)
);
}
@Test
public void testCreateLookupLoadingSpecFromNullContext()
{
// Default spec is returned in the case of context=null.
Assert.assertEquals(
LookupLoadingSpec.NONE,
LookupLoadingSpec.createFromContext(
null,
LookupLoadingSpec.NONE
)
);
Assert.assertEquals(
LookupLoadingSpec.ALL,
LookupLoadingSpec.createFromContext(
null,
LookupLoadingSpec.ALL
)
);
}
@Test
public void testCreateLookupLoadingSpecFromContext()
{
// Only required lookups are returned in the case of context having the lookup keys.
Assert.assertEquals(
LookupLoadingSpec.loadOnly(ImmutableSet.of("lookup1", "lookup2")),
LookupLoadingSpec.createFromContext(
ImmutableMap.of(
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookup1", "lookup2"),
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED
),
LookupLoadingSpec.ALL
)
);
// No lookups are returned in the case of context having mode=NONE, irrespective of the default spec.
Assert.assertEquals(
LookupLoadingSpec.NONE,
LookupLoadingSpec.createFromContext(
ImmutableMap.of(
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE),
LookupLoadingSpec.ALL
)
);
// All lookups are returned in the case of context having mode=ALL, irrespective of the default spec.
Assert.assertEquals(
LookupLoadingSpec.ALL,
LookupLoadingSpec.createFromContext(
ImmutableMap.of(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ALL),
LookupLoadingSpec.NONE
)
);
}
@Test
@Parameters(
{
"NONE1",
"A",
"Random mode",
"all",
"only required",
"none"
}
)
public void testCreateLookupLoadingSpecFromInvalidModeInContext(String mode)
{
final DruidException exception = Assert.assertThrows(DruidException.class, () -> LookupLoadingSpec.createFromContext(
ImmutableMap.of(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, mode), LookupLoadingSpec.ALL));
Assert.assertEquals(StringUtils.format("Invalid value of %s[%s]. Allowed values are [ALL, NONE, ONLY_REQUIRED]",
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, mode), exception.getMessage());
}
@Test
@Parameters(
{
"foo bar",
"foo]"
}
)
public void testCreateLookupLoadingSpecFromInvalidLookupsInContext(Object lookupsToLoad)
{
final DruidException exception = Assert.assertThrows(DruidException.class, () ->
LookupLoadingSpec.createFromContext(
ImmutableMap.of(
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, lookupsToLoad,
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED),
LookupLoadingSpec.ALL)
);
Assert.assertEquals(StringUtils.format("Invalid value of %s[%s]. Please provide a comma-separated list of "
+ "lookup names. For example: [\"lookupName1\", \"lookupName2\"]",
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, lookupsToLoad), exception.getMessage());
}
}

View File

@ -0,0 +1,45 @@
/*
* 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.sql.calcite.expression.builtin;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.InferTypes;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.sql.calcite.expression.DirectOperatorConversion;
import org.apache.druid.sql.calcite.expression.OperatorConversions;
public class DivOperatorConversion extends DirectOperatorConversion
{
private static final SqlOperator SQL_OPERATOR =
OperatorConversions
.operatorBuilder("DIV")
.operandTypeChecker(OperandTypes.DIVISION_OPERATOR)
.operandTypeInference(InferTypes.FIRST_KNOWN)
.returnTypeCascadeNullable(SqlTypeName.BIGINT)
.functionCategory(SqlFunctionCategory.NUMERIC)
.build();
public DivOperatorConversion()
{
super(SQL_OPERATOR, "div");
}
}

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.rex.RexLiteral;
@ -48,6 +49,7 @@ import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@ -82,7 +84,7 @@ public class SearchOperatorConversion implements SqlOperatorConversion
plannerContext,
rowSignature,
virtualColumnRegistry,
expandSearch((RexCall) rexNode, REX_BUILDER)
expandSearch((RexCall) rexNode, REX_BUILDER, plannerContext.queryContext().getInFunctionThreshold())
);
}
@ -97,7 +99,7 @@ public class SearchOperatorConversion implements SqlOperatorConversion
return Expressions.toDruidExpression(
plannerContext,
rowSignature,
expandSearch((RexCall) rexNode, REX_BUILDER)
expandSearch((RexCall) rexNode, REX_BUILDER, plannerContext.queryContext().getInFunctionExprThreshold())
);
}
@ -111,7 +113,8 @@ public class SearchOperatorConversion implements SqlOperatorConversion
*/
public static RexNode expandSearch(
final RexCall call,
final RexBuilder rexBuilder
final RexBuilder rexBuilder,
final int scalarInArrayThreshold
)
{
final RexNode arg = call.operands.get(0);
@ -139,13 +142,10 @@ public class SearchOperatorConversion implements SqlOperatorConversion
notInPoints = getPoints(complement);
notInRexNode = makeIn(
arg,
ImmutableList.copyOf(
Iterables.transform(
notInPoints,
point -> rexBuilder.makeLiteral(point, sargRex.getType(), true, true)
)
),
notInPoints,
sargRex.getType(),
true,
notInPoints.size() >= scalarInArrayThreshold,
rexBuilder
);
}
@ -155,13 +155,10 @@ public class SearchOperatorConversion implements SqlOperatorConversion
sarg.pointCount == 0 ? Collections.emptyList() : (List<Comparable>) getPoints(sarg.rangeSet);
final RexNode inRexNode = makeIn(
arg,
ImmutableList.copyOf(
Iterables.transform(
inPoints,
point -> rexBuilder.makeLiteral(point, sargRex.getType(), true, true)
)
),
inPoints,
sargRex.getType(),
false,
inPoints.size() >= scalarInArrayThreshold,
rexBuilder
);
if (inRexNode != null) {
@ -225,14 +222,36 @@ public class SearchOperatorConversion implements SqlOperatorConversion
return retVal;
}
/**
* Make an IN condition for an "arg" matching certain "points", as in "arg IN (points)".
*
* @param arg lhs of the IN
* @param pointObjects rhs of the IN. Must match the "pointType"
* @param pointType type of "pointObjects"
* @param negate true for NOT IN, false for IN
* @param useScalarInArray if true, use {@link ScalarInArrayOperatorConversion#SQL_FUNCTION} when there is more
* than one point; if false, use a stack of ORs
* @param rexBuilder rex builder
*
* @return SQL rex nodes equivalent to the IN filter, or null if "pointObjects" is empty
*/
@Nullable
public static RexNode makeIn(
final RexNode arg,
final List<RexNode> points,
final Collection<? extends Comparable> pointObjects,
final RelDataType pointType,
final boolean negate,
final boolean useScalarInArray,
final RexBuilder rexBuilder
)
{
final List<RexNode> points = ImmutableList.copyOf(
Iterables.transform(
pointObjects,
point -> rexBuilder.makeLiteral(point, pointType, false, false)
)
);
if (points.isEmpty()) {
return null;
} else if (points.size() == 1) {
@ -244,22 +263,33 @@ public class SearchOperatorConversion implements SqlOperatorConversion
return rexBuilder.makeCall(negate ? SqlStdOperatorTable.NOT_EQUALS : SqlStdOperatorTable.EQUALS, arg, point);
}
} else {
// x = a || x = b || x = c ...
RexNode retVal = rexBuilder.makeCall(
SqlStdOperatorTable.OR,
ImmutableList.copyOf(
Iterables.transform(
points,
point -> {
if (RexUtil.isNullLiteral(point, true)) {
return rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, arg);
} else {
return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, arg, point);
RexNode retVal;
if (useScalarInArray) {
// SCALAR_IN_ARRAY(x, ARRAY[a, b, c])
retVal = rexBuilder.makeCall(
ScalarInArrayOperatorConversion.SQL_FUNCTION,
arg,
rexBuilder.makeCall(SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, points)
);
} else {
// x = a || x = b || x = c ...
retVal = rexBuilder.makeCall(
SqlStdOperatorTable.OR,
ImmutableList.copyOf(
Iterables.transform(
points,
point -> {
if (RexUtil.isNullLiteral(point, true)) {
return rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, arg);
} else {
return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, arg, point);
}
}
}
)
)
);
)
)
);
}
if (negate) {
retVal = rexBuilder.makeCall(SqlStdOperatorTable.NOT, retVal);

View File

@ -81,6 +81,7 @@ import org.apache.druid.sql.calcite.expression.builtin.ConcatOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.ContainsOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.DecodeBase64UTFOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.DivOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.ExtractOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.FloorOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.GreatestOperatorConversion;
@ -369,6 +370,7 @@ public class DruidOperatorTable implements SqlOperatorTable
.add(new AliasedOperatorConversion(CHARACTER_LENGTH_CONVERSION, "STRLEN"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.CONCAT, "concat"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.EXP, "exp"))
.add(new DivOperatorConversion())
.add(new DirectOperatorConversion(SqlStdOperatorTable.DIVIDE_INTEGER, "div"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LN, "log"))
.add(new DirectOperatorConversion(SqlStdOperatorTable.LOWER, "lower"))

View File

@ -60,10 +60,12 @@ import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.sql.calcite.expression.builtin.ScalarInArrayOperatorConversion;
import org.apache.druid.sql.calcite.parser.DruidSqlIngest;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.parser.DruidSqlParserUtils;
@ -532,9 +534,12 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
);
}
}
if (tableMetadata == null) {
final boolean isCatalogValidationEnabled = plannerContext.queryContext().isCatalogValidationEnabled();
if (tableMetadata == null || !isCatalogValidationEnabled) {
return sourceType;
}
// disable sealed mode validation if catalog validation is disabled.
final boolean isStrict = tableMetadata.isSealed();
final List<Map.Entry<String, RelDataType>> fields = new ArrayList<>();
for (RelDataTypeField sourceField : sourceFields) {
@ -590,6 +595,8 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
// matches above.
final RelDataType targetType = typeFactory.createStructType(fields);
final SqlValidatorTable target = insertNs.resolve().getTable();
// disable type checking if catalog validation is disabled.
checkTypeAssignment(scope, target, sourceType, targetType, insert);
return targetType;
}
@ -774,6 +781,59 @@ public class DruidSqlValidator extends BaseDruidSqlValidator
super.validateCall(call, scope);
}
@Override
protected SqlNode performUnconditionalRewrites(SqlNode node, final boolean underFrom)
{
if (node != null && (node.getKind() == SqlKind.IN || node.getKind() == SqlKind.NOT_IN)) {
final SqlNode rewritten = rewriteInToScalarInArrayIfNeeded((SqlCall) node, underFrom);
//noinspection ObjectEquality
if (rewritten != node) {
return rewritten;
}
}
return super.performUnconditionalRewrites(node, underFrom);
}
/**
* Rewrites "x IN (values)" to "SCALAR_IN_ARRAY(x, values)", if appropriate. Checks the form of the IN and checks
* the value of {@link QueryContext#getInFunctionThreshold()}.
*
* @param call call to {@link SqlKind#IN} or {@link SqlKind#NOT_IN}
* @param underFrom underFrom arg from {@link #performUnconditionalRewrites(SqlNode, boolean)}, used for
* recursive calls
*
* @return rewritten call, or the original call if no rewrite was appropriate
*/
private SqlNode rewriteInToScalarInArrayIfNeeded(final SqlCall call, final boolean underFrom)
{
if (call.getOperandList().size() == 2 && call.getOperandList().get(1) instanceof SqlNodeList) {
// expr IN (values)
final SqlNode exprNode = call.getOperandList().get(0);
final SqlNodeList valuesNode = (SqlNodeList) call.getOperandList().get(1);
// Confirm valuesNode is big enough to convert to SCALAR_IN_ARRAY, and references only nonnull literals.
// (Can't include NULL literals in the conversion, because SCALAR_IN_ARRAY matches NULLs as if they were regular
// values, whereas IN does not.)
if (valuesNode.size() > plannerContext.queryContext().getInFunctionThreshold()
&& valuesNode.stream().allMatch(node -> node.getKind() == SqlKind.LITERAL && !SqlUtil.isNull(node))) {
final SqlCall newCall = ScalarInArrayOperatorConversion.SQL_FUNCTION.createCall(
call.getParserPosition(),
performUnconditionalRewrites(exprNode, underFrom),
SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(valuesNode)
);
if (call.getKind() == SqlKind.NOT_IN) {
return SqlStdOperatorTable.NOT.createCall(call.getParserPosition(), newCall);
} else {
return newCall;
}
}
}
return call;
}
public static CalciteContextException buildCalciteContextException(String message, SqlNode call)
{
return buildCalciteContextException(new CalciteException(message, null), message, call);

View File

@ -20,7 +20,7 @@
package org.apache.druid.sql.calcite.planner;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.QueryContexts;
import org.joda.time.DateTimeZone;
@ -431,6 +431,11 @@ public class PlannerConfig
}
}
/**
* Translates {@link PlannerConfig} settings into its equivalent QueryContext map.
*
* @throws DruidException if the translation is not possible.
*/
public Map<String, Object> getNonDefaultAsQueryContext()
{
Map<String, Object> overrides = new HashMap<>();
@ -450,7 +455,11 @@ public class PlannerConfig
PlannerConfig newConfig = PlannerConfig.builder().withOverrides(overrides).build();
if (!equals(newConfig)) {
throw new IAE("Some configs are not handled in this method or not persistable as QueryContext keys!\nold: %s\nnew: %s", this, newConfig);
throw DruidException.defensive(
"Not all PlannerConfig options are not persistable as QueryContext keys!\nold: %s\nnew: %s",
this,
newConfig
);
}
return overrides;
}

View File

@ -80,8 +80,6 @@ public class PlannerContext
public static final String CTX_SQL_CURRENT_TIMESTAMP = "sqlCurrentTimestamp";
public static final String CTX_SQL_TIME_ZONE = "sqlTimeZone";
public static final String CTX_SQL_JOIN_ALGORITHM = "sqlJoinAlgorithm";
public static final String CTX_LOOKUP_LOADING_MODE = "lookupLoadingMode";
public static final String CTX_LOOKUPS_TO_LOAD = "lookupsToLoad";
private static final JoinAlgorithm DEFAULT_SQL_JOIN_ALGORITHM = JoinAlgorithm.BROADCAST;
/**
@ -357,7 +355,7 @@ public class PlannerContext
}
/**
* Returns the lookup to load for a given task.
* Lookup loading spec used if this context corresponds to an MSQ task.
*/
public LookupLoadingSpec getLookupLoadingSpec()
{

View File

@ -46,6 +46,7 @@ import org.apache.druid.query.lookup.LookupExtractionFn;
import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringOperatorConversions;
import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.ScalarInArrayOperatorConversion;
import org.apache.druid.sql.calcite.expression.builtin.SearchOperatorConversion;
import org.apache.druid.sql.calcite.filtration.CollectComparisons;
import org.apache.druid.sql.calcite.planner.Calcites;
@ -275,12 +276,16 @@ public class ReverseLookupRule extends RelOptRule implements SubstitutionRule
}
/**
* When we encounter SEARCH, expand it using {@link SearchOperatorConversion#expandSearch(RexCall, RexBuilder)}
* When we encounter SEARCH, expand it using {@link SearchOperatorConversion#expandSearch(RexCall, RexBuilder, int)}
* and continue processing what lies beneath.
*/
private RexNode visitSearch(final RexCall call)
{
final RexNode expanded = SearchOperatorConversion.expandSearch(call, rexBuilder);
final RexNode expanded = SearchOperatorConversion.expandSearch(
call,
rexBuilder,
plannerContext.queryContext().getInFunctionThreshold()
);
if (expanded instanceof RexCall) {
final RexNode converted = visitCall((RexCall) expanded);
@ -300,10 +305,17 @@ public class ReverseLookupRule extends RelOptRule implements SubstitutionRule
*/
private RexNode visitComparison(final RexCall call)
{
return CollectionUtils.getOnlyElement(
final RexNode retVal = CollectionUtils.getOnlyElement(
new CollectReverseLookups(Collections.singletonList(call), rexBuilder).collect(),
ret -> new ISE("Expected to collect single node, got[%s]", ret)
);
//noinspection ObjectEquality
if (retVal != call) {
return retVal;
} else {
return super.visitCall(call);
}
}
/**
@ -398,12 +410,13 @@ public class ReverseLookupRule extends RelOptRule implements SubstitutionRule
return Collections.singleton(null);
} else {
// Compute the set of values that this comparison operator matches.
// Note that MV_CONTAINS and MV_OVERLAP match nulls, but other comparison operators do not.
// Note that MV_CONTAINS, MV_OVERLAP, and SCALAR_IN_ARRAY match nulls, but other comparison operators do not.
// See "isBinaryComparison" for the set of operators we might encounter here.
final RexNode matchLiteral = call.getOperands().get(1);
final boolean matchNulls =
call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator())
|| call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator());
|| call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator())
|| call.getOperator().equals(ScalarInArrayOperatorConversion.SQL_FUNCTION);
return toStringSet(matchLiteral, matchNulls);
}
}
@ -559,8 +572,16 @@ public class ReverseLookupRule extends RelOptRule implements SubstitutionRule
} else {
return SearchOperatorConversion.makeIn(
reverseLookupKey.arg,
stringsToRexNodes(reversedMatchValues, rexBuilder),
reversedMatchValues,
rexBuilder.getTypeFactory()
.createTypeWithNullability(
rexBuilder.getTypeFactory().createSqlType(SqlTypeName.VARCHAR),
true
),
reverseLookupKey.negate,
// Use regular equals, or SCALAR_IN_ARRAY, depending on inFunctionThreshold.
reversedMatchValues.size() >= plannerContext.queryContext().getInFunctionThreshold(),
rexBuilder
);
}
@ -598,7 +619,8 @@ public class ReverseLookupRule extends RelOptRule implements SubstitutionRule
return call.getKind() == SqlKind.EQUALS
|| call.getKind() == SqlKind.NOT_EQUALS
|| call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator())
|| call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator());
|| call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator())
|| call.getOperator().equals(ScalarInArrayOperatorConversion.SQL_FUNCTION);
} else {
return false;
}

View File

@ -36,6 +36,7 @@ import org.apache.druid.error.DruidException;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
@ -56,9 +57,20 @@ import org.junit.jupiter.api.Test;
import javax.annotation.Nullable;
import java.util.HashMap;
import java.util.Map;
@SqlTestFrameworkConfig.ComponentSupplier(CatalogIngestionDmlComponentSupplier.class)
public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDmlTest
{
private static final Map<String, Object> CONTEXT_WITH_VALIDATION_DISABLED;
static {
CONTEXT_WITH_VALIDATION_DISABLED = new HashMap<>(DEFAULT_CONTEXT);
CONTEXT_WITH_VALIDATION_DISABLED.put(QueryContexts.CATALOG_VALIDATION_ENABLED, false);
}
private final String operationName;
private final String dmlPrefixPattern;
@ -919,7 +931,7 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
" b AS dim1,\n" +
" 1 AS cnt,\n" +
" c AS m2,\n" +
" CAST(d AS BIGINT) AS extra2\n" +
" d AS extra2\n" +
"FROM TABLE(inline(\n" +
" data => ARRAY['2022-12-26T12:34:56,extra,10,\"20\",foo'],\n" +
" format => 'csv'))\n" +
@ -933,6 +945,65 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
.verify();
}
/**
* Adding a new column during ingestion that is not defined in a sealed table, when catalog validation is disabled,
* should plan accordingly.
*/
@Test
public void testInsertAddNonDefinedColumnIntoSealedCatalogTableAndValidationDisabled()
{
ExternalDataSource externalDataSource = new ExternalDataSource(
new InlineInputSource("2022-12-26T12:34:56,extra,10,\"20\",foo\n"),
new CsvInputFormat(ImmutableList.of("a", "b", "c", "d", "e"), null, false, false, 0),
RowSignature.builder()
.add("a", ColumnType.STRING)
.add("b", ColumnType.STRING)
.add("c", ColumnType.LONG)
.add("d", ColumnType.STRING)
.add("e", ColumnType.STRING)
.build()
);
final RowSignature signature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("dim1", ColumnType.STRING)
.add("cnt", ColumnType.LONG)
.add("m2", ColumnType.LONG)
.add("extra2", ColumnType.STRING)
.build();
testIngestionQuery()
.context(CONTEXT_WITH_VALIDATION_DISABLED)
.sql(StringUtils.format(dmlPrefixPattern, "fooSealed") + "\n" +
"SELECT\n" +
" TIME_PARSE(a) AS __time,\n" +
" b AS dim1,\n" +
" 1 AS cnt,\n" +
" c AS m2,\n" +
" d AS extra2\n" +
"FROM TABLE(inline(\n" +
" data => ARRAY['2022-12-26T12:34:56,extra,10,\"20\",foo'],\n" +
" format => 'csv'))\n" +
" (a VARCHAR, b VARCHAR, c BIGINT, d VARCHAR, e VARCHAR)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("fooSealed", signature)
.expectResources(dataSourceWrite("fooSealed"), Externals.externalRead("EXTERNAL"))
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "timestamp_parse(\"a\",null,'UTC')", ColumnType.LONG),
expressionVirtualColumn("v1", "1", ColumnType.LONG)
)
// Scan query lists columns in alphabetical order independent of the
// SQL project list or the defined schema.
.columns("b", "c", "d", "v0", "v1")
.context(CalciteIngestionDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.verify();
}
/**
* Inserting into a catalog table with a WITH source succeeds
@ -1104,6 +1175,10 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
.verify();
}
/**
* Assigning a column during ingestion, to an input type that is not compatible with the defined type of the
* column, should result in a proper validation error.
*/
@Test
public void testInsertIntoExistingWithIncompatibleTypeAssignment()
{
@ -1120,6 +1195,48 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
.verify();
}
/**
* Assigning a column during ingestion, to an input type that is not compatible with the defined type of the
* column, when catalog validation is disabled, should plan accordingly.
*/
@Test
public void testInsertIntoExistingWithIncompatibleTypeAssignmentAndValidationDisabled()
{
final RowSignature signature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("dim1", ColumnType.STRING_ARRAY)
.build();
testIngestionQuery()
.context(CONTEXT_WITH_VALIDATION_DISABLED)
.sql(StringUtils.format(dmlPrefixPattern, "foo") + "\n"
+ "SELECT\n"
+ " __time AS __time,\n"
+ " ARRAY[dim1] AS dim1\n"
+ "FROM foo\n"
+ "PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("foo", signature)
.expectResources(dataSourceRead("foo"), dataSourceWrite("foo"))
.expectQuery(
newScanQueryBuilder()
.dataSource("foo")
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "array(\"dim1\")", ColumnType.STRING_ARRAY)
)
// Scan query lists columns in alphabetical order independent of the
// SQL project list or the defined schema.
.columns("__time", "v0")
.context(CalciteIngestionDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.verify();
}
/**
* Assigning a complex type column during ingestion, to an input type that is not compatible with the defined type of
* the column, should result in a proper validation error.
*/
@Test
public void testGroupByInsertIntoExistingWithIncompatibleTypeAssignment()
{
@ -1135,4 +1252,42 @@ public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDml
"Cannot assign to target field 'unique_dim1' of type COMPLEX<hyperUnique> from source field 'unique_dim1' of type VARCHAR ARRAY (line [4], column [3])")
.verify();
}
/**
* Assigning a complex type column during ingestion, to an input type that is not compatible with the defined type of
* the column, when catalog validation is disabled, should plan accordingly.
*/
@Test
public void testGroupByInsertIntoExistingWithIncompatibleTypeAssignmentAndValidationDisabled()
{
final RowSignature signature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("unique_dim1", ColumnType.STRING_ARRAY)
.build();
testIngestionQuery()
.context(CONTEXT_WITH_VALIDATION_DISABLED)
.sql(StringUtils.format(dmlPrefixPattern, "foo") + "\n"
+ "SELECT\n"
+ " __time AS __time,\n"
+ " ARRAY[dim1] AS unique_dim1\n"
+ "FROM foo\n"
+ "PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("foo", signature)
.expectResources(dataSourceRead("foo"), dataSourceWrite("foo"))
.expectQuery(
newScanQueryBuilder()
.dataSource("foo")
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "array(\"dim1\")", ColumnType.STRING_ARRAY)
)
// Scan query lists columns in alphabetical order independent of the
// SQL project list or the defined schema.
.columns("__time", "v0")
.context(CalciteIngestionDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.verify();
}
}

View File

@ -169,6 +169,30 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testFilterScalarInArrayLookupOfConcat()
{
cannotVectorize();
testQuery(
buildFilterTestSql("SCALAR_IN_ARRAY(LOOKUP(CONCAT(dim1, 'a', dim2), 'lookyloo'), ARRAY['xa', 'xabc'])"),
QUERY_CONTEXT,
buildFilterTestExpectedQuery(
or(
and(
equality("dim1", "", ColumnType.STRING),
equality("dim2", "", ColumnType.STRING)
),
and(
equality("dim1", "", ColumnType.STRING),
equality("dim2", "bc", ColumnType.STRING)
)
)
),
ImmutableList.of()
);
}
@Test
public void testFilterConcatOfLookup()
{
@ -378,6 +402,40 @@ public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testFilterScalarInArray()
{
cannotVectorize();
testQuery(
buildFilterTestSql("SCALAR_IN_ARRAY(LOOKUP(dim1, 'lookyloo'), ARRAY['xabc', 'x6', 'nonexistent'])"),
QUERY_CONTEXT,
buildFilterTestExpectedQuery(in("dim1", Arrays.asList("6", "abc"))),
ImmutableList.of(new Object[]{"xabc", 1L})
);
}
@Test
public void testFilterInOverScalarInArrayThreshold()
{
cannotVectorize();
// Set inFunctionThreshold = 1 to cause the IN to be converted to SCALAR_IN_ARRAY.
final ImmutableMap<String, Object> queryContext =
ImmutableMap.<String, Object>builder()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true)
.put(QueryContexts.IN_FUNCTION_THRESHOLD, 1)
.build();
testQuery(
buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent')"),
queryContext,
buildFilterTestExpectedQuery(in("dim1", Arrays.asList("6", "abc"))),
ImmutableList.of(new Object[]{"xabc", 1L})
);
}
@Test
public void testFilterInOverMaxSize()
{

View File

@ -66,6 +66,7 @@ import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
@ -96,6 +97,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
public static final String DATA_SOURCE_MIXED_2 = "nested_mix_2";
public static final String DATA_SOURCE_ARRAYS = "arrays";
public static final String DATA_SOURCE_ALL = "all_auto";
public static final String DATA_SOURCE_ALL_REALTIME = "all_auto_realtime";
public static final List<ImmutableMap<String, Object>> RAW_ROWS = ImmutableList.of(
ImmutableMap.<String, Object>builder()
@ -334,6 +336,30 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.inputTmpDir(tempDirProducer.newTempFolder())
.buildMMappedIndex();
final IncrementalIndex indexAllTypesAutoRealtime =
IndexBuilder.create()
.tmpDir(tempDirProducer.newTempFolder())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec())
.withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec())
.withMetrics(
new CountAggregatorFactory("cnt")
)
.withRollup(false)
.build()
)
.inputSource(
ResourceInputSource.of(
NestedDataTestUtils.class.getClassLoader(),
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE
)
)
.inputFormat(TestDataBuilder.DEFAULT_JSON_INPUT_FORMAT)
.inputTmpDir(tempDirProducer.newTempFolder())
.buildIncrementalIndex();
SpecificSegmentsQuerySegmentWalker walker = SpecificSegmentsQuerySegmentWalker.createWalker(injector, conglomerate);
walker.add(
@ -399,6 +425,15 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.size(0)
.build(),
indexAllTypesAuto
).add(
DataSegment.builder()
.dataSource(DATA_SOURCE_ALL_REALTIME)
.version("1")
.interval(indexAllTypesAutoRealtime.getInterval())
.shardSpec(new LinearShardSpec(1))
.size(0)
.build(),
indexAllTypesAutoRealtime
);
return walker;
@ -7145,4 +7180,392 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
.build()
);
}
@Test
public void testNvlJsonValueDoubleMissingColumn()
{
testQuery(
"SELECT\n"
+ "JSON_VALUE(nest, '$.nonexistent' RETURNING DOUBLE),\n"
+ "NVL(JSON_VALUE(nest, '$.nonexistent' RETURNING DOUBLE), 1.0),\n"
+ "NVL(JSON_VALUE(nest, '$.nonexistent' RETURNING DOUBLE), 1.0) > 0\n"
+ "FROM druid.nested\n"
+ "WHERE NVL(JSON_VALUE(nest, '$.nonexistent' RETURNING DOUBLE), 1.0) > 0\n"
+ "LIMIT 1",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "nvl(\"v1\",1.0)", ColumnType.DOUBLE),
new NestedFieldVirtualColumn(
"nest",
"$.nonexistent",
"v1",
ColumnType.DOUBLE
),
expressionVirtualColumn("v2", "notnull(nvl(\"v1\",1.0))", ColumnType.LONG)
)
.filters(range("v0", ColumnType.LONG, NullHandling.sqlCompatible() ? 0.0 : "0", null, true, false))
.limit(1)
.columns("v0", "v1", "v2")
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(new Object[]{null, 1.0, true})
: ImmutableList.of(),
RowSignature.builder()
.add("EXPR$0", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.DOUBLE)
.add("EXPR$2", ColumnType.LONG)
.build()
);
}
@Test
public void testNvlJsonValueDoubleSometimesMissing()
{
testQuery(
"SELECT\n"
+ "JSON_VALUE(nest, '$.y' RETURNING DOUBLE),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) > 0,\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) = 1.0\n"
+ "FROM druid.nested",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
new NestedFieldVirtualColumn("nest", "$.y", "v0", ColumnType.DOUBLE),
expressionVirtualColumn("v1", "nvl(\"v0\",1.0)", ColumnType.DOUBLE),
expressionVirtualColumn("v2", "(nvl(\"v0\",1.0) > 0)", ColumnType.LONG),
expressionVirtualColumn("v3", "(nvl(\"v0\",1.0) == 1.0)", ColumnType.LONG)
)
.columns("v0", "v1", "v2", "v3")
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{2.02, 2.02, true, false},
new Object[]{null, 1.0, true, true},
new Object[]{3.03, 3.03, true, false},
new Object[]{null, 1.0, true, true},
new Object[]{null, 1.0, true, true},
new Object[]{2.02, 2.02, true, false},
new Object[]{null, 1.0, true, true}
)
: ImmutableList.of(
new Object[]{2.02, 2.02, true, false},
new Object[]{null, 0.0, false, false},
new Object[]{3.03, 3.03, true, false},
new Object[]{null, 0.0, false, false},
new Object[]{null, 0.0, false, false},
new Object[]{2.02, 2.02, true, false},
new Object[]{null, 0.0, false, false}
),
RowSignature.builder()
.add("EXPR$0", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.DOUBLE)
.add("EXPR$2", ColumnType.LONG)
.add("EXPR$3", ColumnType.LONG)
.build()
);
}
@Test
public void testNvlJsonValueDoubleSometimesMissingRangeFilter()
{
testQuery(
"SELECT\n"
+ "JSON_VALUE(nest, '$.y' RETURNING DOUBLE),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) > 0\n"
+ "FROM druid.nested\n"
+ "WHERE NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) > 0",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "nvl(\"v1\",1.0)", ColumnType.DOUBLE),
new NestedFieldVirtualColumn("nest", "$.y", "v1", ColumnType.DOUBLE),
expressionVirtualColumn("v2", "notnull(nvl(\"v1\",1.0))", ColumnType.LONG)
)
.filters(range("v0", ColumnType.LONG, NullHandling.sqlCompatible() ? 0.0 : "0", null, true, false))
.columns("v0", "v1", "v2")
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{2.02, 2.02, true},
new Object[]{null, 1.0, true},
new Object[]{3.03, 3.03, true},
new Object[]{null, 1.0, true},
new Object[]{null, 1.0, true},
new Object[]{2.02, 2.02, true},
new Object[]{null, 1.0, true}
)
: ImmutableList.of(
new Object[]{2.02, 2.02, true},
new Object[]{3.03, 3.03, true},
new Object[]{2.02, 2.02, true}
),
RowSignature.builder()
.add("EXPR$0", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.DOUBLE)
.add("EXPR$2", ColumnType.LONG)
.build()
);
}
@Test
public void testNvlJsonValueDoubleSometimesMissingEqualityFilter()
{
testQuery(
"SELECT\n"
+ "JSON_VALUE(nest, '$.y' RETURNING DOUBLE),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0),\n"
+ "NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) > 0\n"
+ "FROM druid.nested\n"
+ "WHERE NVL(JSON_VALUE(nest, '$.y' RETURNING DOUBLE), 1.0) = 1.0",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(DATA_SOURCE)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "nvl(\"v1\",1.0)", ColumnType.DOUBLE),
new NestedFieldVirtualColumn("nest", "$.y", "v1", ColumnType.DOUBLE),
expressionVirtualColumn("v2", "notnull(nvl(\"v1\",1.0))", ColumnType.LONG)
)
.filters(equality("v0", 1.0, ColumnType.DOUBLE))
.columns("v0", "v1", "v2")
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{null, 1.0, true},
new Object[]{null, 1.0, true},
new Object[]{null, 1.0, true},
new Object[]{null, 1.0, true}
)
: ImmutableList.of(),
RowSignature.builder()
.add("EXPR$0", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.DOUBLE)
.add("EXPR$2", ColumnType.LONG)
.build()
);
}
@Test
public void testGroupByAutoString()
{
final List<Object[]> expected;
if (NullHandling.sqlCompatible()) {
expected = ImmutableList.of(
new Object[]{null, 1L},
new Object[]{"", 1L},
new Object[]{"a", 1L},
new Object[]{"b", 1L},
new Object[]{"c", 1L},
new Object[]{"d", 1L},
new Object[]{"null", 1L}
);
} else {
expected = ImmutableList.of(
new Object[]{NullHandling.defaultStringValue(), 2L},
new Object[]{"a", 1L},
new Object[]{"b", 1L},
new Object[]{"c", 1L},
new Object[]{"d", 1L},
new Object[]{"null", 1L}
);
}
testQuery(
"SELECT "
+ "str, "
+ "SUM(cnt) "
+ "FROM druid.all_auto GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("str", "d0")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("str", ColumnType.STRING)
.add("EXPR$1", ColumnType.LONG)
.build()
);
cannotVectorize();
msqIncompatible();
testQuery(
"SELECT "
+ "str, "
+ "SUM(cnt) "
+ "FROM druid.all_auto_realtime GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL_REALTIME)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("str", "d0")
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("str", ColumnType.STRING)
.add("EXPR$1", ColumnType.LONG)
.build()
);
}
@Test
public void testGroupByAutoLong()
{
final List<Object[]> expected = ImmutableList.of(
new Object[]{NullHandling.defaultLongValue(), 2L},
new Object[]{1L, 1L},
new Object[]{2L, 1L},
new Object[]{3L, 1L},
new Object[]{4L, 1L},
new Object[]{5L, 1L}
);
testQuery(
"SELECT "
+ "long, "
+ "SUM(cnt) "
+ "FROM druid.all_auto GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("long", "d0", ColumnType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("long", ColumnType.LONG)
.add("EXPR$1", ColumnType.LONG)
.build()
);
cannotVectorize();
msqIncompatible();
testQuery(
"SELECT "
+ "long, "
+ "SUM(cnt) "
+ "FROM druid.all_auto_realtime GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL_REALTIME)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("long", "d0", ColumnType.LONG)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("long", ColumnType.LONG)
.add("EXPR$1", ColumnType.LONG)
.build()
);
}
@Test
public void testGroupByAutoDouble()
{
final List<Object[]> expected = ImmutableList.of(
new Object[]{NullHandling.defaultDoubleValue(), 2L},
new Object[]{1.0D, 1L},
new Object[]{2.0D, 1L},
new Object[]{3.3D, 1L},
new Object[]{4.4D, 1L},
new Object[]{5.9D, 1L}
);
testQuery(
"SELECT "
+ "\"double\", "
+ "SUM(cnt) "
+ "FROM druid.all_auto GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("double", "d0", ColumnType.DOUBLE)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("double", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.LONG)
.build()
);
cannotVectorize();
msqIncompatible();
testQuery(
"SELECT "
+ "\"double\", "
+ "SUM(cnt) "
+ "FROM druid.all_auto_realtime GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(DATA_SOURCE_ALL_REALTIME)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec("double", "d0", ColumnType.DOUBLE)
)
)
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
expected,
RowSignature.builder()
.add("double", ColumnType.DOUBLE)
.add("EXPR$1", ColumnType.LONG)
.build()
);
}
}

View File

@ -23,6 +23,8 @@ import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import org.apache.calcite.runtime.CalciteContextException;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
@ -600,6 +602,38 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testDiv()
{
cannotVectorize();
final Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
testQuery(
"select cnt, m1, div(m1, 2), div(cnt+2, cnt+1) from foo",
context,
ImmutableList.of(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.virtualColumns(
expressionVirtualColumn("v0", "div(\"m1\",2)", ColumnType.LONG),
expressionVirtualColumn("v1", "div((\"cnt\" + 2),(\"cnt\" + 1))", ColumnType.LONG)
)
.columns(ImmutableList.of("cnt", "m1", "v0", "v1"))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{1L, 1.0f, 0L, 1L},
new Object[]{1L, 2.0f, 1L, 1L},
new Object[]{1L, 3.0f, 1L, 1L},
new Object[]{1L, 4.0f, 2L, 1L},
new Object[]{1L, 5.0f, 2L, 1L},
new Object[]{1L, 6.0f, 3L, 1L}
)
);
}
@Test
public void testGroupByLimitWrappingOrderByAgg()
{
@ -4039,6 +4073,49 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testGroupingWithNullPlusNonNullInFilter()
{
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL, 'abc')",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(equality("dim1", "abc", ColumnType.STRING))
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(new Object[]{1L})
);
}
@Test
public void testGroupingWithNotNullPlusNonNullInFilter()
{
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM foo WHERE dim1 NOT IN (NULL, 'abc')",
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
InlineDataSource.fromIterable(
ImmutableList.of(new Object[]{0L}),
RowSignature.builder().add("EXPR$0", ColumnType.LONG).build()
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("EXPR$0")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(new Object[]{0L})
);
}
@Test
public void testGroupByNothingWithLiterallyFalseFilter()
@ -5557,6 +5634,46 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testNotInAndIsNotNullFilter()
{
testQuery(
"SELECT dim1, COUNT(*) FROM druid.foo "
+ "WHERE dim1 NOT IN ('ghi', 'abc', 'def') AND dim1 IS NOT NULL "
+ "GROUP BY dim1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(and(
notNull("dim1"),
not(in("dim1", ColumnType.STRING, ImmutableList.of("abc", "def", "ghi")))
))
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{"", 1L},
new Object[]{"1", 1L},
new Object[]{"10.1", 1L},
new Object[]{"2", 1L}
)
: ImmutableList.of(
new Object[]{"1", 1L},
new Object[]{"10.1", 1L},
new Object[]{"2", 1L}
)
);
}
@Test
public void testNotInAndLessThanFilter()
{
@ -5631,6 +5748,279 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testInExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi'), COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"scalar_in_array(\"dim1\",array('abc','def','ghi'))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, 4L},
new Object[]{true, 2L}
)
);
}
@Test
public void testInExpressionBelowThreshold()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi'), COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
QueryContexts.override(QUERY_CONTEXT_DEFAULT, QueryContexts.IN_FUNCTION_EXPR_THRESHOLD, 100),
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"((\"dim1\" == 'abc') || (\"dim1\" == 'def') || (\"dim1\" == 'ghi'))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, 4L},
new Object[]{true, 2L}
)
);
}
@Test
public void testInOrIsNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi') OR dim1 IS NULL, COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"(isnull(\"dim1\") || scalar_in_array(\"dim1\",array('abc','def','ghi')))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, NullHandling.sqlCompatible() ? 4L : 3L},
new Object[]{true, NullHandling.sqlCompatible() ? 2L : 3L}
)
);
}
@Test
public void testNotInOrIsNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT NOT (dim1 IN ('abc', 'def', 'ghi') OR dim1 IS NULL), COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"(notnull(\"dim1\") && (! scalar_in_array(\"dim1\",array('abc','def','ghi'))))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, NullHandling.sqlCompatible() ? 2L : 3L},
new Object[]{true, NullHandling.sqlCompatible() ? 4L : 3L}
)
);
}
@Test
public void testNotInAndIsNotNullExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') AND dim1 IS NOT NULL, COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"(notnull(\"dim1\") && (! scalar_in_array(\"dim1\",array('abc','def','ghi'))))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, NullHandling.sqlCompatible() ? 2L : 3L},
new Object[]{true, NullHandling.sqlCompatible() ? 4L : 3L}
)
);
}
@Test
public void testInOrGreaterThanExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 IN ('abc', 'def', 'ghi') OR dim1 > 'zzz', COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"(scalar_in_array(\"dim1\",array('abc','def','ghi')) || (\"dim1\" > 'zzz'))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, 4L},
new Object[]{true, 2L}
)
);
}
@Test
public void testNotInAndLessThanExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') AND dim1 < 'zzz', COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"((\"dim1\" < 'zzz') && (! scalar_in_array(\"dim1\",array('abc','def','ghi'))))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, 2L},
new Object[]{true, 4L}
)
);
}
@Test
public void testNotInOrEqualToOneOfThemExpression()
{
// Cannot vectorize scalar_in_array expression.
cannotVectorize();
testQuery(
"SELECT dim1 NOT IN ('abc', 'def', 'ghi') OR dim1 = 'def', COUNT(*)\n"
+ "FROM druid.foo\n"
+ "GROUP BY 1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setVirtualColumns(
expressionVirtualColumn(
"v0",
"(! scalar_in_array(\"dim1\",array('abc','ghi')))",
ColumnType.LONG
)
)
.setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
.setAggregatorSpecs(new CountAggregatorFactory("a0"))
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{false, 1L},
new Object[]{true, 5L}
)
);
}
@Test
public void testSqlIsNullToInFilter()
{
@ -5685,14 +6075,32 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
final String elementsString = Joiner.on(",").join(elements.stream().map(s -> "'" + s + "'").iterator());
testQuery(
"SELECT dim1, COUNT(*) FROM druid.foo WHERE dim1 IN (" + elementsString + ") GROUP BY dim1",
"SELECT dim1, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 IN (" + elementsString + ") OR dim1 = 'xyz' OR dim1 IS NULL\n"
+ "GROUP BY dim1",
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(in("dim1", elements))
.setDimFilter(
NullHandling.sqlCompatible()
? or(
in("dim1", ImmutableSet.<String>builder().addAll(elements).add("xyz").build()),
isNull("dim1")
)
: in(
"dim1",
Lists.newArrayList(
Iterables.concat(
Collections.singleton(null),
elements,
Collections.singleton("xyz")
)
)
)
)
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
@ -5701,7 +6109,139 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
: ImmutableList.of(
new Object[]{"", 1L},
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
);
}
@Test
public void testInFilterWith23Elements_overScalarInArrayThreshold()
{
final List<String> elements = new ArrayList<>();
elements.add("abc");
elements.add("def");
elements.add("ghi");
for (int i = 0; i < 20; i++) {
elements.add("dummy" + i);
}
final String elementsString = Joiner.on(",").join(elements.stream().map(s -> "'" + s + "'").iterator());
testQuery(
"SELECT dim1, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 IN (" + elementsString + ") OR dim1 = 'xyz' OR dim1 IS NULL\n"
+ "GROUP BY dim1",
QueryContexts.override(QUERY_CONTEXT_DEFAULT, QueryContexts.IN_FUNCTION_THRESHOLD, 20),
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
// [dim1 = xyz] is not combined into the IN filter, because SCALAR_IN_ARRAY was used,
// and it is opaque to most optimizations. (That's its main purpose.)
NullHandling.sqlCompatible()
? or(
in("dim1", elements),
isNull("dim1"),
equality("dim1", "xyz", ColumnType.STRING)
)
: or(
in("dim1", Arrays.asList(null, "xyz")),
in("dim1", elements)
)
)
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
: ImmutableList.of(
new Object[]{"", 1L},
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
);
}
@Test
public void testInFilterWith23Elements_overBothScalarInArrayAndInSubQueryThresholds()
{
// Verify that when an IN filter surpasses both inFunctionThreshold and inSubQueryThreshold, the
// inFunctionThreshold takes priority.
final List<String> elements = new ArrayList<>();
elements.add("abc");
elements.add("def");
elements.add("ghi");
for (int i = 0; i < 20; i++) {
elements.add("dummy" + i);
}
final String elementsString = Joiner.on(",").join(elements.stream().map(s -> "'" + s + "'").iterator());
testQuery(
"SELECT dim1, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 IN (" + elementsString + ") OR dim1 = 'xyz' OR dim1 IS NULL\n"
+ "GROUP BY dim1",
QueryContexts.override(
QUERY_CONTEXT_DEFAULT,
ImmutableMap.of(
QueryContexts.IN_FUNCTION_THRESHOLD, 20,
QueryContexts.IN_SUB_QUERY_THRESHOLD_KEY, 20
)
),
ImmutableList.of(
GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
.setDimFilter(
// [dim1 = xyz] is not combined into the IN filter, because SCALAR_IN_ARRAY was used,
// and it is opaque to most optimizations. (That's its main purpose.)
NullHandling.sqlCompatible()
? or(
in("dim1", elements),
isNull("dim1"),
equality("dim1", "xyz", ColumnType.STRING)
)
: or(
in("dim1", Arrays.asList(null, "xyz")),
in("dim1", elements)
)
)
.setAggregatorSpecs(
aggregators(
new CountAggregatorFactory("a0")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)
: ImmutableList.of(
new Object[]{"", 1L},
new Object[]{"abc", 1L},
new Object[]{"def", 1L}
)

View File

@ -19,6 +19,7 @@
package org.apache.druid.sql.calcite;
import com.google.common.base.CaseFormat;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
@ -157,10 +158,10 @@ public class SqlTestFrameworkConfig
}
private static final Set<String> KNOWN_CONFIG_KEYS = ImmutableSet.<String>builder()
.add(NumMergeBuffers.class.getSimpleName())
.add(MinTopNThreshold.class.getSimpleName())
.add(ResultCache.class.getSimpleName())
.add(ComponentSupplier.class.getSimpleName())
.add(NumMergeBuffers.PROCESSOR.getConfigName())
.add(MinTopNThreshold.PROCESSOR.getConfigName())
.add(ResultCache.PROCESSOR.getConfigName())
.add(ComponentSupplier.PROCESSOR.getConfigName())
.build();
public final int numMergeBuffers;
@ -405,10 +406,10 @@ public class SqlTestFrameworkConfig
Map<String, String> map = new HashMap<>();
SqlTestFrameworkConfig def = new SqlTestFrameworkConfig(Collections.emptyList());
if (def.numMergeBuffers != numMergeBuffers) {
map.put("NumMergeBuffers", String.valueOf(numMergeBuffers));
map.put("numMergeBuffers", String.valueOf(numMergeBuffers));
}
if (def.minTopNThreshold != minTopNThreshold) {
map.put("MinTopNThreshold", String.valueOf(minTopNThreshold));
map.put("minTopNThreshold", String.valueOf(minTopNThreshold));
}
if (!equals(new SqlTestFrameworkConfig(map))) {
throw new IAE("Can't reproduce config via map!");
@ -425,6 +426,11 @@ public class SqlTestFrameworkConfig
this.annotationClass = annotationClass;
}
public final String getConfigName()
{
return CaseFormat.UPPER_CAMEL.to(CaseFormat.LOWER_CAMEL, annotationClass.getSimpleName());
}
@SuppressWarnings("unchecked")
public final T fromAnnotations(List<Annotation> annotations) throws Exception
{
@ -453,7 +459,7 @@ public class SqlTestFrameworkConfig
public final T fromMap(Map<String, String> map) throws Exception
{
String key = annotationClass.getSimpleName();
String key = getConfigName();
String value = map.get(key);
if (value == null) {
return defaultValue();

View File

@ -132,8 +132,9 @@ public class SqlTestFrameworkConfigTest
IAE.class,
() -> new SqlTestFrameworkConfig(configMap)
);
assertEquals(
"Invalid configuration key(s) specified [[nonExistent]]; valid options are [[NumMergeBuffers, MinTopNThreshold, ResultCache, ComponentSupplier]]",
"Invalid configuration key(s) specified [[nonExistent]]; valid options are [[numMergeBuffers, minTopNThreshold, resultCache, componentSupplier]]",
e.getMessage()
);
}

View File

@ -1,5 +1,5 @@
!set plannerStrategy DECOUPLED
!use druidtest://?NumMergeBuffers=3
!use druidtest://?numMergeBuffers=3
!set outputformat mysql
select cityName, count(case when delta > 0 then channel end) as cnt, count(1) as aall

View File

@ -1,4 +1,4 @@
!use druidtest://?NumMergeBuffers=3
!use druidtest://?numMergeBuffers=3
!set outputformat mysql
with v as (

View File

@ -1,4 +1,4 @@
!use druidtest://?NumMergeBuffers=3
!use druidtest://?numMergeBuffers=3
!set outputformat mysql
SELECT

View File

@ -1,4 +1,4 @@
!use druidtest://?ComponentSupplier=NestedComponentSupplier
!use druidtest://?componentSupplier=NestedComponentSupplier
!set outputformat mysql
select count(1) from nested;

View File

@ -1,4 +1,4 @@
# testExactTopNOnInnerJoinWithLimit@NullHandling=default case-crc:d28d4632
# testExactTopNOnInnerJoinWithLimit@NullHandling=default case-crc:1b8b1878
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
@ -8,7 +8,7 @@
!set sqlQueryId dummy
!set useApproximateTopN false
!set outputformat mysql
!use druidtest:///?MinTopNThreshold=1
!use druidtest:///?minTopNThreshold=1
select f1."dim4", sum("m1") from numfoo f1 inner join (
select "dim4" from numfoo where dim4 <> 'a' group by 1
) f2 on f1."dim4" = f2."dim4" group by 1 limit 1;

View File

@ -1,4 +1,4 @@
# testExactTopNOnInnerJoinWithLimit@NullHandling=sql case-crc:d28d4632
# testExactTopNOnInnerJoinWithLimit@NullHandling=sql case-crc:1b8b1878
# quidem testcase reason: EQUIV_PLAN
!set debug true
!set defaultTimeout 300000
@ -8,7 +8,7 @@
!set sqlQueryId dummy
!set useApproximateTopN false
!set outputformat mysql
!use druidtest:///?MinTopNThreshold=1
!use druidtest:///?minTopNThreshold=1
select f1."dim4", sum("m1") from numfoo f1 inner join (
select "dim4" from numfoo where dim4 <> 'a' group by 1
) f2 on f1."dim4" = f2."dim4" group by 1 limit 1;

View File

@ -30,8 +30,8 @@ SELECT
+-------+----+
| t1 | t2 |
+-------+----+
| dummy | |
| dummy | b |
| dummy | |
+-------+----+
(2 rows)

View File

@ -1,4 +1,4 @@
# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default case-crc:d6c371b5
# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=default case-crc:5a66a3be
# quidem testcase reason: SLIGHTLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
@ -9,7 +9,7 @@
!set useApproximateCountDistinct false
!set useGroupingSetForExactDistinct true
!set outputformat mysql
!use druidtest:///?NumMergeBuffers=3
!use druidtest:///?numMergeBuffers=3
SELECT
(SUM(CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time<TIMESTAMP '2022-01-05 17:00:00') THEN 1 ELSE 0 END)*1.0/COUNT(DISTINCT CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time<TIMESTAMP '2022-01-05 17:00:00') THEN dim1 END))
FROM druid.foo

View File

@ -1,4 +1,4 @@
# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql case-crc:d6c371b5
# testQueryWithSelectProjectAndIdentityProjectDoesNotRename@NullHandling=sql case-crc:5a66a3be
# quidem testcase reason: SLIGHTLY_WORSE_PLAN
!set debug true
!set defaultTimeout 300000
@ -9,7 +9,7 @@
!set useApproximateCountDistinct false
!set useGroupingSetForExactDistinct true
!set outputformat mysql
!use druidtest:///?NumMergeBuffers=3
!use druidtest:///?numMergeBuffers=3
SELECT
(SUM(CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time<TIMESTAMP '2022-01-05 17:00:00') THEN 1 ELSE 0 END)*1.0/COUNT(DISTINCT CASE WHEN (TIMESTAMP '2000-01-04 17:00:00'<=__time AND __time<TIMESTAMP '2022-01-05 17:00:00') THEN dim1 END))
FROM druid.foo

View File

@ -14,6 +14,7 @@
"@blueprintjs/datetime2": "^0.9.35",
"@blueprintjs/icons": "^4.16.0",
"@blueprintjs/popover2": "^1.14.9",
"@blueprintjs/select": "^4.9.24",
"@druid-toolkit/query": "^0.22.13",
"@druid-toolkit/visuals-core": "^0.3.3",
"@druid-toolkit/visuals-react": "^0.3.3",
@ -823,9 +824,9 @@
}
},
"node_modules/@blueprintjs/core": {
"version": "4.20.1",
"resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.1.tgz",
"integrity": "sha512-nKFXfWj8PQrkweLrCr6VazYd4JJHCwiAxcgT9zzoxbEs0mSJF4yI7Qjoh5QkomtWugXrVLCDSzs4uJdaO1reAA==",
"version": "4.20.2",
"resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.2.tgz",
"integrity": "sha512-5v4Nr0jozfAjiOkjY4zvt1XSpt4ldnrSaxtwo506S2cxJYfwFeMTmDshXNPFcc8L1fjZMxi0IWI2WABXzZXS6w==",
"dependencies": {
"@blueprintjs/colors": "^4.2.1",
"@blueprintjs/icons": "^4.16.0",
@ -929,11 +930,11 @@
}
},
"node_modules/@blueprintjs/popover2": {
"version": "1.14.9",
"resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.9.tgz",
"integrity": "sha512-46gesoxeEbp1owKDGz7JwurXmVqSLq9ALes5etHqtjwdCpenBQc7GM+sFuJoLlEU7twcIlzwE6xaSU2+vnYVCQ==",
"version": "1.14.11",
"resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.11.tgz",
"integrity": "sha512-5XAjeb2mlWjYXC0pqrNDLzHSsX85Zaiv8jixxUN9abarMUUFKGATgGF8MRsWTLAW94Gli6CB1lzVkrYkRHHf6Q==",
"dependencies": {
"@blueprintjs/core": "^4.20.1",
"@blueprintjs/core": "^4.20.2",
"@juggle/resize-observer": "^3.4.0",
"@popperjs/core": "^2.11.7",
"classnames": "^2.3.1",
@ -953,12 +954,12 @@
}
},
"node_modules/@blueprintjs/select": {
"version": "4.9.22",
"resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.22.tgz",
"integrity": "sha512-T5ofz571kPfycbxAuXlTrE7ndXIiHPCDMjADYJpb5HVOk0SYwwzDvnWjsYbyAj1SONbxOAKh4/wmFTDjZv7G2g==",
"version": "4.9.24",
"resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.24.tgz",
"integrity": "sha512-OTjesxH/7UZvM7yAdHJ5u3sIjX1N8Rs4CQQ22AfqNl82SIROqkuXI31XEl6YNX1GsYfmAMiw0K7XohEKOMXR5g==",
"dependencies": {
"@blueprintjs/core": "^4.20.1",
"@blueprintjs/popover2": "^1.14.9",
"@blueprintjs/core": "^4.20.2",
"@blueprintjs/popover2": "^1.14.11",
"classnames": "^2.3.1",
"tslib": "~2.5.0"
},
@ -19029,9 +19030,9 @@
}
},
"@blueprintjs/core": {
"version": "4.20.1",
"resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.1.tgz",
"integrity": "sha512-nKFXfWj8PQrkweLrCr6VazYd4JJHCwiAxcgT9zzoxbEs0mSJF4yI7Qjoh5QkomtWugXrVLCDSzs4uJdaO1reAA==",
"version": "4.20.2",
"resolved": "https://registry.npmjs.org/@blueprintjs/core/-/core-4.20.2.tgz",
"integrity": "sha512-5v4Nr0jozfAjiOkjY4zvt1XSpt4ldnrSaxtwo506S2cxJYfwFeMTmDshXNPFcc8L1fjZMxi0IWI2WABXzZXS6w==",
"requires": {
"@blueprintjs/colors": "^4.2.1",
"@blueprintjs/icons": "^4.16.0",
@ -19100,11 +19101,11 @@
}
},
"@blueprintjs/popover2": {
"version": "1.14.9",
"resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.9.tgz",
"integrity": "sha512-46gesoxeEbp1owKDGz7JwurXmVqSLq9ALes5etHqtjwdCpenBQc7GM+sFuJoLlEU7twcIlzwE6xaSU2+vnYVCQ==",
"version": "1.14.11",
"resolved": "https://registry.npmjs.org/@blueprintjs/popover2/-/popover2-1.14.11.tgz",
"integrity": "sha512-5XAjeb2mlWjYXC0pqrNDLzHSsX85Zaiv8jixxUN9abarMUUFKGATgGF8MRsWTLAW94Gli6CB1lzVkrYkRHHf6Q==",
"requires": {
"@blueprintjs/core": "^4.20.1",
"@blueprintjs/core": "^4.20.2",
"@juggle/resize-observer": "^3.4.0",
"@popperjs/core": "^2.11.7",
"classnames": "^2.3.1",
@ -19114,12 +19115,12 @@
}
},
"@blueprintjs/select": {
"version": "4.9.22",
"resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.22.tgz",
"integrity": "sha512-T5ofz571kPfycbxAuXlTrE7ndXIiHPCDMjADYJpb5HVOk0SYwwzDvnWjsYbyAj1SONbxOAKh4/wmFTDjZv7G2g==",
"version": "4.9.24",
"resolved": "https://registry.npmjs.org/@blueprintjs/select/-/select-4.9.24.tgz",
"integrity": "sha512-OTjesxH/7UZvM7yAdHJ5u3sIjX1N8Rs4CQQ22AfqNl82SIROqkuXI31XEl6YNX1GsYfmAMiw0K7XohEKOMXR5g==",
"requires": {
"@blueprintjs/core": "^4.20.1",
"@blueprintjs/popover2": "^1.14.9",
"@blueprintjs/core": "^4.20.2",
"@blueprintjs/popover2": "^1.14.11",
"classnames": "^2.3.1",
"tslib": "~2.5.0"
}

View File

@ -68,6 +68,7 @@
"@blueprintjs/datetime2": "^0.9.35",
"@blueprintjs/icons": "^4.16.0",
"@blueprintjs/popover2": "^1.14.9",
"@blueprintjs/select": "^4.9.24",
"@druid-toolkit/query": "^0.22.13",
"@druid-toolkit/visuals-core": "^0.3.3",
"@druid-toolkit/visuals-react": "^0.3.3",

View File

@ -23,8 +23,8 @@ const snarkdown = require('snarkdown');
const writefile = 'lib/sql-docs.js';
const MINIMUM_EXPECTED_NUMBER_OF_FUNCTIONS = 167;
const MINIMUM_EXPECTED_NUMBER_OF_DATA_TYPES = 14;
const MINIMUM_EXPECTED_NUMBER_OF_FUNCTIONS = 198;
const MINIMUM_EXPECTED_NUMBER_OF_DATA_TYPES = 15;
const initialFunctionDocs = {
TABLE: [['external', convertMarkdownToHtml('Defines a logical table from an external.')]],
@ -78,6 +78,7 @@ const readDoc = async () => {
await fs.readFile('../docs/querying/sql-array-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-multivalue-string-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-json-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-window-functions.md', 'utf-8'),
await fs.readFile('../docs/querying/sql-operators.md', 'utf-8'),
].join('\n');

View File

@ -62,9 +62,12 @@ function _build_distribution() {
&& cd apache-druid-$(_get_druid_version) \
&& mkdir -p extensions/druid-testing-tools \
&& cp "$(_get_code_root)/extensions-core/testing-tools/target/druid-testing-tools-$(_get_druid_version).jar" extensions/druid-testing-tools/ \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-datasketches\", \"druid-multi-stage-query\", \"druid-testing-tools\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& mkdir -p extensions/druid-compressed-bigdecimal \
&& cp "$(_get_code_root)/extensions-contrib/compressed-bigdecimal/target/druid-compressed-bigdecimal-$(_get_druid_version).jar" extensions/druid-compressed-bigdecimal/ \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-multi-stage-query\", \"druid-testing-tools\", \"druid-bloom-filter\", \"druid-datasketches\", \"druid-histogram\", \"druid-stats\", \"druid-compressed-bigdecimal\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.extensions.loadList=[\"druid-hdfs-storage\", \"druid-kafka-indexing-service\", \"druid-multi-stage-query\", \"druid-testing-tools\", \"druid-bloom-filter\", \"druid-datasketches\", \"druid-histogram\", \"druid-stats\", \"druid-compressed-bigdecimal\"]" >> conf/druid/auto/_common/common.runtime.properties \
&& echo -e "\n\ndruid.server.http.allowedHttpMethods=[\"HEAD\"]" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.generic.useDefaultValueForNull=false" >> conf/druid/single-server/micro-quickstart/_common/common.runtime.properties \
&& echo -e "\n\ndruid.server.http.allowedHttpMethods=[\"HEAD\"]" >> conf/druid/auto/_common/common.runtime.properties \
)
}

View File

@ -95,38 +95,40 @@ exports[`SegmentTimeline matches snapshot 1`] = `
class="bp4-form-content"
>
<div
class="bp4-html-select bp4-fill"
aria-controls="listbox-1"
aria-expanded="false"
aria-haspopup="listbox"
class="bp4-popover2-target bp4-fill"
role="combobox"
>
<select>
<option
value="all"
<button
class="bp4-button bp4-fill"
type="button"
>
<span
class="bp4-button-text"
>
Show all
</option>
</select>
<span
class="bp4-icon bp4-icon-double-caret-vertical"
icon="double-caret-vertical"
>
<svg
aria-labelledby="iconTitle-0"
data-icon="double-caret-vertical"
height="16"
role="img"
viewBox="0 0 16 16"
width="16"
</span>
<span
aria-hidden="true"
class="bp4-icon bp4-icon-caret-down"
icon="caret-down"
>
<title
id="iconTitle-0"
<svg
data-icon="caret-down"
height="16"
role="img"
viewBox="0 0 16 16"
width="16"
>
Open dropdown
</title>
<path
d="M5 7h6a1.003 1.003 0 00.71-1.71l-3-3C8.53 2.11 8.28 2 8 2s-.53.11-.71.29l-3 3A1.003 1.003 0 005 7zm6 2H5a1.003 1.003 0 00-.71 1.71l3 3c.18.18.43.29.71.29s.53-.11.71-.29l3-3A1.003 1.003 0 0011 9z"
fill-rule="evenodd"
/>
</svg>
</span>
<path
d="M12 6.5c0-.28-.22-.5-.5-.5h-7a.495.495 0 00-.37.83l3.5 4c.09.1.22.17.37.17s.28-.07.37-.17l3.5-4c.08-.09.13-.2.13-.33z"
fill-rule="evenodd"
/>
</svg>
</span>
</button>
</div>
</div>
</div>

Some files were not shown because too many files have changed in this diff Show More