Skip to content

Commit

Permalink
input type validation for datasketches hll "build" aggregator factory (
Browse files Browse the repository at this point in the history
…#12131)

* Ingestion will fail for HLLSketchBuild instead of creating with incorrect values

* Addressing review comments for HLL< updated error message introduced test case
  • Loading branch information
somu-imply committed Jan 11, 2022
1 parent eb0bae4 commit 08fea7a
Show file tree
Hide file tree
Showing 2 changed files with 51 additions and 0 deletions.
Expand Up @@ -23,14 +23,17 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.datasketches.hll.HllSketch;
import org.apache.datasketches.hll.TgtHllType;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;

import javax.annotation.Nullable;
Expand All @@ -55,6 +58,7 @@ public HllSketchBuildAggregatorFactory(
super(name, fieldName, lgK, tgtHllType, round);
}


@Override
public ColumnType getIntermediateType()
{
Expand All @@ -71,13 +75,15 @@ protected byte getCacheTypeId()
public Aggregator factorize(final ColumnSelectorFactory columnSelectorFactory)
{
final ColumnValueSelector<Object> selector = columnSelectorFactory.makeColumnValueSelector(getFieldName());
validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName()));
return new HllSketchBuildAggregator(selector, getLgK(), TgtHllType.valueOf(getTgtHllType()));
}

@Override
public BufferAggregator factorizeBuffered(final ColumnSelectorFactory columnSelectorFactory)
{
final ColumnValueSelector<Object> selector = columnSelectorFactory.makeColumnValueSelector(getFieldName());
validateInputs(columnSelectorFactory.getColumnCapabilities(getFieldName()));
return new HllSketchBuildBufferAggregator(
selector,
getLgK(),
Expand All @@ -95,6 +101,7 @@ public boolean canVectorize(ColumnInspector columnInspector)
@Override
public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
{
validateInputs(selectorFactory.getColumnCapabilities(getFieldName()));
return new HllSketchBuildVectorAggregator(
selectorFactory,
getFieldName(),
Expand All @@ -114,4 +121,19 @@ public int getMaxIntermediateSize()
return HllSketch.getMaxUpdatableSerializationBytes(getLgK(), TgtHllType.valueOf(getTgtHllType()));
}

private void validateInputs(@Nullable ColumnCapabilities capabilities)
{
if (capabilities != null) {
if (capabilities.is(ValueType.COMPLEX)) {
throw new ISE(
"Invalid input [%s] of type [%s] for [%s] aggregator [%s]",
getFieldName(),
capabilities.asTypeString(),
HllSketchModule.BUILD_TYPE_NAME,
getName()
);
}
}
}

}
Expand Up @@ -143,6 +143,35 @@ public void buildSketchesAtQueryTime() throws Exception
Assert.assertEquals(200, (double) row.get(0), 0.1);
}

@Test
public void unsuccessfulComplexTypesInHLL() throws Exception
{
String metricSpec = "[{"
+ "\"type\": \"hyperUnique\","
+ "\"name\": \"index_hll\","
+ "\"fieldName\": \"id\""
+ "}]";
try {
Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
new File(this.getClass().getClassLoader().getResource("hll/hll_sketches.tsv").getFile()),
buildParserJson(
Arrays.asList("dim", "multiDim", "id"),
Arrays.asList("timestamp", "dim", "multiDim", "id")
),
metricSpec,
0, // minTimestamp
Granularities.NONE,
200, // maxRowCount
buildGroupByQueryJson("HLLSketchBuild", "index_hll", !ROUND)
);
}
catch (RuntimeException e) {
Assert.assertTrue(
e.getMessage().contains("Invalid input [index_hll] of type [COMPLEX<hyperUnique>] for [HLLSketchBuild]"));
}

}

@Test
public void buildSketchesAtQueryTimeMultiValue() throws Exception
{
Expand Down

0 comments on commit 08fea7a

Please sign in to comment.