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:
Maytas Monsereenusorn 2020-06-03 09:55:52 -10:00 committed by GitHub
parent a934b2664c
commit 0d22462e07
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 292 additions and 6 deletions

View File

@ -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.

View File

@ -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.

View File

@ -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;
}
}

View File

@ -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.");
}
};
}

View File

@ -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.");
}
};
}

View File

@ -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()
{

View File

@ -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()

View File

@ -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

View File

@ -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
{

View File

@ -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
}

View File

@ -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
{

View File

@ -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
{