mirror of https://github.com/apache/druid.git
Document unsupported Join on multi-value column (#9948)
* Document Unsupported Join on multi-value column * Document Unsupported Join on multi-value column * address comments * Add unit tests * address comments * add tests
This commit is contained in:
parent
a934b2664c
commit
0d22462e07
|
@ -348,3 +348,5 @@ future versions:
|
|||
always be correct.
|
||||
- Performance-related optimizations as mentioned in the [previous section](#join-performance).
|
||||
- Join algorithms other than broadcast hash-joins.
|
||||
- Join condition on a column compared to a constant value.
|
||||
- Join conditions on a column containing a multi-value dimension.
|
||||
|
|
|
@ -709,6 +709,8 @@ Druid does not support all SQL features. In particular, the following features a
|
|||
|
||||
- JOIN between native datasources (table, lookup, subquery) and system tables.
|
||||
- JOIN conditions that are not an equality between expressions from the left- and right-hand sides.
|
||||
- JOIN conditions containing a constant value inside the condition.
|
||||
- JOIN conditions on a column which contains a multi-value dimension.
|
||||
- OVER clauses, and analytic functions such as `LAG` and `LEAD`.
|
||||
- OFFSET clauses.
|
||||
- DDL and DML.
|
||||
|
|
|
@ -0,0 +1,70 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query;
|
||||
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.net.InetAddress;
|
||||
|
||||
/**
|
||||
* This exception is for the query engine to surface when a query cannot be run. This can be due to the
|
||||
* following reasons: 1) The query is not supported yet. 2) The query is not something Druid would ever supports.
|
||||
* For these cases, the exact causes and details should also be documented in Druid user facing documents.
|
||||
*
|
||||
* As a {@link QueryException} it is expected to be serialied to a json response, but will be mapped to
|
||||
* {@link #STATUS_CODE} instead of the default HTTP 500 status.
|
||||
*/
|
||||
public class QueryUnsupportedException extends QueryException
|
||||
{
|
||||
private static final String ERROR_CLASS = QueryUnsupportedException.class.getName();
|
||||
public static final String ERROR_CODE = "Unsupported query";
|
||||
public static final int STATUS_CODE = 400;
|
||||
|
||||
@JsonCreator
|
||||
public QueryUnsupportedException(
|
||||
@JsonProperty("error") @Nullable String errorCode,
|
||||
@JsonProperty("errorMessage") String errorMessage,
|
||||
@JsonProperty("errorClass") @Nullable String errorClass,
|
||||
@JsonProperty("host") @Nullable String host
|
||||
)
|
||||
{
|
||||
super(errorCode, errorMessage, errorClass, host);
|
||||
}
|
||||
|
||||
public QueryUnsupportedException(String errorMessage)
|
||||
{
|
||||
super(ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname());
|
||||
}
|
||||
|
||||
private static String resolveHostname()
|
||||
{
|
||||
String host;
|
||||
try {
|
||||
host = InetAddress.getLocalHost().getCanonicalHostName();
|
||||
}
|
||||
catch (Exception e) {
|
||||
host = null;
|
||||
}
|
||||
return host;
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.druid.common.guava.SettableSupplier;
|
|||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.query.lookup.LookupExtractor;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
|
@ -71,9 +72,12 @@ public class LookupJoinMatcher implements JoinMatcher
|
|||
|
||||
if (row.size() == 1) {
|
||||
return selector.lookupName(row.get(0));
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now; treat them as nulls.
|
||||
} else if (row.size() == 0) {
|
||||
return null;
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now
|
||||
// TODO: Remove when https://github.com/apache/druid/issues/9924 is done
|
||||
throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported.");
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import it.unimi.dsi.fastutil.ints.IntRBTreeSet;
|
|||
import it.unimi.dsi.fastutil.ints.IntSet;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseFloatColumnValueSelector;
|
||||
import org.apache.druid.segment.BaseLongColumnValueSelector;
|
||||
|
@ -326,9 +327,12 @@ public class IndexedTableJoinMatcher implements JoinMatcher
|
|||
int dimensionId = row.get(0);
|
||||
IntList rowNumbers = getRowNumbers(selector, dimensionId);
|
||||
return rowNumbers.iterator();
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now; treat them as nulls.
|
||||
} else if (row.size() == 0) {
|
||||
return IntIterators.EMPTY_ITERATOR;
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now
|
||||
// TODO: Remove when https://github.com/apache/druid/issues/9924 is done
|
||||
throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported.");
|
||||
}
|
||||
};
|
||||
} else {
|
||||
|
@ -341,9 +345,12 @@ public class IndexedTableJoinMatcher implements JoinMatcher
|
|||
int dimensionId = row.get(0);
|
||||
IntList rowNumbers = getAndCacheRowNumbers(selector, dimensionId);
|
||||
return rowNumbers.iterator();
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now; treat them as nulls.
|
||||
} else if (row.size() == 0) {
|
||||
return IntIterators.EMPTY_ITERATOR;
|
||||
} else {
|
||||
// Multi-valued rows are not handled by the join system right now
|
||||
// TODO: Remove when https://github.com/apache/druid/issues/9924 is done
|
||||
throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported.");
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -23,14 +23,20 @@ import it.unimi.dsi.fastutil.ints.IntArrayList;
|
|||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntList;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.segment.ConstantDimensionSelector;
|
||||
import org.apache.druid.segment.DimensionDictionarySelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.ArrayBasedIndexedInts;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.runners.Enclosed;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.MockitoAnnotations;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -48,12 +54,89 @@ public class IndexedTableJoinMatcherTest
|
|||
{
|
||||
public static class MakeDimensionProcessorTest
|
||||
{
|
||||
@Mock
|
||||
private DimensionSelector dimensionSelector;
|
||||
|
||||
private static final String KEY = "key";
|
||||
|
||||
static {
|
||||
NullHandling.initializeForTests();
|
||||
}
|
||||
|
||||
@SuppressWarnings("ReturnValueIgnored")
|
||||
@Test(expected = QueryUnsupportedException.class)
|
||||
public void testMatchMultiValuedRowCardinalityUnknownShouldThrowException()
|
||||
{
|
||||
MockitoAnnotations.initMocks(this);
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6});
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality();
|
||||
|
||||
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
|
||||
new IndexedTableJoinMatcher.ConditionMatcherFactory(
|
||||
ValueType.STRING,
|
||||
IndexedTableJoinMatcherTest::createSingletonIntList
|
||||
);
|
||||
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false);
|
||||
// Test match should throw exception
|
||||
dimensionProcessor.get();
|
||||
}
|
||||
|
||||
@SuppressWarnings("ReturnValueIgnored")
|
||||
@Test(expected = QueryUnsupportedException.class)
|
||||
public void testMatchMultiValuedRowCardinalityKnownShouldThrowException()
|
||||
{
|
||||
MockitoAnnotations.initMocks(this);
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6});
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
Mockito.doReturn(3).when(dimensionSelector).getValueCardinality();
|
||||
|
||||
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
|
||||
new IndexedTableJoinMatcher.ConditionMatcherFactory(
|
||||
ValueType.STRING,
|
||||
IndexedTableJoinMatcherTest::createSingletonIntList
|
||||
);
|
||||
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false);
|
||||
// Test match should throw exception
|
||||
dimensionProcessor.get();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchEmptyRowCardinalityUnknown()
|
||||
{
|
||||
MockitoAnnotations.initMocks(this);
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{});
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality();
|
||||
|
||||
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
|
||||
new IndexedTableJoinMatcher.ConditionMatcherFactory(
|
||||
ValueType.STRING,
|
||||
IndexedTableJoinMatcherTest::createSingletonIntList
|
||||
);
|
||||
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false);
|
||||
Assert.assertNotNull(dimensionProcessor.get());
|
||||
Assert.assertFalse(dimensionProcessor.get().hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchEmptyRowCardinalityKnown()
|
||||
{
|
||||
MockitoAnnotations.initMocks(this);
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{});
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
Mockito.doReturn(0).when(dimensionSelector).getValueCardinality();
|
||||
|
||||
IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory =
|
||||
new IndexedTableJoinMatcher.ConditionMatcherFactory(
|
||||
ValueType.STRING,
|
||||
IndexedTableJoinMatcherTest::createSingletonIntList
|
||||
);
|
||||
Supplier<IntIterator> dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false);
|
||||
Assert.assertNotNull(dimensionProcessor.get());
|
||||
Assert.assertFalse(dimensionProcessor.get().hasNext());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getsCorrectResultWhenSelectorCardinalityUnknown()
|
||||
{
|
||||
|
|
|
@ -22,11 +22,13 @@ package org.apache.druid.segment.join.table;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.lookup.LookupExtractor;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.data.ArrayBasedIndexedInts;
|
||||
import org.apache.druid.segment.data.SingleIndexedInt;
|
||||
import org.apache.druid.segment.join.JoinConditionAnalysis;
|
||||
import org.apache.druid.segment.join.lookup.LookupJoinMatcher;
|
||||
|
@ -150,6 +152,40 @@ public class LookupJoinMatcherTest
|
|||
Assert.assertFalse(target.hasMatch());
|
||||
}
|
||||
|
||||
@Test(expected = QueryUnsupportedException.class)
|
||||
public void testMatchMultiValuedRowShouldThrowException()
|
||||
{
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6});
|
||||
Mockito.doReturn(dimensionSelector).when(leftSelectorFactory).makeDimensionSelector(ArgumentMatchers.any(DimensionSpec.class));
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
|
||||
JoinConditionAnalysis condition = JoinConditionAnalysis.forExpression(
|
||||
StringUtils.format("\"%sk\" == foo", PREFIX),
|
||||
PREFIX,
|
||||
ExprMacroTable.nil()
|
||||
);
|
||||
target = LookupJoinMatcher.create(extractor, leftSelectorFactory, condition, true);
|
||||
// Test match should throw exception
|
||||
target.matchCondition();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMatchEmptyRow()
|
||||
{
|
||||
ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{});
|
||||
Mockito.doReturn(dimensionSelector).when(leftSelectorFactory).makeDimensionSelector(ArgumentMatchers.any(DimensionSpec.class));
|
||||
Mockito.doReturn(row).when(dimensionSelector).getRow();
|
||||
|
||||
JoinConditionAnalysis condition = JoinConditionAnalysis.forExpression(
|
||||
StringUtils.format("\"%sk\" == foo", PREFIX),
|
||||
PREFIX,
|
||||
ExprMacroTable.nil()
|
||||
);
|
||||
target = LookupJoinMatcher.create(extractor, leftSelectorFactory, condition, true);
|
||||
target.matchCondition();
|
||||
Assert.assertFalse(target.hasMatch());
|
||||
}
|
||||
|
||||
private void verifyMatch(String expectedKey, String expectedValue)
|
||||
{
|
||||
DimensionSelector selector = target.getColumnSelectorFactory()
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.druid.query.Query;
|
|||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryInterruptedException;
|
||||
import org.apache.druid.query.QueryToolChest;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.query.context.ResponseContext;
|
||||
import org.apache.druid.server.metrics.QueryCountStatsProvider;
|
||||
import org.apache.druid.server.security.Access;
|
||||
|
@ -315,6 +316,11 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
queryLifecycle.emitLogsAndMetrics(cap, req.getRemoteAddr(), -1);
|
||||
return ioReaderWriter.gotLimited(cap);
|
||||
}
|
||||
catch (QueryUnsupportedException unsupported) {
|
||||
failedQueryCount.incrementAndGet();
|
||||
queryLifecycle.emitLogsAndMetrics(unsupported, req.getRemoteAddr(), -1);
|
||||
return ioReaderWriter.gotUnsupported(unsupported);
|
||||
}
|
||||
catch (ForbiddenException e) {
|
||||
// don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and
|
||||
// send an error response if this is thrown.
|
||||
|
@ -446,6 +452,13 @@ public class QueryResource implements QueryCountStatsProvider
|
|||
.entity(newOutputWriter(null, null, false).writeValueAsBytes(e))
|
||||
.build();
|
||||
}
|
||||
|
||||
Response gotUnsupported(QueryUnsupportedException e) throws IOException
|
||||
{
|
||||
return Response.status(QueryUnsupportedException.STATUS_CODE)
|
||||
.entity(newOutputWriter(null, null, false).writeValueAsBytes(e))
|
||||
.build();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.query.Query;
|
|||
import org.apache.druid.query.QueryRunner;
|
||||
import org.apache.druid.query.QuerySegmentWalker;
|
||||
import org.apache.druid.query.QueryToolChestWarehouse;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.query.Result;
|
||||
import org.apache.druid.query.SegmentDescriptor;
|
||||
import org.apache.druid.query.timeboundary.TimeBoundaryResultValue;
|
||||
|
@ -339,6 +340,33 @@ public class QueryResourceTest
|
|||
Assert.assertEquals(Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnsupportedQueryThrowsException() throws IOException
|
||||
{
|
||||
String errorMessage = "This will be support in Druid 9999";
|
||||
ByteArrayInputStream badQuery = EasyMock.createMock(ByteArrayInputStream.class);
|
||||
EasyMock.expect(badQuery.read(EasyMock.anyObject(), EasyMock.anyInt(), EasyMock.anyInt())).andThrow(
|
||||
new QueryUnsupportedException(errorMessage));
|
||||
EasyMock.replay(badQuery);
|
||||
EasyMock.replay(testServletRequest);
|
||||
Response response = queryResource.doPost(
|
||||
badQuery,
|
||||
null /*pretty*/,
|
||||
testServletRequest
|
||||
);
|
||||
Assert.assertNotNull(response);
|
||||
Assert.assertEquals(QueryUnsupportedException.STATUS_CODE, response.getStatus());
|
||||
QueryUnsupportedException ex;
|
||||
try {
|
||||
ex = JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryUnsupportedException.class);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
Assert.assertEquals(errorMessage, ex.getMessage());
|
||||
Assert.assertEquals(QueryUnsupportedException.ERROR_CODE, ex.getErrorCode());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecuredQuery() throws Exception
|
||||
{
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.guava.Yielder;
|
|||
import org.apache.druid.java.util.common.guava.Yielders;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.QueryInterruptedException;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.server.QueryCapacityExceededException;
|
||||
import org.apache.druid.server.security.ForbiddenException;
|
||||
import org.apache.druid.sql.SqlLifecycle;
|
||||
|
@ -176,6 +177,11 @@ public class SqlResource
|
|||
lifecycle.emitLogsAndMetrics(cap, remoteAddr, -1);
|
||||
return Response.status(QueryCapacityExceededException.STATUS_CODE).entity(jsonMapper.writeValueAsBytes(cap)).build();
|
||||
}
|
||||
catch (QueryUnsupportedException unsupported) {
|
||||
log.warn(unsupported, "Failed to handle query: %s", sqlQuery);
|
||||
lifecycle.emitLogsAndMetrics(unsupported, remoteAddr, -1);
|
||||
return Response.status(QueryUnsupportedException.STATUS_CODE).entity(jsonMapper.writeValueAsBytes(unsupported)).build();
|
||||
}
|
||||
catch (ForbiddenException e) {
|
||||
throw e; // let ForbiddenExceptionMapper handle this
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.query.Druids;
|
|||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.QueryException;
|
||||
import org.apache.druid.query.ResourceLimitExceededException;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
|
@ -1790,6 +1791,23 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
// This query is expected to fail as we do not support join on multi valued column
|
||||
// (see issue https://github.com/apache/druid/issues/9924 for more information)
|
||||
// TODO: Remove expected Exception when https://github.com/apache/druid/issues/9924 is fixed
|
||||
@Test(expected = QueryException.class)
|
||||
@Parameters(source = QueryContextForJoinProvider.class)
|
||||
public void testJoinOnMultiValuedColumnShouldThrowException(Map<String, Object> queryContext) throws Exception
|
||||
{
|
||||
final String query = "SELECT dim3, l.v from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k\n";
|
||||
|
||||
testQuery(
|
||||
query,
|
||||
queryContext,
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAnyAggregatorsDoesNotSkipNulls() throws Exception
|
||||
{
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.druid.math.expr.ExprMacroTable;
|
|||
import org.apache.druid.query.QueryException;
|
||||
import org.apache.druid.query.QueryInterruptedException;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.QueryUnsupportedException;
|
||||
import org.apache.druid.query.ResourceLimitExceededException;
|
||||
import org.apache.druid.server.QueryCapacityExceededException;
|
||||
import org.apache.druid.server.QueryScheduler;
|
||||
|
@ -717,6 +718,22 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
checkSqlRequestLog(false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnsupportedQueryThrowsException() throws Exception
|
||||
{
|
||||
String errorMessage = "This will be support in Druid 9999";
|
||||
SqlQuery badQuery = EasyMock.createMock(SqlQuery.class);
|
||||
EasyMock.expect(badQuery.getQuery()).andReturn("SELECT ANSWER TO LIFE");
|
||||
EasyMock.expect(badQuery.getContext()).andReturn(ImmutableMap.of());
|
||||
EasyMock.expect(badQuery.getParameterList()).andThrow(new QueryUnsupportedException(errorMessage));
|
||||
EasyMock.replay(badQuery);
|
||||
final QueryException exception = doPost(badQuery).lhs;
|
||||
|
||||
Assert.assertNotNull(exception);
|
||||
Assert.assertEquals(exception.getErrorCode(), QueryUnsupportedException.ERROR_CODE);
|
||||
Assert.assertEquals(exception.getErrorClass(), QueryUnsupportedException.class.getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTooManyRequests() throws Exception
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue