Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.spectator.histogram.SpectatorHistogramAggregatorFactory;
Expand Down Expand Up @@ -123,7 +124,12 @@ private Aggregation handleSinglePercentile(
final List<Aggregation> existingAggregations
)
{
final double percentile = ((Number) RexLiteral.value(percentileArg)).doubleValue();
final Object value = RexLiteral.value(percentileArg);
if (!(value instanceof Number)) {
throw InvalidSqlInput.exception("SPECTATOR_PERCENTILE percentile parameter must be a numeric literal, got %s",
value == null ? "NULL" : value.getClass().getSimpleName());
}
final double percentile = ((Number) value).doubleValue();

final String histogramName = StringUtils.format("%s:agg", name);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,9 @@
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.error.DruidException;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
Expand Down Expand Up @@ -551,4 +553,33 @@ public void testSpectatorFunctionsOnNullHistogram()
ImmutableList.of(new Object[]{null, null, null})
);
}

@Test
public void testSpectatorPercentileWithStringLiteral()
{
// verify invalid queries return 400 (user error)
final String query = "SELECT SPECTATOR_PERCENTILE(histogram_metric, '99.99') FROM foo";

try {
testQuery(query, ImmutableList.of(), ImmutableList.of());
throw new AssertionError("Expected DruidException but query succeeded");
}
catch (DruidException e) {
if (e.getTargetPersona() != DruidException.Persona.USER) {
throw new AssertionError(
StringUtils.format("Expected USER persona but got %s", e.getTargetPersona())
);
}
if (e.getCategory() != DruidException.Category.INVALID_INPUT) {
throw new AssertionError(
StringUtils.format("Expected INVALID_INPUT category but got %s", e.getCategory())
);
}
if (!e.getMessage().contains("must be a numeric literal")) {
throw new AssertionError(
StringUtils.format("Expected message about numeric literal but got: %s", e.getMessage())
);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringEncoding;
import org.apache.druid.query.aggregation.AggregatorFactory;
Expand Down Expand Up @@ -92,7 +93,14 @@ public Aggregation toDruidAggregation(
return null;
}

logK = ((Number) RexLiteral.value(logKarg)).intValue();
final Object logKValue = RexLiteral.value(logKarg);
if (!(logKValue instanceof Number)) {
throw InvalidSqlInput.exception(
"DS_HLL logK parameter must be a numeric literal, got %s",
logKValue == null ? "NULL" : logKValue.getClass().getSimpleName()
);
}
logK = ((Number) logKValue).intValue();
} else {
logK = HllSketchAggregatorFactory.DEFAULT_LG_K;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.aggregation.AggregatorFactory;
Expand Down Expand Up @@ -98,7 +99,14 @@ public Aggregation toDruidAggregation(
return null;
}

final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
final Object probabilityValue = RexLiteral.value(probabilityArg);
if (!(probabilityValue instanceof Number)) {
throw InvalidSqlInput.exception(
"DS_QUANTILES_SKETCH probability parameter must be a numeric literal, got %s",
probabilityValue == null ? "NULL" : probabilityValue.getClass().getSimpleName()
);
}
final float probability = ((Number) probabilityValue).floatValue();
final int k;

if (aggregateCall.getArgList().size() >= 3) {
Expand All @@ -109,7 +117,14 @@ public Aggregation toDruidAggregation(
return null;
}

k = ((Number) RexLiteral.value(resolutionArg)).intValue();
final Object resolutionValue = RexLiteral.value(resolutionArg);
if (!(resolutionValue instanceof Number)) {
throw InvalidSqlInput.exception(
"DS_QUANTILES_SKETCH resolution parameter must be a numeric literal, got %s",
resolutionValue == null ? "NULL" : resolutionValue.getClass().getSimpleName()
);
}
k = ((Number) resolutionValue).intValue();
} else {
k = DoublesSketchAggregatorFactory.DEFAULT_K;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlKind;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
Expand Down Expand Up @@ -90,7 +91,14 @@ public Aggregation toDruidAggregation(
return null;
}

sketchSize = ((Number) RexLiteral.value(sketchSizeArg)).intValue();
final Object sketchSizeValue = RexLiteral.value(sketchSizeArg);
if (!(sketchSizeValue instanceof Number)) {
throw InvalidSqlInput.exception(
"DS_THETA size parameter must be a numeric literal, got %s",
sketchSizeValue == null ? "NULL" : sketchSizeValue.getClass().getSimpleName()
);
}
sketchSize = ((Number) sketchSizeValue).intValue();
} else {
sketchSize = SketchAggregatorFactory.DEFAULT_MAX_SKETCH_SIZE;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.bloom.BloomFilterAggregatorFactory;
Expand Down Expand Up @@ -88,7 +89,14 @@ public Aggregation toDruidAggregation(
return null;
}

final int maxNumEntries = ((Number) RexLiteral.value(maxNumEntriesOperand)).intValue();
final Object maxNumEntriesValue = RexLiteral.value(maxNumEntriesOperand);
if (!(maxNumEntriesValue instanceof Number)) {
throw InvalidSqlInput.exception(
"BLOOM_FILTER maxNumEntries parameter must be a numeric literal, got %s",
maxNumEntriesValue == null ? "NULL" : maxNumEntriesValue.getClass().getSimpleName()
);
}
final int maxNumEntries = ((Number) maxNumEntriesValue).intValue();

// Look for existing matching aggregatorFactory.
for (final Aggregation existing : existingAggregations) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.histogram.ApproximateHistogram;
Expand Down Expand Up @@ -91,7 +92,14 @@ public Aggregation toDruidAggregation(
return null;
}

final float probability = ((Number) RexLiteral.value(probabilityArg)).floatValue();
final Object probabilityValue = RexLiteral.value(probabilityArg);
if (!(probabilityValue instanceof Number)) {
throw InvalidSqlInput.exception(
"APPROX_QUANTILE probability parameter must be a numeric literal, got %s",
probabilityValue == null ? "NULL" : probabilityValue.getClass().getSimpleName()
);
}
final float probability = ((Number) probabilityValue).floatValue();
final int resolution;

if (aggregateCall.getArgList().size() >= 3) {
Expand All @@ -102,7 +110,14 @@ public Aggregation toDruidAggregation(
return null;
}

resolution = ((Number) RexLiteral.value(resolutionArg)).intValue();
final Object resolutionValue = RexLiteral.value(resolutionArg);
if (!(resolutionValue instanceof Number)) {
throw InvalidSqlInput.exception(
"APPROX_QUANTILE resolution parameter must be a numeric literal, got %s",
resolutionValue == null ? "NULL" : resolutionValue.getClass().getSimpleName()
);
}
resolution = ((Number) resolutionValue).intValue();
} else {
resolution = ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
Expand Down Expand Up @@ -83,7 +84,14 @@ public Aggregation toDruidAggregation(
// maxBytes must be a literal
return null;
}
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
final Object maxBytesValue = RexLiteral.value(maxBytes);
if (!(maxBytesValue instanceof Number)) {
throw InvalidSqlInput.exception(
"ARRAY_CONCAT_AGG maxBytes parameter must be a numeric literal, got %s",
maxBytesValue == null ? "NULL" : maxBytesValue.getClass().getSimpleName()
);
}
maxSizeBytes = ((Number) maxBytesValue).intValue();
}
final DruidExpression arg = Expressions.toDruidExpression(plannerContext, inputAccessor.getInputRowSignature(), arguments.get(0));
final ExprMacroTable macroTable = plannerContext.getPlannerToolbox().exprMacroTable();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
Expand Down Expand Up @@ -86,7 +87,14 @@ public Aggregation toDruidAggregation(
// maxBytes must be a literal
return null;
}
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
final Object maxBytesValue = RexLiteral.value(maxBytes);
if (!(maxBytesValue instanceof Number)) {
throw InvalidSqlInput.exception(
"ARRAY_AGG maxBytes parameter must be a numeric literal, got %s",
maxBytesValue == null ? "NULL" : maxBytesValue.getClass().getSimpleName()
);
}
maxSizeBytes = ((Number) maxBytesValue).intValue();
}
final DruidExpression arg = Expressions.toDruidExpression(plannerContext, inputAccessor.getInputRowSignature(), arguments.get(0));
if (arg == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.Optionality;
import org.apache.druid.error.InvalidSqlInput;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
Expand Down Expand Up @@ -130,7 +131,14 @@ public Aggregation toDruidAggregation(
// maxBytes must be a literal
return null;
}
maxSizeBytes = ((Number) RexLiteral.value(maxBytes)).intValue();
final Object maxBytesValue = RexLiteral.value(maxBytes);
if (!(maxBytesValue instanceof Number)) {
throw InvalidSqlInput.exception(
"STRING_AGG maxBytes parameter must be a numeric literal, got %s",
maxBytesValue == null ? "NULL" : maxBytesValue.getClass().getSimpleName()
);
}
maxSizeBytes = ((Number) maxBytesValue).intValue();
}

final DruidExpression arg = arguments.get(0);
Expand Down
Loading