mirror of https://github.com/apache/druid.git
Remove null and empty fields from native queries (#12634)
* Remove null and empty fields from native queries * Test fixes * Attempted IT fix. * Revisions from review comments * Build fixes resulting from changes suggested by reviews * IT fix for changed segment size
This commit is contained in:
parent
f050069767
commit
893759de91
|
@ -46,7 +46,10 @@
|
|||
<Or>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$ScanRowsLimitJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$ScanTimeOrderJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$BatchSizeJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$IsLegacyJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.groupby.orderby.DefaultLimitSpec$LimitJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.segment.VirtualColumns$JsonIncludeFilter"/>
|
||||
</Or>
|
||||
</And>
|
||||
</Match>
|
||||
|
@ -56,6 +59,12 @@
|
|||
<Class name="org.apache.druid.server.AsyncQueryForwardingServlet" />
|
||||
</And>
|
||||
</Match>
|
||||
<Match>
|
||||
<And>
|
||||
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
|
||||
<Class name="org.apache.druid.sql.calcite.planner.CapturedState" />
|
||||
</And>
|
||||
</Match>
|
||||
|
||||
<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"/>
|
||||
<Bug pattern="BC_UNCONFIRMED_CAST"/>
|
||||
|
|
|
@ -90,7 +90,6 @@ public class EmitterTest
|
|||
return builder;
|
||||
}
|
||||
|
||||
|
||||
MockHttpClient httpClient;
|
||||
HttpPostEmitter emitter;
|
||||
|
||||
|
|
|
@ -164,7 +164,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
@ -211,7 +210,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass());
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new HashedPartitionsSpec(null, 3, null),
|
||||
|
@ -259,7 +257,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass());
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new SingleDimensionPartitionsSpec(7, null, "dim", false),
|
||||
|
@ -310,7 +307,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass());
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false),
|
||||
|
@ -358,7 +354,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass());
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new SingleDimensionPartitionsSpec(7, null, "dim", false),
|
||||
|
@ -409,7 +404,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass());
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false),
|
||||
|
@ -490,7 +484,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
@ -545,7 +538,6 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
|
|
@ -202,7 +202,6 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
return new CompactionState(
|
||||
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
@ -355,7 +354,6 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedState = new CompactionState(
|
||||
new HashedPartitionsSpec(null, 3, null),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
@ -648,7 +646,6 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedCompactionState = new CompactionState(
|
||||
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
@ -710,7 +707,6 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||
expectedLongSumMetric.put("type", "longSum");
|
||||
expectedLongSumMetric.put("name", "val");
|
||||
expectedLongSumMetric.put("fieldName", "val");
|
||||
expectedLongSumMetric.put("expression", null);
|
||||
CompactionState expectedCompactionState = new CompactionState(
|
||||
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
|
||||
|
|
|
@ -115,9 +115,6 @@ public class RemoteTaskRunnerTest
|
|||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
|
||||
@Rule
|
||||
public final TestRule timeout = new DeadlockDetectingTimeout(60, TimeUnit.SECONDS);
|
||||
|
||||
|
|
|
@ -117,7 +117,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
@BeforeMethod
|
||||
public void setup() throws Exception
|
||||
{
|
||||
// Set comapction slot to 5
|
||||
// Set compaction slot to 5
|
||||
updateCompactionTaskSlot(0.5, 10, null);
|
||||
fullDatasourceName = "wikipedia_index_test_" + UUID.randomUUID() + config.getExtraDatasourceNameSuffix();
|
||||
}
|
||||
|
@ -458,8 +458,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
fullDatasourceName,
|
||||
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
|
||||
0,
|
||||
14906,
|
||||
14905,
|
||||
14762,
|
||||
14761,
|
||||
0,
|
||||
2,
|
||||
2,
|
||||
|
@ -476,7 +476,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
fullDatasourceName,
|
||||
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
|
||||
0,
|
||||
23372,
|
||||
23156,
|
||||
0,
|
||||
0,
|
||||
3,
|
||||
|
@ -592,8 +592,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
getAndAssertCompactionStatus(
|
||||
fullDatasourceName,
|
||||
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
|
||||
14906,
|
||||
14905,
|
||||
14762,
|
||||
14761,
|
||||
0,
|
||||
2,
|
||||
2,
|
||||
|
@ -601,7 +601,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
1,
|
||||
1,
|
||||
0);
|
||||
Assert.assertEquals(compactionResource.getCompactionProgress(fullDatasourceName).get("remainingSegmentSize"), "14906");
|
||||
Assert.assertEquals(compactionResource.getCompactionProgress(fullDatasourceName).get("remainingSegmentSize"), "14762");
|
||||
// Run compaction again to compact the remaining day
|
||||
// Remaining day compacted (1 new segment). Now both days compacted (2 total)
|
||||
forceTriggerAutoCompaction(2);
|
||||
|
@ -612,7 +612,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
|
|||
fullDatasourceName,
|
||||
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
|
||||
0,
|
||||
23372,
|
||||
23156,
|
||||
0,
|
||||
0,
|
||||
3,
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.apache.druid.query;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -101,6 +103,7 @@ public abstract class BaseQuery<T> implements Query<T>
|
|||
|
||||
@JsonProperty
|
||||
@Override
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public boolean isDescending()
|
||||
{
|
||||
return descending;
|
||||
|
@ -165,6 +168,7 @@ public abstract class BaseQuery<T> implements Query<T>
|
|||
|
||||
@Override
|
||||
@JsonProperty
|
||||
@JsonInclude(Include.NON_DEFAULT)
|
||||
public Map<String, Object> getContext()
|
||||
{
|
||||
return context.getMergedParams();
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.apache.druid.query;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -33,6 +35,7 @@ import org.apache.druid.segment.join.JoinPrefixUtils;
|
|||
import org.apache.druid.segment.join.JoinType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
@ -176,6 +179,7 @@ public class JoinDataSource implements DataSource
|
|||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public DimFilter getLeftFilter()
|
||||
{
|
||||
return leftFilter;
|
||||
|
|
|
@ -34,7 +34,7 @@ import java.util.Set;
|
|||
public class QueryDataSource implements DataSource
|
||||
{
|
||||
@JsonProperty
|
||||
private final Query query;
|
||||
private final Query<?> query;
|
||||
|
||||
@JsonCreator
|
||||
public QueryDataSource(@JsonProperty("query") Query query)
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.aggregation;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
@ -204,6 +205,7 @@ public class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
|||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public Set<String> getFields()
|
||||
{
|
||||
return fields;
|
||||
|
@ -211,6 +213,7 @@ public class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
|||
|
||||
@JsonProperty
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getAccumulatorIdentifier()
|
||||
{
|
||||
return accumulatorId;
|
||||
|
@ -260,6 +263,7 @@ public class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
|||
|
||||
@JsonProperty("compare")
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getCompareExpressionString()
|
||||
{
|
||||
return compareExpressionString;
|
||||
|
@ -267,6 +271,7 @@ public class ExpressionLambdaAggregatorFactory extends AggregatorFactory
|
|||
|
||||
@JsonProperty("finalize")
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getFinalizeExpressionString()
|
||||
{
|
||||
return finalizeExpressionString;
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.apache.druid.query.aggregation;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
||||
import org.apache.druid.guice.annotations.ExtensionPoint;
|
||||
import org.apache.druid.java.util.common.Cacheable;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
|
@ -45,6 +47,7 @@ public interface PostAggregator extends Cacheable
|
|||
Object compute(Map<String, Object> combinedAggregators);
|
||||
|
||||
@Nullable
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
String getName();
|
||||
|
||||
/**
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.aggregation;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
@ -234,6 +235,7 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getFieldName()
|
||||
{
|
||||
return fieldName;
|
||||
|
@ -241,6 +243,7 @@ public abstract class SimpleDoubleAggregatorFactory extends NullableNumericAggre
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getExpression()
|
||||
{
|
||||
return expression;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.aggregation;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
@ -212,6 +213,7 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getFieldName()
|
||||
{
|
||||
return fieldName;
|
||||
|
@ -219,6 +221,7 @@ public abstract class SimpleFloatAggregatorFactory extends NullableNumericAggreg
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getExpression()
|
||||
{
|
||||
return expression;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.aggregation;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
|||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
@ -215,6 +217,7 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getFieldName()
|
||||
{
|
||||
return fieldName;
|
||||
|
@ -222,6 +225,7 @@ public abstract class SimpleLongAggregatorFactory extends NullableNumericAggrega
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getExpression()
|
||||
{
|
||||
return expression;
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
package org.apache.druid.query.aggregation.post;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
|
@ -33,6 +35,7 @@ import org.apache.druid.segment.ColumnInspector;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
|
@ -169,6 +172,7 @@ public class ArithmeticPostAggregator implements PostAggregator
|
|||
}
|
||||
|
||||
@JsonProperty("ordering")
|
||||
@JsonInclude(Include.NON_NULL)
|
||||
public String getOrdering()
|
||||
{
|
||||
return ordering;
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.aggregation.post;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -213,6 +214,7 @@ public class ExpressionPostAggregator implements PostAggregator
|
|||
|
||||
@Nullable
|
||||
@JsonProperty("ordering")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getOrdering()
|
||||
{
|
||||
return ordering;
|
||||
|
|
|
@ -132,6 +132,7 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getUpper()
|
||||
{
|
||||
return upper;
|
||||
|
@ -139,18 +140,21 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getLower()
|
||||
{
|
||||
return lower;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public boolean isLowerStrict()
|
||||
{
|
||||
return lowerStrict;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
|
||||
public boolean isUpperStrict()
|
||||
{
|
||||
return upperStrict;
|
||||
|
@ -168,6 +172,7 @@ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimF
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public ExtractionFn getExtractionFn()
|
||||
{
|
||||
return extractionFn;
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
|||
import org.apache.druid.segment.filter.SelectorFilter;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -116,6 +117,9 @@ public class SelectorDimFilter extends AbstractOptimizableDimFilter implements D
|
|||
return dimension;
|
||||
}
|
||||
|
||||
/**
|
||||
* Value to filter against. If {@code null}, then the meaning is `is null`.
|
||||
*/
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
public String getValue()
|
||||
|
@ -125,14 +129,15 @@ public class SelectorDimFilter extends AbstractOptimizableDimFilter implements D
|
|||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public ExtractionFn getExtractionFn()
|
||||
{
|
||||
return extractionFn;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public FilterTuning getFilterTuning()
|
||||
{
|
||||
return filterTuning;
|
||||
|
|
|
@ -73,6 +73,7 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -271,6 +272,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
|
||||
@JsonProperty
|
||||
@Override
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = VirtualColumns.JsonIncludeFilter.class)
|
||||
public VirtualColumns getVirtualColumns()
|
||||
{
|
||||
return virtualColumns;
|
||||
|
@ -278,6 +280,7 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
|
||||
@Nullable
|
||||
@JsonProperty("filter")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public DimFilter getDimFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
|
@ -290,18 +293,21 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
}
|
||||
|
||||
@JsonProperty("aggregations")
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public List<AggregatorFactory> getAggregatorSpecs()
|
||||
{
|
||||
return aggregatorSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty("postAggregations")
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public List<PostAggregator> getPostAggregatorSpecs()
|
||||
{
|
||||
return postAggregatorSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty("having")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public HavingSpec getHavingSpec()
|
||||
{
|
||||
return havingSpec;
|
||||
|
@ -313,9 +319,12 @@ public class GroupByQuery extends BaseQuery<ResultRow>
|
|||
return limitSpec;
|
||||
}
|
||||
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
@JsonProperty("subtotalsSpec")
|
||||
/**
|
||||
* Subtotals spec may be empty which has a distinct meaning from {@code null}.
|
||||
*/
|
||||
@Nullable
|
||||
@JsonProperty("subtotalsSpec")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public List<List<String>> getSubtotalsSpec()
|
||||
{
|
||||
return subtotalsSpec;
|
||||
|
|
|
@ -156,7 +156,7 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
|
||||
/**
|
||||
* Limit for this query; behaves like SQL "LIMIT". Will always be positive. {@link Integer#MAX_VALUE} is used in
|
||||
* situations where the user wants an effectively unlimited resultset.
|
||||
* situations where the user wants an effectively unlimited result set.
|
||||
*/
|
||||
@JsonProperty
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = LimitJsonIncludeFilter.class)
|
||||
|
@ -580,24 +580,16 @@ public class DefaultLimitSpec implements LimitSpec
|
|||
/**
|
||||
* {@link JsonInclude} filter for {@link #getLimit()}.
|
||||
*
|
||||
* This API works by "creative" use of equals. It requires warnings to be suppressed and also requires spotbugs
|
||||
* exclusions (see spotbugs-exclude.xml).
|
||||
* This API works by "creative" use of equals. It requires warnings to be suppressed
|
||||
* and also requires spotbugs exclusions (see spotbugs-exclude.xml).
|
||||
*/
|
||||
@SuppressWarnings({"EqualsAndHashcode", "EqualsHashCode"})
|
||||
static class LimitJsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode]
|
||||
public static class LimitJsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode]
|
||||
{
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
if (obj == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (obj.getClass() == this.getClass()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return obj instanceof Long && (long) obj == Long.MAX_VALUE;
|
||||
return obj instanceof Integer && (Integer) obj == Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class DataSourceAnalysis
|
|||
|
||||
if (!(subQuery instanceof BaseQuery)) {
|
||||
// We must verify that the subQuery is a BaseQuery, because it is required to make "getBaseQuerySegmentSpec"
|
||||
// work properly. All builtin query types are BaseQuery, so we only expect this with funky extension queries.
|
||||
// work properly. All built-in query types are BaseQuery, so we only expect this with funky extension queries.
|
||||
throw new IAE("Cannot analyze subquery of class[%s]", subQuery.getClass().getName());
|
||||
}
|
||||
|
||||
|
@ -213,7 +213,7 @@ public class DataSourceAnalysis
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the bottommost (i.e. innermost) {@link Query} from a possible stack of outer queries at the root of
|
||||
* Returns the bottom-most (i.e. innermost) {@link Query} from a possible stack of outer queries at the root of
|
||||
* the datasource tree. This is the query that will be applied to the base datasource plus any joinables that might
|
||||
* be present.
|
||||
*
|
||||
|
|
|
@ -171,6 +171,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
* the query is sent to).
|
||||
*/
|
||||
public static final String CTX_KEY_OUTERMOST = "scanOutermost";
|
||||
public static final int DEFAULT_BATCH_SIZE = 4096 * 5;
|
||||
|
||||
private final VirtualColumns virtualColumns;
|
||||
private final ResultFormat resultFormat;
|
||||
|
@ -205,7 +206,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
super(dataSource, querySegmentSpec, false, context);
|
||||
this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns);
|
||||
this.resultFormat = (resultFormat == null) ? ResultFormat.RESULT_FORMAT_LIST : resultFormat;
|
||||
this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize;
|
||||
this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
|
||||
Preconditions.checkArgument(
|
||||
this.batchSize > 0,
|
||||
"batchSize must be greater than 0"
|
||||
|
@ -284,6 +285,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
|
||||
@JsonProperty
|
||||
@Override
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = VirtualColumns.JsonIncludeFilter.class)
|
||||
public VirtualColumns getVirtualColumns()
|
||||
{
|
||||
return virtualColumns;
|
||||
|
@ -296,6 +298,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = BatchSizeJsonIncludeFilter.class)
|
||||
public int getBatchSize()
|
||||
{
|
||||
return batchSize;
|
||||
|
@ -313,7 +316,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
|
||||
/**
|
||||
* Limit for this query; behaves like SQL "LIMIT". Will always be positive. {@link Long#MAX_VALUE} is used in
|
||||
* situations where the user wants an effectively unlimited resultset.
|
||||
* situations where the user wants an effectively unlimited result set.
|
||||
*/
|
||||
@JsonProperty("limit")
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = ScanRowsLimitJsonIncludeFilter.class)
|
||||
|
@ -389,6 +392,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
@Override
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public DimFilter getFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
|
@ -413,7 +417,7 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
*/
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = IsLegacyJsonIncludeFilter.class)
|
||||
public Boolean isLegacy()
|
||||
{
|
||||
return legacy;
|
||||
|
@ -643,4 +647,37 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
return obj instanceof Long && (long) obj == Long.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link JsonInclude} filter for {@link #getBatchSize()}.
|
||||
*
|
||||
* This API works by "creative" use of equals. It requires warnings to be suppressed and also requires spotbugs
|
||||
* exclusions (see spotbugs-exclude.xml).
|
||||
*/
|
||||
@SuppressWarnings({"EqualsAndHashcode", "EqualsHashCode"})
|
||||
static class BatchSizeJsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode]
|
||||
{
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
return obj instanceof Integer && (int) obj == DEFAULT_BATCH_SIZE;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link JsonInclude} filter for {@link #isLegacy()}.
|
||||
*
|
||||
* This API works by "creative" use of equals. It requires warnings to be suppressed and also requires spotbugs
|
||||
* exclusions (see spotbugs-exclude.xml).
|
||||
*/
|
||||
@SuppressWarnings({"EqualsAndHashcode", "EqualsHashCode"})
|
||||
static class IsLegacyJsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode]
|
||||
{
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
return obj == null ||
|
||||
obj instanceof Boolean && !(Boolean) obj;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.timeboundary;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import it.unimi.dsi.fastutil.bytes.ByteArrays;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -79,6 +80,7 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||
|
||||
@JsonProperty("filter")
|
||||
@Override
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public DimFilter getFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.timeseries;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -35,6 +36,7 @@ import org.apache.druid.query.Result;
|
|||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec.LimitJsonIncludeFilter;
|
||||
import org.apache.druid.query.spec.QuerySegmentSpec;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
||||
|
@ -116,30 +118,35 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
|||
|
||||
@JsonProperty
|
||||
@Override
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = VirtualColumns.JsonIncludeFilter.class)
|
||||
public VirtualColumns getVirtualColumns()
|
||||
{
|
||||
return virtualColumns;
|
||||
}
|
||||
|
||||
@JsonProperty("filter")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public DimFilter getDimensionsFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
}
|
||||
|
||||
@JsonProperty("aggregations")
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public List<AggregatorFactory> getAggregatorSpecs()
|
||||
{
|
||||
return aggregatorSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty("postAggregations")
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public List<PostAggregator> getPostAggregatorSpecs()
|
||||
{
|
||||
return postAggregatorSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty("limit")
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = LimitJsonIncludeFilter.class)
|
||||
public int getLimit()
|
||||
{
|
||||
return limit;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.topn;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -29,6 +30,8 @@ import org.apache.druid.query.ordering.StringComparator;
|
|||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
|
@ -59,7 +62,9 @@ public class DimensionTopNMetricSpec implements TopNMetricSpec
|
|||
{
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public String getPreviousStop()
|
||||
{
|
||||
return previousStop;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query.topn;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -38,6 +39,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
|
|||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -118,6 +120,7 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
|
|||
|
||||
@JsonProperty
|
||||
@Override
|
||||
@JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = VirtualColumns.JsonIncludeFilter.class)
|
||||
public VirtualColumns getVirtualColumns()
|
||||
{
|
||||
return virtualColumns;
|
||||
|
@ -141,7 +144,9 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
|
|||
return threshold;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@JsonProperty("filter")
|
||||
@JsonInclude(JsonInclude.Include.NON_NULL)
|
||||
public DimFilter getDimensionsFilter()
|
||||
{
|
||||
return dimFilter;
|
||||
|
@ -154,6 +159,7 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
|
|||
}
|
||||
|
||||
@JsonProperty("postAggregations")
|
||||
@JsonInclude(JsonInclude.Include.NON_EMPTY)
|
||||
public List<PostAggregator> getPostAggregatorSpecs()
|
||||
{
|
||||
return postAggregatorSpecs;
|
||||
|
|
|
@ -49,6 +49,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
|
|||
if (dimValues instanceof List) {
|
||||
throw new UnsupportedOperationException("Numeric columns do not support multivalue rows.");
|
||||
}
|
||||
|
||||
Double d = DimensionHandlerUtils.convertObjectToDouble(dimValues, reportParseExceptions);
|
||||
if (d == null) {
|
||||
hasNulls = NullHandling.sqlCompatible();
|
||||
|
|
|
@ -39,7 +39,7 @@ import java.util.Objects;
|
|||
|
||||
public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
||||
{
|
||||
public static final Comparator LONG_COMPARATOR = Comparators.<Long>naturalNullsFirst();
|
||||
public static final Comparator<Long> LONG_COMPARATOR = Comparators.naturalNullsFirst();
|
||||
|
||||
private volatile boolean hasNulls = false;
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
|
@ -501,4 +502,20 @@ public class VirtualColumns implements Cacheable
|
|||
return virtualColumns.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* {@link JsonInclude} filter for {@code getVirtualColumns()}.
|
||||
*
|
||||
* This API works by "creative" use of equals. It requires warnings to be suppressed
|
||||
* and also requires spotbugs exclusions (see spotbugs-exclude.xml).
|
||||
*/
|
||||
@SuppressWarnings({"EqualsAndHashcode", "EqualsHashCode"})
|
||||
public static class JsonIncludeFilter // lgtm [java/inconsistent-equals-and-hashcode]
|
||||
{
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
return obj instanceof VirtualColumns &&
|
||||
((VirtualColumns) obj).virtualColumns.isEmpty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,22 +43,15 @@ public class ScanQuerySpecTest
|
|||
String legacy =
|
||||
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
||||
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"market\",\"quality\",\"index\"],"
|
||||
+ "\"limit\":3,"
|
||||
+ "\"context\":null}";
|
||||
+ "\"limit\":3}";
|
||||
|
||||
String current =
|
||||
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
||||
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"list\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"limit\":3,"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"market\",\"quality\",\"index\"],"
|
||||
+ "\"context\":{},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}}";
|
||||
|
||||
ScanQuery query = new ScanQuery(
|
||||
|
@ -89,15 +82,10 @@ public class ScanQuerySpecTest
|
|||
String originalJson =
|
||||
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
||||
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"list\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"limit\":3,"
|
||||
+ "\"order\":\"ascending\","
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],"
|
||||
+ "\"context\":{},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}}";
|
||||
|
||||
ScanQuery expectedQuery = new ScanQuery(
|
||||
|
@ -132,15 +120,10 @@ public class ScanQuerySpecTest
|
|||
String originalJson =
|
||||
"{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},"
|
||||
+ "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"list\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"limit\":3,"
|
||||
+ "\"orderBy\":[{\"columnName\":\"quality\",\"order\":\"ascending\"}],"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],"
|
||||
+ "\"context\":{},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}}";
|
||||
|
||||
ScanQuery expectedQuery = new ScanQuery(
|
||||
|
|
|
@ -72,7 +72,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Query handler for Broker processes (see CliBroker).
|
||||
* Query handler for the Broker processes (see CliBroker).
|
||||
*
|
||||
* This class is responsible for:
|
||||
*
|
||||
|
|
|
@ -157,7 +157,7 @@ public class QueryLifecycle
|
|||
throw new ISE("Unauthorized");
|
||||
}
|
||||
|
||||
final QueryLifecycle.QueryResponse queryResponse = execute();
|
||||
final QueryResponse queryResponse = execute();
|
||||
results = queryResponse.getResults();
|
||||
}
|
||||
catch (Throwable e) {
|
||||
|
@ -278,7 +278,7 @@ public class QueryLifecycle
|
|||
|
||||
final ResponseContext responseContext = DirectDruidClient.makeResponseContextForQuery();
|
||||
|
||||
final Sequence res = QueryPlus.wrap(baseQuery)
|
||||
final Sequence<?> res = QueryPlus.wrap(baseQuery)
|
||||
.withIdentity(authenticationResult.getIdentity())
|
||||
.run(texasRanger, responseContext);
|
||||
|
||||
|
|
|
@ -57,6 +57,7 @@ public class ClientCompactionTaskTransformSpecTest
|
|||
Assert.assertEquals(expected, fromJson);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testAsMap()
|
||||
{
|
||||
|
@ -67,9 +68,9 @@ public class ClientCompactionTaskTransformSpecTest
|
|||
final ClientCompactionTaskTransformSpec spec = new ClientCompactionTaskTransformSpec(new SelectorDimFilter(dimension, value, null));
|
||||
final Map<String, Object> map = spec.asMap(objectMapper);
|
||||
Assert.assertNotNull(map);
|
||||
Assert.assertEquals(4, ((Map) map.get("filter")).size());
|
||||
Assert.assertEquals(dimension, ((Map) map.get("filter")).get("dimension"));
|
||||
Assert.assertEquals(value, ((Map) map.get("filter")).get("value"));
|
||||
Assert.assertEquals(3, ((Map<String, Object>) map.get("filter")).size());
|
||||
Assert.assertEquals(dimension, ((Map<String, Object>) map.get("filter")).get("dimension"));
|
||||
Assert.assertEquals(value, ((Map<String, Object>) map.get("filter")).get("value"));
|
||||
ClientCompactionTaskTransformSpec actual = objectMapper.convertValue(map, ClientCompactionTaskTransformSpec.class);
|
||||
Assert.assertEquals(spec, actual);
|
||||
}
|
||||
|
|
|
@ -70,7 +70,12 @@ public class DefaultRequestLogEventTest
|
|||
nativeLine);
|
||||
|
||||
String logEventJson = objectMapper.writeValueAsString(defaultRequestLogEvent);
|
||||
String expected = "{\"feed\":\"feed\",\"query\":{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"dummy\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"2015-01-01T00:00:00.000Z/2015-01-02T00:00:00.000Z\"]},\"descending\":true,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[],\"postAggregations\":[],\"limit\":5,\"context\":{\"key\":\"value\"}},\"host\":\"127.0.0.1\",\"timestamp\":\"2019-12-12T03:01:00.000Z\",\"service\":\"druid-service\",\"sql\":null,\"sqlQueryContext\":{},\"remoteAddr\":\"127.0.0.1\",\"queryStats\":{\"query/time\":13,\"query/bytes\":10,\"success\":true,\"identity\":\"allowAll\"}}";
|
||||
String expected = "{\"feed\":\"feed\",\"query\":{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"dummy\"},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"2015-01-01T00:00:00.000Z/2015-01-02T00:00:00.000Z\"]},"
|
||||
+ "\"descending\":true,\"granularity\":{\"type\":\"all\"},\"limit\":5,"
|
||||
+ "\"context\":{\"key\":\"value\"}},\"host\":\"127.0.0.1\",\"timestamp\":\"2019-12-12T03:01:00.000Z\","
|
||||
+ "\"service\":\"druid-service\",\"sql\":null,\"sqlQueryContext\":{},\"remoteAddr\":\"127.0.0.1\","
|
||||
+ "\"queryStats\":{\"query/time\":13,\"query/bytes\":10,\"success\":true,\"identity\":\"allowAll\"}}";
|
||||
Assert.assertEquals(objectMapper.readTree(expected), objectMapper.readTree(logEventJson));
|
||||
}
|
||||
|
||||
|
|
|
@ -242,8 +242,6 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
clientLosAngeles = DriverManager.getConnection(url, propertiesLosAngeles);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
|
@ -380,7 +378,7 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
ImmutableList.of(
|
||||
ImmutableMap.of(
|
||||
"PLAN",
|
||||
StringUtils.format("DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"sqlQueryId\":\"%s\",\"sqlStringifyArrays\":false,\"sqlTimeZone\":\"America/Los_Angeles\"}}], signature=[{a0:LONG}])\n",
|
||||
StringUtils.format("DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"sqlQueryId\":\"%s\",\"sqlStringifyArrays\":false,\"sqlTimeZone\":\"America/Los_Angeles\"}}], signature=[{a0:LONG}])\n",
|
||||
DUMMY_SQL_QUERY_ID
|
||||
),
|
||||
"RESOURCES",
|
||||
|
|
|
@ -392,8 +392,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testTopNLimitWrapping() throws Exception
|
||||
{
|
||||
|
@ -431,7 +429,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testTopNLimitWrappingOrderByAgg() throws Exception
|
||||
{
|
||||
|
@ -1554,7 +1551,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testOrderByLatestFloat() throws Exception
|
||||
{
|
||||
|
||||
List<Object[]> expected;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expected = ImmutableList.of(
|
||||
|
@ -2566,7 +2562,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testTopNWithSelectAndOrderByProjections() throws Exception
|
||||
{
|
||||
|
||||
testQuery(
|
||||
"SELECT\n"
|
||||
+ " dim1,"
|
||||
|
@ -3130,7 +3125,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testPruneDeadAggregators() throws Exception
|
||||
{
|
||||
|
@ -3414,7 +3408,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testNullFloatFilter() throws Exception
|
||||
{
|
||||
|
@ -4372,18 +4365,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
skipVectorize();
|
||||
|
||||
final String query = "EXPLAIN PLAN FOR SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\",\"extractionFn\":null},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n";
|
||||
final String explanation = "[{"
|
||||
+ "\"query\":{\"queryType\":\"timeseries\","
|
||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\",\"extractionFn\":null},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},"
|
||||
+ "\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},"
|
||||
+ "\"granularity\":{\"type\":\"all\"},"
|
||||
+ "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
|
||||
+ "\"postAggregations\":[],"
|
||||
+ "\"limit\":2147483647,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}},"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]"
|
||||
+ "}]";
|
||||
|
@ -4668,7 +4657,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testSimpleAggregations() throws Exception
|
||||
{
|
||||
|
@ -5399,7 +5387,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT foo.dim1, foo.dim2, l.k, l.v\n"
|
||||
+ "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n",
|
||||
"Possible error: SQL requires a join with 'GREATER_THAN' condition that is not supported."
|
||||
|
||||
);
|
||||
|
||||
for (final Map.Entry<String, String> queryErrorPair : queries.entrySet()) {
|
||||
|
@ -5505,7 +5492,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testCountStarWithBoundFilterSimplifyAnd() throws Exception
|
||||
{
|
||||
|
@ -6563,7 +6549,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testNestedGroupBy() throws Exception
|
||||
{
|
||||
|
||||
testQuery(
|
||||
"SELECT\n"
|
||||
+ " FLOOR(__time to hour) AS __time,\n"
|
||||
|
@ -7129,7 +7114,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testExplainExactCountDistinctOfSemiJoinResult() throws Exception
|
||||
{
|
||||
|
@ -7145,24 +7129,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ " )\n"
|
||||
+ ")";
|
||||
final String legacyExplanation =
|
||||
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"list\",\"batchSize\":20480,\"filter\":null,\"context\":{},\"descending\":false,\"granularity\":{\"type\":\"all\"}}},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{a0:LONG}])\n"
|
||||
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n";
|
||||
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"list\",\"granularity\":{\"type\":\"all\"}}},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
|
||||
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n";
|
||||
final String explanation = "["
|
||||
+ "{\"query\":{\"queryType\":\"groupBy\","
|
||||
+ "\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"table\",\"name\":\"foo\"},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}},\"rightPrefix\":\"j0.\",\"condition\":\"(substring(\\\"dim2\\\", 0, 1) == \\\"j0.d0\\\")\",\"joinType\":\"INNER\",\"leftFilter\":null},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}},"
|
||||
+ "\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"table\",\"name\":\"foo\"},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(substring(\\\"dim2\\\", 0, 1) == \\\"j0.d0\\\")\",\"joinType\":\"INNER\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"granularity\":{\"type\":\"all\"},"
|
||||
+ "\"dimensions\":[],"
|
||||
+ "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
|
||||
+ "\"postAggregations\":[],"
|
||||
+ "\"having\":null,"
|
||||
+ "\"limitSpec\":{\"type\":\"NoopLimitSpec\"},"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"descending\":false},"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}},"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
|
@ -7196,20 +7175,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
|
||||
// Skip vectorization since otherwise the "context" will change for each subtest.
|
||||
skipVectorize();
|
||||
String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
String explanation = "[{"
|
||||
+ "\"query\":{\"queryType\":\"scan\","
|
||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"compactedList\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
|
||||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "}]";
|
||||
|
@ -7218,14 +7192,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"query\":{\"queryType\":\"scan\","
|
||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"compactedList\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
|
||||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "}]";
|
||||
|
@ -7273,21 +7242,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ "UNION ALL (SELECT dim1 FROM druid.foo WHERE dim1 = '42'\n"
|
||||
+ "UNION ALL SELECT dim1 FROM druid.foo WHERE dim1 = '44')";
|
||||
final String legacyExplanation = "DruidUnionRel(limit=[-1])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
|
||||
+ " DruidUnionRel(limit=[-1])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\",\"extractionFn\":null},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\",\"extractionFn\":null},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n";
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"
|
||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n";
|
||||
final String explanation = "["
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "},"
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\",\"extractionFn\":null},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "},"
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\",\"extractionFn\":null},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
|
@ -7402,7 +7371,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT\n"
|
||||
+ " SUM(cnt),\n"
|
||||
+ " COUNT(*)\n"
|
||||
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)"
|
||||
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n"
|
||||
+ "WHERE cnt > 0",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
|
@ -8073,8 +8042,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testFilterOnCurrentTimestampLosAngeles() throws Exception
|
||||
{
|
||||
|
@ -11029,7 +10996,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testTimeExtractWithTooFewArguments() throws Exception
|
||||
{
|
||||
|
@ -11047,7 +11013,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception
|
||||
{
|
||||
|
@ -11247,7 +11212,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testOrderByAlongWithAliasOrderByTimeGroupByMulti() throws Exception
|
||||
{
|
||||
|
@ -11281,7 +11245,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testOrderByAlongWithAliasOrderByTimeGroupByOneCol() throws Exception
|
||||
{
|
||||
|
@ -11413,7 +11376,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testProjectAfterSort3WithoutAmbiguity() throws Exception
|
||||
{
|
||||
|
@ -11836,7 +11798,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
);
|
||||
|
||||
// nested groupby only requires time condition for inner most query
|
||||
// nested GROUP BY only requires time condition for inner most query
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT\n"
|
||||
|
@ -12884,10 +12846,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testRoundFuc() throws Exception
|
||||
public void testRoundFunc() throws Exception
|
||||
{
|
||||
|
||||
testQuery(
|
||||
|
@ -13337,8 +13297,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void testStringAgg() throws Exception
|
||||
{
|
||||
|
@ -13785,7 +13743,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* see {@link CalciteTests#RAW_ROWS1_WITH_NUMERIC_DIMS} for the input data source of this test
|
||||
*/
|
||||
|
@ -13887,8 +13844,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
this.expectedException.expect(SqlPlanningException.class);
|
||||
|
||||
/*
|
||||
* frankly speaking, the exception message thrown here is a little bit confusion
|
||||
* it says it's 'expecting 1 arguments' but acturally HUMAN_READABLE_BINARY_BYTE_FORMAT supports 1 or 2 arguments
|
||||
* frankly speaking, the exception message thrown here is a little bit confusing
|
||||
* it says it's 'expecting 1 arguments' but actually HUMAN_READABLE_BINARY_BYTE_FORMAT supports 1 or 2 arguments
|
||||
*
|
||||
* The message is returned from {@link org.apache.calcite.sql.validate.SqlValidatorImpl#handleUnresolvedFunction},
|
||||
* and we can see from its implementation that it gets the min number arguments to format the exception message.
|
||||
|
@ -13996,7 +13953,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithMutlipleConstantDimensions() throws Exception
|
||||
public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithMultipleConstantDimensions() throws Exception
|
||||
{
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
|
|
|
@ -534,18 +534,13 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"query\":{\"queryType\":\"scan\","
|
||||
+ "\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"compactedList\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"EXPR$0\"],"
|
||||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}]"
|
||||
+ "}]";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"EXPR$0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n";
|
||||
final String resources = "[]";
|
||||
|
||||
testQuery(
|
||||
|
@ -1275,22 +1270,17 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
|||
skipVectorize();
|
||||
|
||||
final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||
final String explanation = "[{"
|
||||
+ "\"query\":{\"queryType\":\"scan\","
|
||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"virtualColumns\":[],"
|
||||
+ "\"resultFormat\":\"compactedList\","
|
||||
+ "\"batchSize\":20480,"
|
||||
+ "\"filter\":null,"
|
||||
+ "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],"
|
||||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"descending\":false,"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "}]";
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
|
||||
testQuery(
|
||||
|
|
|
@ -86,11 +86,11 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||
{
|
||||
private SpecificSegmentsQuerySegmentWalker walker = null;
|
||||
private SpecificSegmentsQuerySegmentWalker walker;
|
||||
private TestServerInventoryView serverView;
|
||||
private List<ImmutableDruidServer> druidServers;
|
||||
private DruidSchema schema = null;
|
||||
private DruidSchema schema2 = null;
|
||||
private DruidSchema schema;
|
||||
private DruidSchema schema2;
|
||||
private CountDownLatch buildTableLatch = new CountDownLatch(1);
|
||||
private CountDownLatch markDataSourceLatch = new CountDownLatch(1);
|
||||
private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper();
|
||||
|
|
|
@ -1171,7 +1171,7 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
ImmutableMap.<String, Object>of(
|
||||
"PLAN",
|
||||
StringUtils.format(
|
||||
"DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"sqlQueryId\":\"%s\"}}], signature=[{a0:LONG}])\n",
|
||||
"DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"sqlQueryId\":\"%s\"}}], signature=[{a0:LONG}])\n",
|
||||
DUMMY_SQL_QUERY_ID
|
||||
),
|
||||
"RESOURCES",
|
||||
|
|
Loading…
Reference in New Issue