Enable ArrayListRowsAndColumns to StorageAdapter conversion (#15735)

This commit is contained in:
Zoltan Haindrich 2024-01-31 08:36:58 +01:00 committed by GitHub
parent 9f95a691f7
commit f701197224
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
16 changed files with 219 additions and 106 deletions

View File

@ -29,6 +29,8 @@ import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -75,8 +77,11 @@ public final class Rows
// guava's toString function fails on null objects, so please do not use it
return ((List<?>) inputValue).stream().map(String::valueOf).collect(Collectors.toList());
} else if (inputValue instanceof byte[]) {
// convert byte[] to base64 encoded string
return Collections.singletonList(StringUtils.encodeBase64String((byte[]) inputValue));
byte[] array = (byte[]) inputValue;
return objectToStringsByteA(array);
} else if (inputValue instanceof ByteBuffer) {
byte[] array = ((ByteBuffer) inputValue).array();
return objectToStringsByteA(array);
} else if (inputValue instanceof Object[]) {
return Arrays.stream((Object[]) inputValue).map(String::valueOf).collect(Collectors.toList());
} else {
@ -84,6 +89,12 @@ public final class Rows
}
}
private static List<String> objectToStringsByteA(byte[] array)
{
// convert byte[] to base64 encoded string
return Collections.singletonList(StringUtils.encodeBase64String(array));
}
/**
* Convert an object to a number.
*

View File

@ -24,6 +24,7 @@ import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntComparator;
import it.unimi.dsi.fastutil.ints.IntList;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.operator.ColumnWithDirection;
import org.apache.druid.query.rowsandcols.column.Column;
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
@ -38,6 +39,8 @@ import org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner;
import org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner;
import org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker;
import org.apache.druid.segment.RowAdapter;
import org.apache.druid.segment.RowBasedStorageAdapter;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -46,7 +49,6 @@ import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
@ -71,7 +73,8 @@ import java.util.function.Function;
public class ArrayListRowsAndColumns<RowType> implements AppendableRowsAndColumns
{
@SuppressWarnings("rawtypes")
private static final HashMap<Class<?>, Function<ArrayListRowsAndColumns, ?>> AS_MAP = makeAsMap();
private static final Map<Class<?>, Function<ArrayListRowsAndColumns, ?>> AS_MAP = RowsAndColumns
.makeAsMap(ArrayListRowsAndColumns.class);
private final ArrayList<RowType> rows;
private final RowAdapter<RowType> rowAdapter;
@ -316,41 +319,42 @@ public class ArrayListRowsAndColumns<RowType> implements AppendableRowsAndColumn
);
}
@SuppressWarnings("rawtypes")
private static HashMap<Class<?>, Function<ArrayListRowsAndColumns, ?>> makeAsMap()
@SuppressWarnings("unused")
@SemanticCreator
public ClusteredGroupPartitioner toClusteredGroupPartitioner()
{
HashMap<Class<?>, Function<ArrayListRowsAndColumns, ?>> retVal = new HashMap<>();
return new MyClusteredGroupPartitioner();
}
retVal.put(
ClusteredGroupPartitioner.class,
(Function<ArrayListRowsAndColumns, ClusteredGroupPartitioner>) rac -> rac.new MyClusteredGroupPartitioner()
);
@SuppressWarnings("unused")
@SemanticCreator
public NaiveSortMaker toNaiveSortMaker()
{
if (startOffset != 0) {
throw new ISE(
"The NaiveSortMaker should happen on the first RAC, start was [%,d], end was [%,d]",
startOffset,
endOffset
);
}
if (endOffset == rows.size()) {
// In this case, we are being sorted along with other RowsAndColumns objects, we don't have an optimized
// implementation for that, so just return null
//noinspection ReturnOfNull
return null;
}
retVal.put(
NaiveSortMaker.class,
(Function<ArrayListRowsAndColumns, NaiveSortMaker>) rac -> {
if (rac.startOffset != 0) {
throw new ISE(
"The NaiveSortMaker should happen on the first RAC, start was [%,d], end was [%,d]",
rac.startOffset,
rac.endOffset
);
}
if (rac.endOffset == rac.rows.size()) {
// In this case, we are being sorted along with other RowsAndColumns objects, we don't have an optimized
// implementation for that, so just return null
//noinspection ReturnOfNull
return null;
}
// When we are doing a naive sort and we are dealing with the first sub-window from ourselves, then we assume
// that we will see all of the other sub-windows as well, we can run through them and then sort the underlying
// rows at the very end.
return new MyNaiveSortMaker();
}
// When we are doing a naive sort and we are dealing with the first sub-window from ourselves, then we assume
// that we will see all of the other sub-windows as well, we can run through them and then sort the underlying
// rows at the very end.
return rac.new MyNaiveSortMaker();
}
);
return retVal;
@SuppressWarnings("unused")
@SemanticCreator
public StorageAdapter toStorageAdapter()
{
return new RowBasedStorageAdapter<RowType>(Sequences.simple(rows), rowAdapter, rowSignature);
}
private class MyClusteredGroupPartitioner implements ClusteredGroupPartitioner

View File

@ -67,8 +67,8 @@ import java.util.function.Function;
public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
{
private static final Map<Class<?>, Function<LazilyDecoratedRowsAndColumns, ?>> AS_MAP =
RowsAndColumns.makeAsMap(LazilyDecoratedRowsAndColumns.class);
private static final Map<Class<?>, Function<LazilyDecoratedRowsAndColumns, ?>> AS_MAP = RowsAndColumns
.makeAsMap(LazilyDecoratedRowsAndColumns.class);
private RowsAndColumns base;
private Interval interval;

View File

@ -29,8 +29,6 @@ import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.Map;
@ -96,17 +94,6 @@ public class MapOfColumnsRowsAndColumns implements RowsAndColumns
for (int i = 0; i < signature.size(); ++i) {
final ColumnType type = signature.getColumnType(i).orElse(null);
// If the column is String type, we likely got String objects instead of utf8 bytes, so convert to utf8Bytes
// to align with expectations.
if (ColumnType.STRING.equals(type)) {
for (int j = 0; j < columnOriented[i].length; j++) {
if (columnOriented[i][j] instanceof String) {
columnOriented[i][j] = ByteBuffer.wrap(((String) columnOriented[i][j]).getBytes(StandardCharsets.UTF_8));
}
}
}
bob.add(signature.getColumnName(i), columnOriented[i], type);
}
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.query.rowsandcols.concrete;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.rowsandcols.RowsAndColumns;
import org.apache.druid.query.rowsandcols.SemanticCreator;
import org.apache.druid.query.rowsandcols.column.Column;
import org.apache.druid.segment.CloseableShapeshifter;
import org.apache.druid.segment.QueryableIndex;
@ -33,14 +34,15 @@ import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter
{
private static final HashMap<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> AS_MAP = makeAsMap();
private static final Map<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> AS_MAP = RowsAndColumns
.makeAsMap(QueryableIndexRowsAndColumns.class);
private final QueryableIndex index;
@ -104,13 +106,17 @@ public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseab
}
}
private static HashMap<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> makeAsMap()
@SuppressWarnings("unused")
@SemanticCreator
public StorageAdapter toStorageAdapter()
{
HashMap<Class<?>, Function<QueryableIndexRowsAndColumns, ?>> retVal = new HashMap<>();
return new QueryableIndexStorageAdapter(index);
}
retVal.put(StorageAdapter.class, rac -> new QueryableIndexStorageAdapter(rac.index));
retVal.put(QueryableIndex.class, rac -> rac.index);
return retVal;
@SuppressWarnings("unused")
@SemanticCreator
public QueryableIndex toQueryableIndex()
{
return index;
}
}

View File

@ -55,11 +55,10 @@ public class RowBasedStorageAdapter<RowType> implements StorageAdapter
private final RowAdapter<RowType> rowAdapter;
private final RowSignature rowSignature;
RowBasedStorageAdapter(
public RowBasedStorageAdapter(
final Sequence<RowType> rowSequence,
final RowAdapter<RowType> rowAdapter,
final RowSignature rowSignature
)
final RowSignature rowSignature)
{
this.rowSequence = Preconditions.checkNotNull(rowSequence, "rowSequence");
this.rowAdapter = Preconditions.checkNotNull(rowAdapter, "rowAdapter");

View File

@ -0,0 +1,110 @@
/*
* 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.rowsandcols.semantic;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.rowsandcols.SemanticCreator;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import org.reflections.Reflections;
import org.reflections.scanners.MethodAnnotationsScanner;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
import java.util.Set;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Ensures that the usage of the {@link SemanticCreator} annotations follows some basic rules.
*/
@RunWith(Parameterized.class)
public class SemanticCreatorUsageTest
{
private final Method method;
@Parameters(name = "{0}")
public static List<Object[]> getParameters()
{
List<Object[]> params = new ArrayList<Object[]>();
Set<Method> methodsAnnotatedWith = new Reflections("org.apache", new MethodAnnotationsScanner())
.getMethodsAnnotatedWith(SemanticCreator.class);
for (Method method : methodsAnnotatedWith) {
String simpleMethodName = method.getDeclaringClass().getSimpleName() + "#" + method.getName();
params.add(new Object[] {simpleMethodName, method});
}
params.sort(Comparator.comparing(o -> (String) o[0]));
return params;
}
public SemanticCreatorUsageTest(@SuppressWarnings("unused") String simpleMethodName, Method method)
{
this.method = method;
}
/**
* {@link SemanticCreator} methods must be public to be accessible by the creator.
*/
@Test
public void testPublic()
{
int modifiers = method.getModifiers();
assertTrue(StringUtils.format("method [%s] is not public", method), Modifier.isPublic(modifiers));
}
/**
* {@link SemanticCreator} must return with an interface.
*
* An exact implementation may indicate that some interface methods might be missing.
*/
@Test
public void testReturnType()
{
Class<?> returnType = method.getReturnType();
assertTrue(
returnType + " is not an interface; this method must return with an interface; ",
returnType.isInterface()
);
}
/**
* {@link SemanticCreator} method names must follow the naming pattern toReturnType().
*
* For example: a method returning with a type of Ball should be named as "toBall"
*/
@Test
public void testMethodName()
{
Class<?> returnType = method.getReturnType();
String desiredMethodName = "to" + returnType.getSimpleName();
assertEquals("should be named as " + desiredMethodName, desiredMethodName, method.getName());
}
}

View File

@ -41,10 +41,7 @@ import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.joda.time.Duration;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -56,6 +53,8 @@ import java.util.function.Supplier;
import java.util.function.ToLongFunction;
import java.util.stream.Collectors;
import static org.junit.Assert.assertThrows;
public class RowBasedStorageAdapterTest
{
private static final String UNKNOWN_TYPE_NAME = "unknownType";
@ -209,9 +208,6 @@ public class RowBasedStorageAdapterTest
}
};
@Rule
public ExpectedException expectedException = ExpectedException.none();
public final AtomicLong numCloses = new AtomicLong();
private RowBasedStorageAdapter<Integer> createIntAdapter(final int... ints)
@ -437,8 +433,7 @@ public class RowBasedStorageAdapterTest
public void test_getNumRows()
{
final RowBasedStorageAdapter<Integer> adapter = createIntAdapter(0, 1, 2);
expectedException.expect(UnsupportedOperationException.class);
adapter.getMetadata();
assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata());
}
@Test
@ -452,8 +447,7 @@ public class RowBasedStorageAdapterTest
public void test_getMetadata()
{
final RowBasedStorageAdapter<Integer> adapter = createIntAdapter(0, 1, 2);
expectedException.expect(UnsupportedOperationException.class);
adapter.getMetadata();
assertThrows(UnsupportedOperationException.class, () -> adapter.getMetadata());
}
@Test

View File

@ -1481,7 +1481,7 @@ public class ClientQuerySegmentWalkerTest
}
}
Injector injector = QueryStackTests.injector();
Injector injector = QueryStackTests.injectorWithLookup();
walker = QueryStackTests.createClientQuerySegmentWalker(
injector,
new CapturingWalker(

View File

@ -375,7 +375,7 @@ public class QueryStackTests
return new MapJoinableFactory(setBuilder.build(), mapBuilder.build());
}
public static DruidInjectorBuilder injectorBuilder()
public static DruidInjectorBuilder defaultInjectorBuilder()
{
Injector startupInjector = new StartupInjectorBuilder()
.build();
@ -389,13 +389,13 @@ public class QueryStackTests
return injectorBuilder;
}
public static Injector injector()
public static Injector injectorWithLookup()
{
final LookupExtractorFactoryContainerProvider lookupProvider;
lookupProvider = LookupEnabledTestExprMacroTable.createTestLookupProvider(Collections.emptyMap());
return injectorBuilder()
return defaultInjectorBuilder()
.addModule(binder -> binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupProvider))
.build();
}

View File

@ -91,7 +91,7 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
public static SpecificSegmentsQuerySegmentWalker createWalker(
final QueryRunnerFactoryConglomerate conglomerate)
{
return createWalker(QueryStackTests.injector(), conglomerate);
return createWalker(QueryStackTests.injectorWithLookup(), conglomerate);
}
/**

View File

@ -1505,13 +1505,7 @@ public class DruidQuery
if (dataSource.isConcrete()) {
// Currently only non-time orderings of subqueries are allowed.
List<String> orderByColumnNames = sorting.getOrderBys()
.stream().map(OrderByColumnSpec::getDimension)
.collect(Collectors.toList());
plannerContext.setPlanningError(
"SQL query requires ordering a table by non-time column [%s], which is not supported.",
orderByColumnNames
);
setPlanningErrorOrderByNonTimeIsUnsupported();
return null;
}
@ -1544,6 +1538,17 @@ public class DruidQuery
);
}
private void setPlanningErrorOrderByNonTimeIsUnsupported()
{
List<String> orderByColumnNames = sorting.getOrderBys()
.stream().map(OrderByColumnSpec::getDimension)
.collect(Collectors.toList());
plannerContext.setPlanningError(
"SQL query requires ordering a table by non-time column [%s], which is not supported.",
orderByColumnNames
);
}
private ArrayList<ColumnWithDirection> getColumnWithDirectionsFromOrderBys(List<OrderByColumnSpec> orderBys)
{
ArrayList<ColumnWithDirection> ordering = new ArrayList<>();
@ -1623,10 +1628,7 @@ public class DruidQuery
// potential branches of exploration rather than being a semantic requirement of the query itself. So, it is
// not safe to send an error message telling the end-user exactly what is happening, instead we need to set the
// planning error and hope.
plannerContext.setPlanningError(
"SQL query requires order by non-time column [%s], which is not supported.",
orderByColumns
);
setPlanningErrorOrderByNonTimeIsUnsupported();
return null;
}
}

View File

@ -2804,7 +2804,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@NotYetSupported(Modes.PLAN_MISMATCH)
@Test
public void testGroupByWithSelectAndOrderByProjections()
{
@ -2889,7 +2889,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@NotYetSupported(Modes.PLAN_MISMATCH)
@Test
public void testTopNWithSelectAndOrderByProjections()
{
@ -2927,7 +2927,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported
@NotYetSupported(Modes.MISSING_UNION_CONVERSION)
@Test
public void testUnionAllQueries()
{
@ -2961,7 +2961,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.NOT_ENOUGH_RULES)
@NotYetSupported(Modes.MISSING_UNION_CONVERSION)
@Test
public void testUnionAllQueriesWithLimit()
{
@ -3401,7 +3401,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
* doesn't reset framework once the merge buffers
*/
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@NotYetSupported
@NotYetSupported(Modes.MISSING_UNION_CONVERSION)
@Test
public void testUnionAllSameTableThreeTimes()
{
@ -3446,7 +3446,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@NotYetSupported(Modes.NOT_ENOUGH_RULES)
@NotYetSupported(Modes.MISSING_UNION_CONVERSION)
@Test
public void testExactCountDistinctUsingSubqueryOnUnionAllTables()
{
@ -4857,7 +4857,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@NotYetSupported(Modes.PLAN_MISMATCH)
@Test
public void testGroupByWithSortOnPostAggregationDefault()
{
@ -4889,7 +4889,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@NotYetSupported(Modes.PLAN_MISMATCH)
@Test
public void testGroupByWithSortOnPostAggregationNoTopNConfig()
{
@ -4933,7 +4933,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@NotYetSupported(Modes.PLAN_MISMATCH)
@Test
public void testGroupByWithSortOnPostAggregationNoTopNContext()
{
@ -6975,7 +6975,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.NOT_ENOUGH_RULES)
@NotYetSupported(Modes.MISSING_JOIN_CONVERSION)
@Test
public void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin()
{
@ -12138,7 +12138,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported
@NotYetSupported(Modes.MISSING_JOIN_CONVERSION)
@Test
public void testRequireTimeConditionPositive()
{
@ -14635,7 +14635,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported
@NotYetSupported(Modes.MISSING_JOIN_CONVERSION)
@Test
public void testOrderByAlongWithInternalScanQuery()
{
@ -14678,7 +14678,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@NotYetSupported(Modes.NOT_ENOUGH_RULES)
@NotYetSupported(Modes.MISSING_JOIN_CONVERSION)
@Test
public void testOrderByAlongWithInternalScanQueryNoDistinct()
{

View File

@ -4362,7 +4362,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest
windowQueryTest();
}
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
@DrillTest("nestedAggs/multiWin_5")
@Test
public void test_nestedAggs_multiWin_5()

View File

@ -21,7 +21,6 @@ package org.apache.druid.sql.calcite;
import com.google.common.base.Throwables;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.UOE;
import org.junit.AssumptionViolatedException;
import org.junit.rules.TestRule;
import org.junit.runner.Description;
@ -65,7 +64,7 @@ import static org.junit.Assert.assertThrows;
@Target({ElementType.METHOD})
public @interface NotYetSupported
{
Modes value() default Modes.NOT_ENOUGH_RULES;
Modes value();
enum Modes
{
@ -80,7 +79,6 @@ public @interface NotYetSupported
NPE_PLAIN(NullPointerException.class, "java.lang.NullPointerException"),
NPE(DruidException.class, "java.lang.NullPointerException"),
AGGREGATION_NOT_SUPPORT_TYPE(DruidException.class, "Aggregation \\[(MIN|MAX)\\] does not support type \\[STRING\\]"),
CANNOT_APPLY_VIRTUAL_COL(UOE.class, "apply virtual columns"),
MISSING_DESC(DruidException.class, "function signature DESC"),
RESULT_COUNT_MISMATCH(AssertionError.class, "result count:"),
ALLDATA_CSV(DruidException.class, "allData.csv"),
@ -90,7 +88,10 @@ public @interface NotYetSupported
T_ALLTYPES_ISSUES(AssertionError.class, "(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"),
RESULT_MISMATCH(AssertionError.class, "(assertResultsEquals|AssertionError: column content mismatch)"),
UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"),
CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference");
CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference"),
MISSING_UNION_CONVERSION(DruidException.class, "Missing conversions? (is|are) LogicalUnion"),
MISSING_WINDOW_CONVERSION(DruidException.class, "Missing conversions? is Window"),
MISSING_JOIN_CONVERSION(DruidException.class, "Missing conversions? is (Logical)?Join");
public Class<? extends Throwable> throwableClass;
public String regex;

View File

@ -235,7 +235,7 @@ public class CalciteTests
null
);
public static final Injector INJECTOR = QueryStackTests.injectorBuilder()
public static final Injector INJECTOR = QueryStackTests.defaultInjectorBuilder()
.addModule(new LookylooModule())
.addModule(new SqlAggregationModule())
.addModule(new CalciteTestOperatorModule())