mirror of https://github.com/apache/druid.git
DataSchema: Improve duplicate-column error message. (#11082)
* DataSchema: Improve duplicate-column error message. Now, when duplicate columns are specified, the error message will include information about where those duplicate columns were seen. Also, if there are multiple duplicate columns, all will be listed in the error message instead of just the first one encountered. * Fix style for checkstyle. * Further improve error message.
This commit is contained in:
parent
c86178aaeb
commit
c3faa24f26
|
@ -27,13 +27,16 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Multiset;
|
||||
import com.google.common.collect.TreeMultiset;
|
||||
import org.apache.druid.common.utils.IdUtils;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
import org.apache.druid.data.input.impl.ParseSpec;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
@ -42,11 +45,14 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
|
|||
import org.apache.druid.segment.transform.TransformSpec;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -104,16 +110,16 @@ public class DataSchema
|
|||
this.parserMap = parserMap;
|
||||
this.objectMapper = objectMapper;
|
||||
|
||||
if (aggregators != null && aggregators.length != 0) {
|
||||
// validate for no duplication
|
||||
Set<String> names = new HashSet<>();
|
||||
for (AggregatorFactory factory : aggregators) {
|
||||
if (!names.add(factory.getName())) {
|
||||
throw new IAE("duplicate aggregators found with name [%s].", factory.getName());
|
||||
}
|
||||
}
|
||||
} else if (this.granularitySpec.isRollup()) {
|
||||
log.warn("No metricsSpec has been specified. Are you sure this is what you want?");
|
||||
// Fail-fast if there are output name collisions. Note: because of the pull-from-parser magic in getDimensionsSpec,
|
||||
// this validation is not necessarily going to be able to catch everything. It will run again in getDimensionsSpec.
|
||||
computeAndValidateOutputFieldNames(this.dimensionsSpec, this.aggregators);
|
||||
|
||||
if (this.granularitySpec.isRollup()) {
|
||||
log.warn(
|
||||
"Rollup is enabled for dataSource [%s] but no metricsSpec has been provided. "
|
||||
+ "Are you sure this is what you want?",
|
||||
dataSource
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -159,29 +165,8 @@ public class DataSchema
|
|||
final AggregatorFactory[] aggregators
|
||||
)
|
||||
{
|
||||
final Set<String> inputFieldNames = new HashSet<>();
|
||||
final Set<String> outputFieldNames = new HashSet<>();
|
||||
|
||||
// Populate inputFieldNames.
|
||||
inputFieldNames.add(timestampSpec.getTimestampColumn());
|
||||
inputFieldNames.addAll(dimensionsSpec.getDimensionNames());
|
||||
Arrays.stream(aggregators)
|
||||
.flatMap(aggregator -> aggregator.requiredFields().stream())
|
||||
.forEach(inputFieldNames::add);
|
||||
|
||||
// Populate outputFieldNames, validating along the way for lack of duplicates.
|
||||
outputFieldNames.add(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
||||
Stream.concat(
|
||||
dimensionsSpec.getDimensions().stream().map(DimensionSchema::getName),
|
||||
Arrays.stream(aggregators).map(AggregatorFactory::getName)
|
||||
).forEach(
|
||||
field -> {
|
||||
if (!outputFieldNames.add(field)) {
|
||||
throw new IAE("Cannot specify field [%s] more than once", field);
|
||||
}
|
||||
}
|
||||
);
|
||||
final Set<String> inputFieldNames = computeInputFieldNames(timestampSpec, dimensionsSpec, aggregators);
|
||||
final Set<String> outputFieldNames = computeAndValidateOutputFieldNames(dimensionsSpec, aggregators);
|
||||
|
||||
// Set up additional exclusions: all inputs and outputs, minus defined dimensions.
|
||||
final Set<String> additionalDimensionExclusions = new HashSet<>();
|
||||
|
@ -192,6 +177,99 @@ public class DataSchema
|
|||
return dimensionsSpec.withDimensionExclusions(additionalDimensionExclusions);
|
||||
}
|
||||
|
||||
private static Set<String> computeInputFieldNames(
|
||||
final TimestampSpec timestampSpec,
|
||||
final DimensionsSpec dimensionsSpec,
|
||||
final AggregatorFactory[] aggregators
|
||||
)
|
||||
{
|
||||
final Set<String> fields = new HashSet<>();
|
||||
|
||||
fields.add(timestampSpec.getTimestampColumn());
|
||||
fields.addAll(dimensionsSpec.getDimensionNames());
|
||||
Arrays.stream(aggregators)
|
||||
.flatMap(aggregator -> aggregator.requiredFields().stream())
|
||||
.forEach(fields::add);
|
||||
|
||||
return fields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes the set of field names that are specified by the provided dimensions and aggregator lists.
|
||||
*
|
||||
* If either list is null, it is ignored.
|
||||
*
|
||||
* @throws IllegalArgumentException if there are duplicate field names, or if any dimension or aggregator
|
||||
* has a null name
|
||||
*/
|
||||
private static Set<String> computeAndValidateOutputFieldNames(
|
||||
@Nullable final DimensionsSpec dimensionsSpec,
|
||||
@Nullable final AggregatorFactory[] aggregators
|
||||
)
|
||||
{
|
||||
// Field name -> where it was seen
|
||||
final Map<String, Multiset<String>> fields = new TreeMap<>();
|
||||
|
||||
fields.computeIfAbsent(ColumnHolder.TIME_COLUMN_NAME, k -> TreeMultiset.create()).add(
|
||||
StringUtils.format(
|
||||
"primary timestamp (%s cannot appear as a dimension or metric)",
|
||||
ColumnHolder.TIME_COLUMN_NAME
|
||||
)
|
||||
);
|
||||
|
||||
if (dimensionsSpec != null) {
|
||||
for (int i = 0; i < dimensionsSpec.getDimensions().size(); i++) {
|
||||
final String field = dimensionsSpec.getDimensions().get(i).getName();
|
||||
if (Strings.isNullOrEmpty(field)) {
|
||||
throw new IAE("Encountered dimension with null or empty name at position %d", i);
|
||||
}
|
||||
|
||||
fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("dimensions list");
|
||||
}
|
||||
}
|
||||
|
||||
if (aggregators != null) {
|
||||
for (int i = 0; i < aggregators.length; i++) {
|
||||
final String field = aggregators[i].getName();
|
||||
if (Strings.isNullOrEmpty(field)) {
|
||||
throw new IAE("Encountered metric with null or empty name at position %d", i);
|
||||
}
|
||||
|
||||
fields.computeIfAbsent(field, k -> TreeMultiset.create()).add("metricsSpec list");
|
||||
}
|
||||
}
|
||||
|
||||
final List<String> errors = new ArrayList<>();
|
||||
|
||||
for (Map.Entry<String, Multiset<String>> fieldEntry : fields.entrySet()) {
|
||||
if (fieldEntry.getValue().entrySet().stream().mapToInt(Multiset.Entry::getCount).sum() > 1) {
|
||||
errors.add(
|
||||
StringUtils.format(
|
||||
"[%s] seen in %s",
|
||||
fieldEntry.getKey(),
|
||||
fieldEntry.getValue().entrySet().stream().map(
|
||||
entry ->
|
||||
StringUtils.format(
|
||||
"%s%s",
|
||||
entry.getElement(),
|
||||
entry.getCount() == 1 ? "" : StringUtils.format(
|
||||
" (%d occurrences)",
|
||||
entry.getCount()
|
||||
)
|
||||
)
|
||||
).collect(Collectors.joining(", "))
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
if (errors.isEmpty()) {
|
||||
return fields.keySet();
|
||||
} else {
|
||||
throw new IAE("Cannot specify a column more than once: %s", String.join("; ", errors));
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.druid.data.input.impl.JSONParseSpec;
|
|||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.DurationGranularity;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -202,7 +201,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
Assert.assertNull(row2);
|
||||
}
|
||||
|
||||
@Test(expected = IAE.class)
|
||||
@Test
|
||||
public void testOverlapMetricNameAndDim()
|
||||
{
|
||||
Map<String, Object> parser = jsonMapper.convertValue(
|
||||
|
@ -234,10 +233,80 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
null,
|
||||
jsonMapper
|
||||
);
|
||||
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage(
|
||||
"Cannot specify a column more than once: [metric1] seen in dimensions list, metricsSpec list"
|
||||
);
|
||||
|
||||
schema.getParser();
|
||||
}
|
||||
|
||||
@Test(expected = IAE.class)
|
||||
@Test
|
||||
public void testOverlapTimeAndDim()
|
||||
{
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage(
|
||||
"Cannot specify a column more than once: [__time] seen in dimensions list, "
|
||||
+ "primary timestamp (__time cannot appear as a dimension or metric)"
|
||||
);
|
||||
|
||||
DataSchema schema = new DataSchema(
|
||||
IdUtilsTest.VALID_ID_CHARS,
|
||||
new TimestampSpec("time", "auto", null),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(ImmutableList.of("__time", "dimA", "dimB", "metric1")),
|
||||
ImmutableList.of("dimC"),
|
||||
null
|
||||
),
|
||||
null,
|
||||
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
|
||||
null,
|
||||
null,
|
||||
jsonMapper
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverlapTimeAndDimLegacy()
|
||||
{
|
||||
Map<String, Object> parser = jsonMapper.convertValue(
|
||||
new StringInputRowParser(
|
||||
new JSONParseSpec(
|
||||
new TimestampSpec("time", "auto", null),
|
||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of(
|
||||
"__time",
|
||||
"dimA",
|
||||
"dimB",
|
||||
"metric1"
|
||||
)), ImmutableList.of("dimC"), null),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
null
|
||||
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
||||
);
|
||||
|
||||
DataSchema schema = new DataSchema(
|
||||
IdUtilsTest.VALID_ID_CHARS,
|
||||
parser,
|
||||
null,
|
||||
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
|
||||
null,
|
||||
jsonMapper
|
||||
);
|
||||
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage(
|
||||
"Cannot specify a column more than once: [__time] seen in dimensions list, primary timestamp "
|
||||
+ "(__time cannot appear as a dimension or metric)"
|
||||
);
|
||||
|
||||
schema.getParser();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDuplicateAggregators()
|
||||
{
|
||||
Map<String, Object> parser = jsonMapper.convertValue(
|
||||
|
@ -257,6 +326,12 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
|
||||
);
|
||||
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage(
|
||||
"Cannot specify a column more than once: [metric1] seen in metricsSpec list (2 occurrences); "
|
||||
+ "[metric3] seen in metricsSpec list (2 occurrences)"
|
||||
);
|
||||
|
||||
DataSchema schema = new DataSchema(
|
||||
IdUtilsTest.VALID_ID_CHARS,
|
||||
parser,
|
||||
|
@ -264,12 +339,13 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
new DoubleSumAggregatorFactory("metric1", "col1"),
|
||||
new DoubleSumAggregatorFactory("metric2", "col2"),
|
||||
new DoubleSumAggregatorFactory("metric1", "col3"),
|
||||
new DoubleSumAggregatorFactory("metric3", "col4"),
|
||||
new DoubleSumAggregatorFactory("metric3", "col5"),
|
||||
},
|
||||
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
|
||||
null,
|
||||
jsonMapper
|
||||
);
|
||||
schema.getParser();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -559,6 +635,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
|
|||
DimensionsSpec oldDimSpec = Mockito.mock(DimensionsSpec.class);
|
||||
DimensionsSpec newDimSpec = Mockito.mock(DimensionsSpec.class);
|
||||
AggregatorFactory aggFactory = Mockito.mock(AggregatorFactory.class);
|
||||
Mockito.when(aggFactory.getName()).thenReturn("myAgg");
|
||||
TransformSpec transSpec = Mockito.mock(TransformSpec.class);
|
||||
Map<String, Object> parserMap = Mockito.mock(Map.class);
|
||||
Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec);
|
||||
|
|
Loading…
Reference in New Issue