mirror of https://github.com/apache/druid.git
Add columnMappings to explain plan output (#14187)
* Add columnMappings to explain plan output * * fix checkstyle * add tests * * improve test coverage * * temporarily remove unit-test need to run ITs * * depend on build * * temporarily lower unit test threshold * * add back dependency on unit-tests * * add license headers * * fix header order * * review comments * * fix intellij inspection errors * * revert code coverage change
This commit is contained in:
parent
edfd46ed45
commit
48cde236c4
|
@ -195,6 +195,16 @@ The EXPLAIN PLAN statement returns the following result with plan, resources, an
|
|||
"name": "a0",
|
||||
"type": "LONG"
|
||||
}
|
||||
],
|
||||
"columnMappings": [
|
||||
{
|
||||
"queryColumn": "d0",
|
||||
"outputColumn": "channel"
|
||||
},
|
||||
{
|
||||
"queryColumn": "a0",
|
||||
"outputColumn": "EXPR$1"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
|
|
|
@ -86,8 +86,6 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.counters.CounterSnapshots;
|
||||
import org.apache.druid.msq.counters.CounterSnapshotsTree;
|
||||
import org.apache.druid.msq.indexing.ColumnMapping;
|
||||
import org.apache.druid.msq.indexing.ColumnMappings;
|
||||
import org.apache.druid.msq.indexing.DataSourceMSQDestination;
|
||||
import org.apache.druid.msq.indexing.InputChannelFactory;
|
||||
import org.apache.druid.msq.indexing.InputChannelsImpl;
|
||||
|
@ -185,6 +183,8 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec;
|
|||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||
import org.apache.druid.sql.calcite.run.SqlResults;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Map;
|
||||
|
|
|
@ -54,6 +54,8 @@ import org.apache.druid.segment.column.RowSignature;
|
|||
import org.apache.druid.segment.realtime.appenderator.Appenderator;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
|
|||
import org.apache.druid.segment.realtime.appenderator.Appenderators;
|
||||
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.Period;
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.segment.column.RowSignature;
|
|||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -72,7 +73,7 @@ public class QueryKitUtils
|
|||
/**
|
||||
* Enables QueryKit-generated processors to understand which output column will be mapped to
|
||||
* {@link org.apache.druid.segment.column.ColumnHolder#TIME_COLUMN_NAME}. Necessary because {@link QueryKit}
|
||||
* does not get direct access to {@link org.apache.druid.msq.indexing.ColumnMappings}.
|
||||
* does not get direct access to {@link ColumnMappings}.
|
||||
*/
|
||||
public static final String CTX_TIME_COLUMN_NAME = "__timeColumn";
|
||||
|
||||
|
|
|
@ -34,8 +34,6 @@ import org.apache.druid.java.util.common.granularity.Granularities;
|
|||
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.msq.exec.MSQTasks;
|
||||
import org.apache.druid.msq.indexing.ColumnMapping;
|
||||
import org.apache.druid.msq.indexing.ColumnMappings;
|
||||
import org.apache.druid.msq.indexing.DataSourceMSQDestination;
|
||||
import org.apache.druid.msq.indexing.MSQControllerTask;
|
||||
import org.apache.druid.msq.indexing.MSQDestination;
|
||||
|
@ -54,7 +52,10 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.server.QueryResponse;
|
||||
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
|
||||
import org.apache.druid.sql.calcite.parser.DruidSqlReplace;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.QueryUtils;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||
import org.apache.druid.sql.calcite.rel.Grouping;
|
||||
import org.apache.druid.sql.calcite.run.QueryMaker;
|
||||
|
@ -173,11 +174,10 @@ public class MSQTaskQueryMaker implements QueryMaker
|
|||
finalizeAggregations ? null /* Not needed */ : buildAggregationIntermediateTypeMap(druidQuery);
|
||||
|
||||
final List<SqlTypeName> sqlTypeNames = new ArrayList<>();
|
||||
final List<ColumnMapping> columnMappings = new ArrayList<>();
|
||||
final List<ColumnMapping> columnMappings = QueryUtils.buildColumnMappings(fieldMapping, druidQuery);
|
||||
|
||||
for (final Pair<Integer, String> entry : fieldMapping) {
|
||||
final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey());
|
||||
final String outputColumns = entry.getValue();
|
||||
|
||||
final SqlTypeName sqlTypeName;
|
||||
|
||||
|
@ -189,7 +189,6 @@ public class MSQTaskQueryMaker implements QueryMaker
|
|||
}
|
||||
|
||||
sqlTypeNames.add(sqlTypeName);
|
||||
columnMappings.add(new ColumnMapping(queryColumn, outputColumns));
|
||||
}
|
||||
|
||||
final MSQDestination destination;
|
||||
|
|
|
@ -30,8 +30,6 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.msq.indexing.ColumnMapping;
|
||||
import org.apache.druid.msq.indexing.ColumnMappings;
|
||||
import org.apache.druid.msq.indexing.MSQSpec;
|
||||
import org.apache.druid.msq.indexing.MSQTuningConfig;
|
||||
import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault;
|
||||
|
@ -67,6 +65,8 @@ import org.apache.druid.sql.SqlPlanningException;
|
|||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
import org.apache.druid.sql.calcite.external.ExternalDataSource;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.sql.calcite.planner.JoinAlgorithm;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.planner.UnsupportedSQLQueryException;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.msq.indexing;
|
||||
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.junit.Test;
|
||||
|
||||
public class ColumnMappingTest
|
||||
|
|
|
@ -25,6 +25,8 @@ import org.apache.druid.java.util.common.granularity.Granularities;
|
|||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.msq.indexing.ColumnMapping;
|
||||
import org.apache.druid.msq.indexing.ColumnMappings;
|
||||
import org.apache.druid.msq.indexing.MSQSpec;
|
||||
import org.apache.druid.msq.indexing.MSQTuningConfig;
|
||||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
|
@ -42,6 +40,8 @@ import org.apache.druid.segment.column.RowSignature;
|
|||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||
import org.apache.druid.sql.calcite.external.ExternalDataSource;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMapping;
|
||||
import org.apache.druid.sql.calcite.planner.ColumnMappings;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing;
|
||||
package org.apache.druid.sql.calcite.planner;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.indexing;
|
||||
package org.apache.druid.sql.calcite.planner;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
|
@ -37,10 +37,10 @@ import java.util.Set;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Maps column names from {@link MSQSpec#getQuery()} to output names desired by the user, in the order
|
||||
* Maps column names from the query to output names desired by the user, in the order
|
||||
* desired by the user.
|
||||
*
|
||||
* The {@link MSQSpec#getQuery()} is translated by {@link org.apache.druid.msq.querykit.QueryKit} into
|
||||
* The query is translated by {@link org.apache.druid.msq.querykit.QueryKit} into
|
||||
* a {@link org.apache.druid.msq.kernel.QueryDefinition}. So, this class also represents mappings from
|
||||
* {@link org.apache.druid.msq.kernel.QueryDefinition#getFinalStageDefinition()} into the output names desired
|
||||
* by the user.
|
|
@ -305,7 +305,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
handlerContext.hook().captureBindableRel(bindableRel);
|
||||
PlannerContext plannerContext = handlerContext.plannerContext();
|
||||
if (explain != null) {
|
||||
return planExplanation(bindableRel, false);
|
||||
return planExplanation(rootQueryRel, bindableRel, false);
|
||||
} else {
|
||||
final BindableRel theRel = bindableRel;
|
||||
final DataContext dataContext = plannerContext.createDataContext(
|
||||
|
@ -352,9 +352,10 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
}
|
||||
|
||||
/**
|
||||
* Construct a {@link PlannerResult} for an 'explain' query from a {@link RelNode}
|
||||
* Construct a {@link PlannerResult} for an 'explain' query from a {@link RelNode} and root {@link RelRoot}
|
||||
*/
|
||||
protected PlannerResult planExplanation(
|
||||
final RelRoot relRoot,
|
||||
final RelNode rel,
|
||||
final boolean isDruidConventionExplanation
|
||||
)
|
||||
|
@ -368,7 +369,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
if (plannerContext.getPlannerConfig().isUseNativeQueryExplain()) {
|
||||
DruidRel<?> druidRel = (DruidRel<?>) rel;
|
||||
try {
|
||||
explanation = explainSqlPlanAsNativeQueries(druidRel);
|
||||
explanation = explainSqlPlanAsNativeQueries(relRoot, druidRel);
|
||||
}
|
||||
catch (Exception ex) {
|
||||
log.warn(ex, "Unable to translate to a native Druid query. Resorting to legacy Druid explain plan.");
|
||||
|
@ -407,11 +408,12 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
* and not indicative of the native Druid Queries which will get executed.
|
||||
* This method assumes that the Planner has converted the RelNodes to DruidRels, and thereby we can implicitly cast it
|
||||
*
|
||||
* @param relRoot The rel root.
|
||||
* @param rel Instance of the root {@link DruidRel} which is formed by running the planner transformations on it
|
||||
* @return A string representing an array of native queries that correspond to the given SQL query, in JSON format
|
||||
* @throws JsonProcessingException
|
||||
*/
|
||||
private String explainSqlPlanAsNativeQueries(DruidRel<?> rel) throws JsonProcessingException
|
||||
private String explainSqlPlanAsNativeQueries(final RelRoot relRoot, DruidRel<?> rel) throws JsonProcessingException
|
||||
{
|
||||
ObjectMapper jsonMapper = handlerContext.jsonMapper();
|
||||
List<DruidQuery> druidQueryList;
|
||||
|
@ -431,6 +433,9 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
ObjectNode objectNode = jsonMapper.createObjectNode();
|
||||
objectNode.put("query", jsonMapper.convertValue(nativeQuery, ObjectNode.class));
|
||||
objectNode.put("signature", jsonMapper.convertValue(druidQuery.getOutputRowSignature(), ArrayNode.class));
|
||||
objectNode.put(
|
||||
"columnMappings",
|
||||
jsonMapper.convertValue(QueryUtils.buildColumnMappings(relRoot.fields, druidQuery), ArrayNode.class));
|
||||
nativeQueriesArrayNode.add(objectNode);
|
||||
}
|
||||
|
||||
|
@ -517,7 +522,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand
|
|||
handlerContext.hook().captureDruidRel(druidRel);
|
||||
|
||||
if (explain != null) {
|
||||
return planExplanation(druidRel, true);
|
||||
return planExplanation(possiblyLimitedRoot, druidRel, true);
|
||||
} else {
|
||||
// Compute row type.
|
||||
final RelDataType rowType = prepareResult.getReturnedRowType();
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import org.apache.calcite.util.Pair;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQuery;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Utility class for queries
|
||||
*/
|
||||
public class QueryUtils
|
||||
{
|
||||
|
||||
private QueryUtils()
|
||||
{
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds the mappings for queryColumn to outputColumn
|
||||
* @param fieldMapping The field mappings
|
||||
* @param druidQuery The Druid query
|
||||
* @return Mappings for queryColumn to outputColumn
|
||||
*/
|
||||
public static List<ColumnMapping> buildColumnMappings(
|
||||
final List<Pair<Integer, String>> fieldMapping,
|
||||
final DruidQuery druidQuery
|
||||
)
|
||||
{
|
||||
final List<ColumnMapping> columnMappings = new ArrayList<>();
|
||||
for (final Pair<Integer, String> entry : fieldMapping) {
|
||||
final String queryColumn = druidQuery.getOutputRowSignature().getColumnName(entry.getKey());
|
||||
final String outputColumn = entry.getValue();
|
||||
columnMappings.add(new ColumnMapping(queryColumn, outputColumn));
|
||||
}
|
||||
|
||||
return columnMappings;
|
||||
}
|
||||
}
|
|
@ -465,7 +465,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
ImmutableMap.of(
|
||||
"PLAN",
|
||||
StringUtils.format(
|
||||
"[{\"query\":{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"sqlQueryId\":\"%s\",\"sqlStringifyArrays\":false,\"sqlTimeZone\":\"America/Los_Angeles\"}},\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]}]",
|
||||
"[{\"query\":{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"sqlQueryId\":\"%s\",\"sqlStringifyArrays\":false,\"sqlTimeZone\":\"America/Los_Angeles\"}},\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"a0\",\"outputColumn\":\"cnt\"}]}]",
|
||||
DUMMY_SQL_QUERY_ID
|
||||
),
|
||||
"RESOURCES",
|
||||
|
|
|
@ -48,7 +48,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"granularity\":{\"type\":\"all\"},"
|
||||
+ "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}},"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"a0\",\"outputColumn\":\"EXPR$0\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"aview\",\"type\":\"VIEW\"}]";
|
||||
final String attributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
@ -124,7 +125,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],"
|
||||
+ "\"limitSpec\":{\"type\":\"NoopLimitSpec\"},"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}},"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]"
|
||||
+ "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"a0\",\"outputColumn\":\"EXPR$0\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
final String attributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
@ -168,7 +170,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]"
|
||||
+ "}]";
|
||||
|
||||
String explanationWithContext = "[{"
|
||||
|
@ -180,7 +183,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]"
|
||||
+ "}]";
|
||||
String sql = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
||||
String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
|
@ -234,15 +238,18 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
final String explanation = "["
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
|
||||
+ "},"
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
|
||||
+ "},"
|
||||
+ "{"
|
||||
+ "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]"
|
||||
+ "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]"
|
||||
+ "}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
final String attributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
@ -295,7 +302,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]"
|
||||
+ "}]";
|
||||
final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
final String expectedAttributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
@ -320,7 +328,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]"
|
||||
+ "}]";
|
||||
final Map<String, Object> mvFilteredContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
|
||||
mvFilteredContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true);
|
||||
|
@ -358,7 +367,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}]"
|
||||
+ "\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"}]"
|
||||
+ "}]";
|
||||
final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
final String expectedAttributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
|
|
@ -883,7 +883,8 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
|
|||
+ "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},"
|
||||
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false,"
|
||||
+ "\"context\":{\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]"
|
||||
+ "}]";
|
||||
|
||||
final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]";
|
||||
|
|
|
@ -627,7 +627,8 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
|
|||
+ "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"legacy\":false,"
|
||||
+ "\"context\":{\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\","
|
||||
+ "\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}]";
|
||||
+ "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]";
|
||||
|
||||
final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]";
|
||||
final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\"}";
|
||||
|
|
|
@ -539,7 +539,8 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}]"
|
||||
+ "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]"
|
||||
+ "}]";
|
||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n";
|
||||
final String resources = "[]";
|
||||
|
@ -1286,7 +1287,8 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
|||
+ "\"legacy\":false,"
|
||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]}]";
|
||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}],"
|
||||
+ "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]";
|
||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||
final String attributes = "{\"statementType\":\"SELECT\",\"targetDataSource\":null}";
|
||||
|
||||
|
|
|
@ -315,7 +315,8 @@ public class IngestTableFunctionTest extends CalciteIngestionDmlTest
|
|||
"\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," +
|
||||
"\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," +
|
||||
"\"granularity\":{\"type\":\"all\"}}," +
|
||||
"\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}]";
|
||||
"\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," +
|
||||
"\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]";
|
||||
final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]";
|
||||
final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\"}";
|
||||
|
||||
|
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.junit.Test;
|
||||
|
||||
public class ColumnMappingTest
|
||||
{
|
||||
@Test
|
||||
public void testEquals()
|
||||
{
|
||||
EqualsVerifier.simple().forClass(ColumnMapping.class)
|
||||
.usingGetClass()
|
||||
.verify();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* 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.sql.calcite.planner;
|
||||
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.junit.Test;
|
||||
|
||||
public class ColumnMappingsTest
|
||||
{
|
||||
@Test
|
||||
public void testEquals()
|
||||
{
|
||||
EqualsVerifier.simple().forClass(ColumnMappings.class)
|
||||
.usingGetClass()
|
||||
.withIgnoredFields("outputColumnNameToPositionMap", "queryColumnNameToPositionMap")
|
||||
.verify();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue