mirror of https://github.com/apache/druid.git
Fix error message for groupByEnableMultiValueUnnesting. (#12325)
* Fix error message for groupByEnableMultiValueUnnesting. It referred to the incorrect context parameter. Also, create a dedicated exception class, to allow easier detection of this specific error. * Fix other test. * More better error messages. * Test getDimensionName method.
This commit is contained in:
parent
2efb74ff1e
commit
cb2b2b696d
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -249,6 +250,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<ResultRow>
|
|||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Exception with one of the sequences!");
|
||||
Throwables.propagateIfPossible(e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -599,19 +599,15 @@ public class GroupByQueryEngineV2
|
|||
return indexedInts.size() == 1 ? indexedInts.get(0) : GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Throws {@link UnexpectedMultiValueDimensionException} if "allowMultiValueGrouping" is false.
|
||||
*/
|
||||
protected void checkIfMultiValueGroupingIsAllowed(String dimName)
|
||||
{
|
||||
if (!allowMultiValueGrouping) {
|
||||
throw new ISE(
|
||||
"Encountered multi-value dimension %s that cannot be processed with %s set to false."
|
||||
+ " Consider setting %s to true.",
|
||||
dimName,
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY,
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY
|
||||
);
|
||||
throw new UnexpectedMultiValueDimensionException(dimName);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class HashAggregateIterator extends GroupByEngineIterator<ByteBuffer>
|
||||
|
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* 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.groupby.epinephelinae;
|
||||
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
|
||||
public class UnexpectedMultiValueDimensionException extends RuntimeException
|
||||
{
|
||||
private final String dimensionName;
|
||||
|
||||
public UnexpectedMultiValueDimensionException(String dimensionName)
|
||||
{
|
||||
super(
|
||||
StringUtils.format(
|
||||
"Encountered multi-value dimension [%s] that cannot be processed with '%s' set to false."
|
||||
+ " Consider setting '%s' to true in your query context.",
|
||||
dimensionName,
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING,
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING
|
||||
)
|
||||
);
|
||||
|
||||
this.dimensionName = dimensionName;
|
||||
}
|
||||
|
||||
public String getDimensionName()
|
||||
{
|
||||
return dimensionName;
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -62,7 +63,7 @@ public class ArrayDoubleGroupByColumnSelectorStrategy extends ArrayNumericGroupB
|
|||
.map(a -> (Double) a)
|
||||
.collect(Collectors.toList()));
|
||||
} else {
|
||||
throw new ISE("Found unknowm type %s in ColumnValueSelector.", object.getClass().toString());
|
||||
throw new ISE("Found unexpected object type [%s] in %s array.", object.getClass().getName(), ValueType.DOUBLE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
@ -62,7 +63,7 @@ public class ArrayLongGroupByColumnSelectorStrategy extends ArrayNumericGroupByC
|
|||
.map(a -> (Long) a)
|
||||
.collect(Collectors.toList()));
|
||||
} else {
|
||||
throw new ISE("Found unknowm type %s in ColumnValueSelector.", object.getClass().toString());
|
||||
throw new ISE("Found unexpected object type [%s] in %s array.", object.getClass().getName(), ValueType.LONG);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.query.groupby.epinephelinae.Grouper;
|
|||
import org.apache.druid.query.ordering.StringComparator;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.ComparableIntArray;
|
||||
import org.apache.druid.segment.data.ComparableStringArray;
|
||||
|
||||
|
@ -173,7 +174,7 @@ public class ArrayStringGroupByColumnSelectorStrategy implements GroupByColumnSe
|
|||
intRepresentation[i] = addToIndexedDictionary((String) ((Object[]) object)[i]);
|
||||
}
|
||||
} else {
|
||||
throw new ISE("Found unknowm type %s in ColumnValueSelector.", object.getClass().toString());
|
||||
throw new ISE("Found unexpected object type [%s] in %s array.", object.getClass().getName(), ValueType.STRING);
|
||||
}
|
||||
|
||||
final ComparableIntArray comparableIntArray = ComparableIntArray.of(intRepresentation);
|
||||
|
|
|
@ -114,6 +114,7 @@ import org.apache.druid.query.filter.OrDimFilter;
|
|||
import org.apache.druid.query.filter.RegexDimFilter;
|
||||
import org.apache.druid.query.filter.SearchQueryDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.epinephelinae.UnexpectedMultiValueDimensionException;
|
||||
import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
|
||||
import org.apache.druid.query.groupby.having.DimensionSelectorHavingSpec;
|
||||
import org.apache.druid.query.groupby.having.EqualToHavingSpec;
|
||||
|
@ -137,6 +138,9 @@ import org.apache.druid.segment.data.ComparableList;
|
|||
import org.apache.druid.segment.data.ComparableStringArray;
|
||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.hamcrest.BaseMatcher;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.hamcrest.Description;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Period;
|
||||
|
@ -145,6 +149,7 @@ import org.junit.Assert;
|
|||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableCauseMatcher;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -160,6 +165,7 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
|
@ -1321,6 +1327,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMultiValueDimensionNotAllowed()
|
||||
{
|
||||
final String dimName = "placementish";
|
||||
|
||||
if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) {
|
||||
expectedException.expect(UOE.class);
|
||||
|
@ -1330,13 +1337,38 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
));
|
||||
} else if (!vectorize) {
|
||||
expectedException.expect(RuntimeException.class);
|
||||
expectedException.expectMessage(StringUtils.format(
|
||||
"Encountered multi-value dimension %s that cannot be processed with %s set to false."
|
||||
+ " Consider setting %s to true.",
|
||||
"placementish",
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY,
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY
|
||||
));
|
||||
expectedException.expectCause(CoreMatchers.instanceOf(ExecutionException.class));
|
||||
expectedException.expectCause(
|
||||
ThrowableCauseMatcher.hasCause(CoreMatchers.instanceOf(UnexpectedMultiValueDimensionException.class))
|
||||
);
|
||||
expectedException.expect(
|
||||
new BaseMatcher<Throwable>()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(Object o)
|
||||
{
|
||||
final UnexpectedMultiValueDimensionException cause =
|
||||
(UnexpectedMultiValueDimensionException) ((Throwable) o).getCause().getCause();
|
||||
|
||||
return dimName.equals(cause.getDimensionName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void describeTo(Description description)
|
||||
{
|
||||
description.appendText("an UnexpectedMultiValueDimensionException with dimension [placementish]");
|
||||
}
|
||||
}
|
||||
);
|
||||
expectedException.expectMessage(
|
||||
StringUtils.format(
|
||||
"Encountered multi-value dimension [%s] that cannot be processed with '%s' set to false."
|
||||
+ " Consider setting '%s' to true in your query context.",
|
||||
dimName,
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING,
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING
|
||||
)
|
||||
);
|
||||
} else {
|
||||
cannotVectorize();
|
||||
}
|
||||
|
@ -1344,7 +1376,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
|
|||
GroupByQuery query = makeQueryBuilder()
|
||||
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD)
|
||||
.setDimensions(new DefaultDimensionSpec("placementish", "alias"))
|
||||
.setDimensions(new DefaultDimensionSpec(dimName, "alias"))
|
||||
.setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index"))
|
||||
.setGranularity(QueryRunnerTestHelper.ALL_GRAN)
|
||||
.overrideContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false))
|
||||
|
|
|
@ -123,11 +123,11 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
|
|||
exception -> {
|
||||
exception.expect(RuntimeException.class);
|
||||
expectedException.expectMessage(StringUtils.format(
|
||||
"Encountered multi-value dimension %s that cannot be processed with %s set to false."
|
||||
+ " Consider setting %s to true.",
|
||||
"Encountered multi-value dimension [%s] that cannot be processed with '%s' set to false."
|
||||
+ " Consider setting '%s' to true in your query context.",
|
||||
"v0",
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY,
|
||||
GroupByQueryConfig.CTX_KEY_EXECUTING_NESTED_QUERY
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING,
|
||||
GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING
|
||||
));
|
||||
}
|
||||
);
|
||||
|
|
Loading…
Reference in New Issue