mirror of https://github.com/apache/druid.git
Merge remote-tracking branch 'apache/master' into quidem-runner-extension-submit
This commit is contained in:
commit
206363c255
|
@ -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"
|
||||
}
|
||||
```
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
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;
|
||||
|
@ -29,14 +30,21 @@ 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.
|
||||
|
@ -445,4 +453,171 @@ public abstract class ITCatalogIngestAndQueryTest
|
|||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,11 @@
|
|||
[
|
||||
{
|
||||
"query": "SELECT * FROM %%DATASOURCE%%",
|
||||
"expectedResults": [
|
||||
{
|
||||
"__time": 1672058096000,
|
||||
"double_col": 0.0
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -0,0 +1,11 @@
|
|||
[
|
||||
{
|
||||
"query": "SELECT * FROM %%DATASOURCE%%",
|
||||
"expectedResults": [
|
||||
{
|
||||
"__time": 1672058096000,
|
||||
"extra": "foo"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -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
|
||||
|
||||
---
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -621,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));
|
||||
|
|
|
@ -88,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";
|
||||
|
@ -126,6 +127,7 @@ public class QueryContexts
|
|||
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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -281,7 +281,7 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return globalDictionary.getCardinality();
|
||||
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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));
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -534,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) {
|
||||
|
@ -592,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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
@ -7322,4 +7357,215 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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>
|
||||
<span
|
||||
class="bp4-icon bp4-icon-double-caret-vertical"
|
||||
icon="double-caret-vertical"
|
||||
aria-hidden="true"
|
||||
class="bp4-icon bp4-icon-caret-down"
|
||||
icon="caret-down"
|
||||
>
|
||||
<svg
|
||||
aria-labelledby="iconTitle-0"
|
||||
data-icon="double-caret-vertical"
|
||||
data-icon="caret-down"
|
||||
height="16"
|
||||
role="img"
|
||||
viewBox="0 0 16 16"
|
||||
width="16"
|
||||
>
|
||||
<title
|
||||
id="iconTitle-0"
|
||||
>
|
||||
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"
|
||||
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>
|
||||
|
|
|
@ -16,7 +16,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import { FormGroup, HTMLSelect, Radio, RadioGroup, ResizeSensor } from '@blueprintjs/core';
|
||||
import { Button, FormGroup, MenuItem, Radio, RadioGroup, ResizeSensor } from '@blueprintjs/core';
|
||||
import { IconNames } from '@blueprintjs/icons';
|
||||
import type { ItemPredicate, ItemRenderer } from '@blueprintjs/select';
|
||||
import { Select2 } from '@blueprintjs/select';
|
||||
import type { AxisScale } from 'd3-axis';
|
||||
import { scaleLinear, scaleUtc } from 'd3-scale';
|
||||
import React from 'react';
|
||||
|
@ -528,6 +531,63 @@ ORDER BY "start" DESC`;
|
|||
const { capabilities } = this.props;
|
||||
const { datasources, activeDataType, activeDatasource, startDate, endDate } = this.state;
|
||||
|
||||
const filterDatasource: ItemPredicate<string> = (query, val, _index, exactMatch) => {
|
||||
const normalizedTitle = val.toLowerCase();
|
||||
const normalizedQuery = query.toLowerCase();
|
||||
|
||||
if (exactMatch) {
|
||||
return normalizedTitle === normalizedQuery;
|
||||
} else {
|
||||
return ` ${normalizedTitle}`.includes(normalizedQuery);
|
||||
}
|
||||
};
|
||||
|
||||
const datasourceRenderer: ItemRenderer<string> = (
|
||||
val,
|
||||
{ handleClick, handleFocus, modifiers },
|
||||
) => {
|
||||
if (!modifiers.matchesPredicate) {
|
||||
return null;
|
||||
}
|
||||
return (
|
||||
<MenuItem
|
||||
key={val}
|
||||
disabled={modifiers.disabled}
|
||||
active={modifiers.active}
|
||||
onClick={handleClick}
|
||||
onFocus={handleFocus}
|
||||
roleStructure="listoption"
|
||||
text={val}
|
||||
/>
|
||||
);
|
||||
};
|
||||
|
||||
const DatasourceSelect: React.FC = () => {
|
||||
const showAll = 'Show all';
|
||||
const handleItemSelected = (selectedItem: string) => {
|
||||
this.setState({
|
||||
activeDatasource: selectedItem === showAll ? null : selectedItem,
|
||||
});
|
||||
};
|
||||
const datasourcesWzAll = [showAll].concat(datasources);
|
||||
return (
|
||||
<Select2<string>
|
||||
items={datasourcesWzAll}
|
||||
onItemSelect={handleItemSelected}
|
||||
itemRenderer={datasourceRenderer}
|
||||
noResults={<MenuItem disabled text="No results." roleStructure="listoption" />}
|
||||
itemPredicate={filterDatasource}
|
||||
fill
|
||||
>
|
||||
<Button
|
||||
text={activeDatasource === null ? showAll : activeDatasource}
|
||||
fill
|
||||
rightIcon={IconNames.CARET_DOWN}
|
||||
/>
|
||||
</Select2>
|
||||
);
|
||||
};
|
||||
|
||||
return (
|
||||
<div className="segment-timeline app-view">
|
||||
{this.renderStackedBarChart()}
|
||||
|
@ -543,24 +603,7 @@ ORDER BY "start" DESC`;
|
|||
</FormGroup>
|
||||
|
||||
<FormGroup label="Datasource">
|
||||
<HTMLSelect
|
||||
onChange={(e: any) =>
|
||||
this.setState({
|
||||
activeDatasource: e.target.value === 'all' ? null : e.target.value,
|
||||
})
|
||||
}
|
||||
value={activeDatasource == null ? 'all' : activeDatasource}
|
||||
fill
|
||||
>
|
||||
<option value="all">Show all</option>
|
||||
{datasources.map(d => {
|
||||
return (
|
||||
<option key={d} value={d}>
|
||||
{d}
|
||||
</option>
|
||||
);
|
||||
})}
|
||||
</HTMLSelect>
|
||||
<DatasourceSelect />
|
||||
</FormGroup>
|
||||
|
||||
<FormGroup label="Interval">
|
||||
|
|
|
@ -36,7 +36,7 @@ import * as JSONBig from 'json-bigint-native';
|
|||
import { v4 as uuidv4 } from 'uuid';
|
||||
|
||||
import type { RowColumn } from '../../utils';
|
||||
import { deleteKeys } from '../../utils';
|
||||
import { caseInsensitiveEquals, deleteKeys } from '../../utils';
|
||||
import type { DruidEngine } from '../druid-engine/druid-engine';
|
||||
import { validDruidEngine } from '../druid-engine/druid-engine';
|
||||
import type { LastExecution } from '../execution/execution';
|
||||
|
@ -512,7 +512,11 @@ export class WorkbenchQuery {
|
|||
}
|
||||
|
||||
const ingestQuery = this.isIngestQuery();
|
||||
if (!unlimited && !ingestQuery && queryContext.selectDestination !== 'durableStorage') {
|
||||
if (
|
||||
!unlimited &&
|
||||
!ingestQuery &&
|
||||
!caseInsensitiveEquals(queryContext.selectDestination, 'durableStorage')
|
||||
) {
|
||||
apiQuery.context ||= {};
|
||||
apiQuery.context.sqlOuterLimit = 1001;
|
||||
}
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
import {
|
||||
arrangeWithPrefixSuffix,
|
||||
caseInsensitiveEquals,
|
||||
formatBytes,
|
||||
formatBytesCompact,
|
||||
formatInteger,
|
||||
|
@ -196,4 +197,14 @@ describe('general', () => {
|
|||
});
|
||||
});
|
||||
});
|
||||
|
||||
describe('caseInsensitiveEquals', () => {
|
||||
it('works', () => {
|
||||
expect(caseInsensitiveEquals(undefined, undefined)).toEqual(true);
|
||||
expect(caseInsensitiveEquals(undefined, 'x')).toEqual(false);
|
||||
expect(caseInsensitiveEquals('x', undefined)).toEqual(false);
|
||||
expect(caseInsensitiveEquals('x', 'X')).toEqual(true);
|
||||
expect(caseInsensitiveEquals(undefined, '')).toEqual(false);
|
||||
});
|
||||
});
|
||||
});
|
||||
|
|
|
@ -89,6 +89,10 @@ export function addOrUpdate<T>(xs: readonly T[], x: T, keyFn: (x: T) => string |
|
|||
|
||||
// ----------------------------
|
||||
|
||||
export function caseInsensitiveEquals(str1: string | undefined, str2: string | undefined): boolean {
|
||||
return str1?.toLowerCase() === str2?.toLowerCase();
|
||||
}
|
||||
|
||||
export function caseInsensitiveContains(testString: string, searchString: string): boolean {
|
||||
if (!searchString) return true;
|
||||
return testString.toLowerCase().includes(searchString.toLowerCase());
|
||||
|
|
Loading…
Reference in New Issue