mirror of https://github.com/apache/druid.git
add a sql option to force user to specify time condition (#6246)
* add a sql option to force user to specify time condition * rename forceTimeCondition to requireTimeCondition, refine error message
This commit is contained in:
parent
2fac6743d4
commit
edf0c13807
|
@ -1243,6 +1243,7 @@ The Druid SQL server is configured through the following properties on the broke
|
|||
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|
||||
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|
||||
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|
||||
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|
||||
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|
||||
|
||||
### Broker Caching
|
||||
|
|
|
@ -540,4 +540,5 @@ The Druid SQL server is configured through the following properties on the broke
|
|||
|`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinalty algorithm for `COUNT(DISTINCT foo)`.|true|
|
||||
|`druid.sql.planner.useApproximateTopN`|Whether to use approximate [TopN queries](../querying/topnquery.html) when a SQL query could be expressed as such. If false, exact [GroupBy queries](../querying/groupbyquery.html) will be used instead.|true|
|
||||
|`druid.sql.planner.useFallback`|Whether to evaluate operations on the broker when they cannot be expressed as Druid queries. This option is not recommended for production since it can generate unscalable query plans. If false, SQL queries that cannot be translated to Druid queries will fail.|false|
|
||||
|`druid.sql.planner.requireTimeCondition`|Whether to require SQL to have filter conditions on __time column so that all generated native queries will have user specified intervals. If true, all queries wihout filter condition on __time column will fail|false|
|
||||
|`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC|
|
||||
|
|
|
@ -57,6 +57,9 @@ public class PlannerConfig
|
|||
@JsonProperty
|
||||
private boolean useFallback = false;
|
||||
|
||||
@JsonProperty
|
||||
private boolean requireTimeCondition = false;
|
||||
|
||||
@JsonProperty
|
||||
private DateTimeZone sqlTimeZone = DateTimeZone.UTC;
|
||||
|
||||
|
@ -100,6 +103,11 @@ public class PlannerConfig
|
|||
return useFallback;
|
||||
}
|
||||
|
||||
public boolean isRequireTimeCondition()
|
||||
{
|
||||
return requireTimeCondition;
|
||||
}
|
||||
|
||||
public DateTimeZone getSqlTimeZone()
|
||||
{
|
||||
return sqlTimeZone;
|
||||
|
@ -132,6 +140,7 @@ public class PlannerConfig
|
|||
CTX_KEY_USE_FALLBACK,
|
||||
isUseFallback()
|
||||
);
|
||||
newConfig.requireTimeCondition = isRequireTimeCondition();
|
||||
newConfig.sqlTimeZone = getSqlTimeZone();
|
||||
return newConfig;
|
||||
}
|
||||
|
@ -171,6 +180,7 @@ public class PlannerConfig
|
|||
useApproximateCountDistinct == that.useApproximateCountDistinct &&
|
||||
useApproximateTopN == that.useApproximateTopN &&
|
||||
useFallback == that.useFallback &&
|
||||
requireTimeCondition == that.requireTimeCondition &&
|
||||
Objects.equals(metadataRefreshPeriod, that.metadataRefreshPeriod) &&
|
||||
Objects.equals(sqlTimeZone, that.sqlTimeZone);
|
||||
}
|
||||
|
@ -188,6 +198,7 @@ public class PlannerConfig
|
|||
useApproximateCountDistinct,
|
||||
useApproximateTopN,
|
||||
useFallback,
|
||||
requireTimeCondition,
|
||||
sqlTimeZone
|
||||
);
|
||||
}
|
||||
|
@ -204,6 +215,7 @@ public class PlannerConfig
|
|||
", useApproximateCountDistinct=" + useApproximateCountDistinct +
|
||||
", useApproximateTopN=" + useApproximateTopN +
|
||||
", useFallback=" + useFallback +
|
||||
", requireTimeCondition=" + requireTimeCondition +
|
||||
", sqlTimeZone=" + sqlTimeZone +
|
||||
'}';
|
||||
}
|
||||
|
|
|
@ -28,10 +28,12 @@ import com.google.common.primitives.Ints;
|
|||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.Result;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
|
@ -98,6 +100,14 @@ public class QueryMaker
|
|||
{
|
||||
final Query query = druidQuery.getQuery();
|
||||
|
||||
final Query innerMostQuery = findInnerMostQuery(query);
|
||||
if (plannerContext.getPlannerConfig().isRequireTimeCondition() &&
|
||||
innerMostQuery.getIntervals().equals(Intervals.ONLY_ETERNITY)) {
|
||||
throw new CannotBuildQueryException(
|
||||
"requireTimeCondition is enabled, all queries must include a filter condition on the __time column"
|
||||
);
|
||||
}
|
||||
|
||||
if (query instanceof TimeseriesQuery) {
|
||||
return executeTimeseries(druidQuery, (TimeseriesQuery) query);
|
||||
} else if (query instanceof TopNQuery) {
|
||||
|
@ -113,6 +123,15 @@ public class QueryMaker
|
|||
}
|
||||
}
|
||||
|
||||
private Query findInnerMostQuery(Query outerQuery)
|
||||
{
|
||||
Query query = outerQuery;
|
||||
while (query.getDataSource() instanceof QueryDataSource) {
|
||||
query = ((QueryDataSource) query.getDataSource()).getQuery();
|
||||
}
|
||||
return query;
|
||||
}
|
||||
|
||||
private Sequence<Object[]> executeScan(
|
||||
final DruidQuery druidQuery,
|
||||
final ScanQuery query
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.druid.common.config.NullHandling;
|
|||
import org.apache.druid.hll.HLLCV1;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.JodaUtils;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -103,6 +104,7 @@ import org.apache.druid.sql.calcite.planner.PlannerConfig;
|
|||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerFactory;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerResult;
|
||||
import org.apache.druid.sql.calcite.rel.CannotBuildQueryException;
|
||||
import org.apache.druid.sql.calcite.schema.DruidSchema;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTestBase;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
|
@ -140,6 +142,13 @@ public class CalciteQueryTest extends CalciteTestBase
|
|||
private static final Logger log = new Logger(CalciteQueryTest.class);
|
||||
|
||||
private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig();
|
||||
private static final PlannerConfig PLANNER_CONFIG_REQUIRE_TIME_CONDITION = new PlannerConfig() {
|
||||
@Override
|
||||
public boolean isRequireTimeCondition()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
};
|
||||
private static final PlannerConfig PLANNER_CONFIG_NO_TOPN = new PlannerConfig()
|
||||
{
|
||||
@Override
|
||||
|
@ -7451,6 +7460,173 @@ public class CalciteQueryTest extends CalciteTestBase
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequireTimeConditionPositive() throws Exception
|
||||
{
|
||||
// simple timeseries
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT SUM(cnt), gran FROM (\n"
|
||||
+ " SELECT __time as t, floor(__time TO month) AS gran,\n"
|
||||
+ " cnt FROM druid.foo\n"
|
||||
+ ") AS x\n"
|
||||
+ "WHERE t >= '2000-01-01' and t < '2002-01-01'"
|
||||
+ "GROUP BY gran\n"
|
||||
+ "ORDER BY gran",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Intervals.of("2000-01-01/2002-01-01")))
|
||||
.granularity(Granularities.MONTH)
|
||||
.aggregators(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{3L, T("2000-01-01")},
|
||||
new Object[]{3L, T("2001-01-01")}
|
||||
)
|
||||
);
|
||||
|
||||
// nested groupby only requires time condition for inner most query
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT\n"
|
||||
+ " SUM(cnt),\n"
|
||||
+ " COUNT(*)\n"
|
||||
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo WHERE __time >= '2000-01-01' GROUP BY dim2)",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(QSS(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(DIMS(new DefaultDimensionSpec("dim2", "d0")))
|
||||
.setAggregatorSpecs(AGGS(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setInterval(QSS(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(AGGS(
|
||||
new LongSumAggregatorFactory("_a0", "a0"),
|
||||
new CountAggregatorFactory("_a1")
|
||||
))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
NullHandling.replaceWithDefault() ?
|
||||
ImmutableList.of(
|
||||
new Object[]{6L, 3L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{6L, 4L}
|
||||
)
|
||||
);
|
||||
|
||||
// semi-join requires time condition on both left and right query
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT COUNT(*) FROM druid.foo\n"
|
||||
+ "WHERE __time >= '2000-01-01' AND SUBSTRING(dim2, 1, 1) IN (\n"
|
||||
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo\n"
|
||||
+ " WHERE dim1 <> '' AND __time >= '2000-01-01'\n"
|
||||
+ ")",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(QSS(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimFilter(NOT(SELECTOR("dim1", "", null)))
|
||||
.setDimensions(DIMS(new ExtractionDimensionSpec(
|
||||
"dim1",
|
||||
"d0",
|
||||
new SubstringDimExtractionFn(0, 1)
|
||||
)))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build(),
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(QSS(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT)))
|
||||
.granularity(Granularities.ALL)
|
||||
.filters(IN(
|
||||
"dim2",
|
||||
ImmutableList.of("1", "2", "a", "d"),
|
||||
new SubstringDimExtractionFn(0, 1)
|
||||
))
|
||||
.aggregators(AGGS(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{3L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequireTimeConditionSimpleQueryNegative() throws Exception
|
||||
{
|
||||
expectedException.expect(CannotBuildQueryException.class);
|
||||
expectedException.expectMessage("__time column");
|
||||
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT SUM(cnt), gran FROM (\n"
|
||||
+ " SELECT __time as t, floor(__time TO month) AS gran,\n"
|
||||
+ " cnt FROM druid.foo\n"
|
||||
+ ") AS x\n"
|
||||
+ "GROUP BY gran\n"
|
||||
+ "ORDER BY gran",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequireTimeConditionSubQueryNegative() throws Exception
|
||||
{
|
||||
expectedException.expect(CannotBuildQueryException.class);
|
||||
expectedException.expectMessage("__time column");
|
||||
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT\n"
|
||||
+ " SUM(cnt),\n"
|
||||
+ " COUNT(*)\n"
|
||||
+ "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2)",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequireTimeConditionSemiJoinNegative() throws Exception
|
||||
{
|
||||
expectedException.expect(CannotBuildQueryException.class);
|
||||
expectedException.expectMessage("__time column");
|
||||
|
||||
testQuery(
|
||||
PLANNER_CONFIG_REQUIRE_TIME_CONDITION,
|
||||
"SELECT COUNT(*) FROM druid.foo\n"
|
||||
+ "WHERE SUBSTRING(dim2, 1, 1) IN (\n"
|
||||
+ " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo\n"
|
||||
+ " WHERE dim1 <> '' AND __time >= '2000-01-01'\n"
|
||||
+ ")",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
private void testQuery(
|
||||
final String sql,
|
||||
final List<Query> expectedQueries,
|
||||
|
|
Loading…
Reference in New Issue