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:
Gian Merlino 2022-03-10 11:37:24 -08:00 committed by GitHub
parent 2efb74ff1e
commit cb2b2b696d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 104 additions and 23 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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