mirror of https://github.com/apache/druid.git
fix off by one error in FrontCodedIndexedWriter and FrontCodedIntArrayIndexedWriter getCardinality method (#14047)
* fix off by one error in FrontCodedIndexedWriter and FrontCodedIntArrayIndexedWriter getCardinality method
This commit is contained in:
parent
f47b05a98c
commit
f41468fd46
|
@ -227,7 +227,7 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
|
||||||
@Override
|
@Override
|
||||||
public int getCardinality()
|
public int getCardinality()
|
||||||
{
|
{
|
||||||
return numWritten;
|
return numWritten + (hasNulls ? 1 : 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
private long getBucketOffset(int index) throws IOException
|
private long getBucketOffset(int index) throws IOException
|
||||||
|
|
|
@ -227,7 +227,7 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
|
||||||
@Override
|
@Override
|
||||||
public int getCardinality()
|
public int getCardinality()
|
||||||
{
|
{
|
||||||
return numWritten;
|
return numWritten + (hasNulls ? 1 : 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
private long getBucketOffset(int index) throws IOException
|
private long getBucketOffset(int index) throws IOException
|
||||||
|
|
|
@ -43,9 +43,11 @@ import org.apache.druid.query.expression.TestExprMacroTable;
|
||||||
import org.apache.druid.segment.AutoTypeColumnSchema;
|
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||||
import org.apache.druid.segment.IncrementalIndexSegment;
|
import org.apache.druid.segment.IncrementalIndexSegment;
|
||||||
import org.apache.druid.segment.IndexBuilder;
|
import org.apache.druid.segment.IndexBuilder;
|
||||||
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.QueryableIndexSegment;
|
import org.apache.druid.segment.QueryableIndexSegment;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.TestHelper;
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.segment.column.StringEncodingStrategy;
|
||||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import org.apache.druid.segment.transform.ExpressionTransform;
|
import org.apache.druid.segment.transform.ExpressionTransform;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
import org.apache.druid.segment.transform.TransformSpec;
|
||||||
|
@ -180,7 +182,8 @@ public class NestedDataTestUtils
|
||||||
SIMPLE_DATA_TSV_TRANSFORM,
|
SIMPLE_DATA_TSV_TRANSFORM,
|
||||||
COUNT,
|
COUNT,
|
||||||
granularity,
|
granularity,
|
||||||
rollup
|
rollup,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -205,7 +208,8 @@ public class NestedDataTestUtils
|
||||||
closer,
|
closer,
|
||||||
SIMPLE_DATA_FILE,
|
SIMPLE_DATA_FILE,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -246,7 +250,8 @@ public class NestedDataTestUtils
|
||||||
Closer closer,
|
Closer closer,
|
||||||
String inputFile,
|
String inputFile,
|
||||||
Granularity granularity,
|
Granularity granularity,
|
||||||
boolean rollup
|
boolean rollup,
|
||||||
|
IndexSpec indexSpec
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
return createSegments(
|
return createSegments(
|
||||||
|
@ -259,7 +264,8 @@ public class NestedDataTestUtils
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
COUNT,
|
COUNT,
|
||||||
granularity,
|
granularity,
|
||||||
rollup
|
rollup,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -288,14 +294,16 @@ public class NestedDataTestUtils
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
COUNT,
|
COUNT,
|
||||||
granularity,
|
granularity,
|
||||||
rollup
|
rollup,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public static List<Segment> createSegmentsForJsonInput(
|
public static List<Segment> createSegmentsForJsonInput(
|
||||||
TemporaryFolder tempFolder,
|
TemporaryFolder tempFolder,
|
||||||
Closer closer,
|
Closer closer,
|
||||||
String inputFile
|
String inputFile,
|
||||||
|
IndexSpec indexSpec
|
||||||
)
|
)
|
||||||
throws Exception
|
throws Exception
|
||||||
{
|
{
|
||||||
|
@ -304,7 +312,8 @@ public class NestedDataTestUtils
|
||||||
closer,
|
closer,
|
||||||
inputFile,
|
inputFile,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
true
|
true,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -355,7 +364,8 @@ public class NestedDataTestUtils
|
||||||
TransformSpec transformSpec,
|
TransformSpec transformSpec,
|
||||||
AggregatorFactory[] aggregators,
|
AggregatorFactory[] aggregators,
|
||||||
Granularity queryGranularity,
|
Granularity queryGranularity,
|
||||||
boolean rollup
|
boolean rollup,
|
||||||
|
IndexSpec indexSpec
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
return createSegments(
|
return createSegments(
|
||||||
|
@ -368,7 +378,8 @@ public class NestedDataTestUtils
|
||||||
transformSpec,
|
transformSpec,
|
||||||
aggregators,
|
aggregators,
|
||||||
queryGranularity,
|
queryGranularity,
|
||||||
rollup
|
rollup,
|
||||||
|
indexSpec
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -382,7 +393,8 @@ public class NestedDataTestUtils
|
||||||
TransformSpec transformSpec,
|
TransformSpec transformSpec,
|
||||||
AggregatorFactory[] aggregators,
|
AggregatorFactory[] aggregators,
|
||||||
Granularity queryGranularity,
|
Granularity queryGranularity,
|
||||||
boolean rollup
|
boolean rollup,
|
||||||
|
IndexSpec indexSpec
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
final List<Segment> segments = Lists.newArrayListWithCapacity(inputs.size());
|
final List<Segment> segments = Lists.newArrayListWithCapacity(inputs.size());
|
||||||
|
@ -400,6 +412,7 @@ public class NestedDataTestUtils
|
||||||
.withMinTimestamp(0)
|
.withMinTimestamp(0)
|
||||||
.build()
|
.build()
|
||||||
)
|
)
|
||||||
|
.indexSpec(indexSpec)
|
||||||
.inputSource(inputSource)
|
.inputSource(inputSource)
|
||||||
.inputFormat(inputFormat)
|
.inputFormat(inputFormat)
|
||||||
.transform(transformSpec)
|
.transform(transformSpec)
|
||||||
|
@ -464,7 +477,8 @@ public class NestedDataTestUtils
|
||||||
NestedDataTestUtils.createSegmentsForJsonInput(
|
NestedDataTestUtils.createSegmentsForJsonInput(
|
||||||
tempFolder,
|
tempFolder,
|
||||||
closer,
|
closer,
|
||||||
jsonInputFile
|
jsonInputFile,
|
||||||
|
new IndexSpec()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.add(NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile))
|
.add(NestedDataTestUtils.createIncrementalIndexForJsonInput(tempFolder, jsonInputFile))
|
||||||
|
@ -514,13 +528,16 @@ public class NestedDataTestUtils
|
||||||
NestedDataTestUtils.createSegmentsForJsonInput(
|
NestedDataTestUtils.createSegmentsForJsonInput(
|
||||||
tempFolder,
|
tempFolder,
|
||||||
closer,
|
closer,
|
||||||
jsonInputFile
|
jsonInputFile,
|
||||||
|
new IndexSpec()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.addAll(NestedDataTestUtils.createSegmentsForJsonInput(
|
.addAll(
|
||||||
|
NestedDataTestUtils.createSegmentsForJsonInput(
|
||||||
tempFolder,
|
tempFolder,
|
||||||
closer,
|
closer,
|
||||||
jsonInputFile
|
jsonInputFile,
|
||||||
|
new IndexSpec()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
@ -536,6 +553,58 @@ public class NestedDataTestUtils
|
||||||
return "segments";
|
return "segments";
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
segmentsGenerators.add(new BiFunction<TemporaryFolder, Closer, List<Segment>>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public List<Segment> apply(TemporaryFolder tempFolder, Closer closer)
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
return ImmutableList.<Segment>builder()
|
||||||
|
.addAll(
|
||||||
|
NestedDataTestUtils.createSegmentsForJsonInput(
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
jsonInputFile,
|
||||||
|
new IndexSpec(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
new StringEncodingStrategy.FrontCoded(4, (byte) 0x01),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.addAll(
|
||||||
|
NestedDataTestUtils.createSegmentsForJsonInput(
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
jsonInputFile,
|
||||||
|
new IndexSpec(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
new StringEncodingStrategy.FrontCoded(4, (byte) 0x00),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "segments-frontcoded";
|
||||||
|
}
|
||||||
|
});
|
||||||
return segmentsGenerators;
|
return segmentsGenerators;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -569,7 +569,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (!"segments".equals(segmentsName)) {
|
if (!"segments".equals(segmentsName) && !"segments-frontcoded".equals(segmentsName)) {
|
||||||
if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) {
|
if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) {
|
||||||
Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
|
Throwable t = Assert.assertThrows(RuntimeException.class, runner::get);
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.query.filter.BoundDimFilter;
|
||||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||||
import org.apache.druid.query.ordering.StringComparators;
|
import org.apache.druid.query.ordering.StringComparators;
|
||||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.transform.TransformSpec;
|
import org.apache.druid.segment.transform.TransformSpec;
|
||||||
|
@ -135,7 +136,8 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
NestedDataTestUtils.COUNT,
|
NestedDataTestUtils.COUNT,
|
||||||
Granularities.YEAR,
|
Granularities.YEAR,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
)
|
)
|
||||||
).build();
|
).build();
|
||||||
|
|
||||||
|
@ -308,7 +310,8 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
closer,
|
closer,
|
||||||
NestedDataTestUtils.SIMPLE_DATA_FILE,
|
NestedDataTestUtils.SIMPLE_DATA_FILE,
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
||||||
|
|
||||||
|
@ -491,7 +494,8 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
NestedDataTestUtils.COUNT,
|
NestedDataTestUtils.COUNT,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
|
@ -550,7 +554,8 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
aggs,
|
aggs,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -444,6 +444,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest
|
||||||
}
|
}
|
||||||
index++;
|
index++;
|
||||||
}
|
}
|
||||||
|
Assert.assertEquals(index, writer.getCardinality());
|
||||||
|
|
||||||
// check 'get' again so that we aren't always reading from current page
|
// check 'get' again so that we aren't always reading from current page
|
||||||
index = 0;
|
index = 0;
|
||||||
|
|
|
@ -414,6 +414,7 @@ public class FrontCodedIntArrayIndexedTest
|
||||||
assertSame(index, next, writer.get(index));
|
assertSame(index, next, writer.get(index));
|
||||||
index++;
|
index++;
|
||||||
}
|
}
|
||||||
|
Assert.assertEquals(index, writer.getCardinality());
|
||||||
|
|
||||||
// check 'get' again so that we aren't always reading from current page
|
// check 'get' again so that we aren't always reading from current page
|
||||||
index = 0;
|
index = 0;
|
||||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
import org.apache.druid.segment.Cursor;
|
import org.apache.druid.segment.Cursor;
|
||||||
import org.apache.druid.segment.DoubleColumnSelector;
|
import org.apache.druid.segment.DoubleColumnSelector;
|
||||||
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.LongColumnSelector;
|
import org.apache.druid.segment.LongColumnSelector;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.StorageAdapter;
|
import org.apache.druid.segment.StorageAdapter;
|
||||||
|
@ -336,7 +337,8 @@ public class NestedFieldColumnSelectorsTest
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
NestedDataTestUtils.COUNT,
|
NestedDataTestUtils.COUNT,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
Assert.assertEquals(1, segments.size());
|
Assert.assertEquals(1, segments.size());
|
||||||
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
||||||
|
@ -366,7 +368,8 @@ public class NestedFieldColumnSelectorsTest
|
||||||
TransformSpec.NONE,
|
TransformSpec.NONE,
|
||||||
NestedDataTestUtils.COUNT,
|
NestedDataTestUtils.COUNT,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
true
|
true,
|
||||||
|
new IndexSpec()
|
||||||
);
|
);
|
||||||
Assert.assertEquals(1, segments.size());
|
Assert.assertEquals(1, segments.size());
|
||||||
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
||||||
|
|
Loading…
Reference in New Issue