mirror of https://github.com/apache/druid.git
remove druid.processing.columnCache.sizeBytes and CachingIndexed, combine string column implementations (#14500)
* combine string column implementations changes: * generic indexed, front-coded, and auto string columns now all share the same column and index supplier implementations * remove CachingIndexed implementation, which I think is largely no longer needed by the switch of many things to directly using ByteBuffer, avoiding the cost of creating Strings * remove ColumnConfig.columnCacheSizeBytes since CachingIndexed was the only user
This commit is contained in:
parent
58f3faf299
commit
277aaa5c57
|
@ -36,7 +36,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.BoundFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -161,10 +161,6 @@ public class BoundFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints).transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(
|
||||
|
@ -183,7 +179,7 @@ public class BoundFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.segment.column.StringValueSetIndex;
|
|||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -93,11 +93,6 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final Iterable<Integer> ints = intGenerator();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -115,8 +110,8 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
.iterator(),
|
||||
serdeFactory.getObjectStrategy()
|
||||
);
|
||||
DictionaryEncodedStringIndexSupplier indexSupplier =
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null);
|
||||
StringUtf8ColumnIndexSupplier<?> indexSupplier =
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null);
|
||||
stringValueSetIndex = (IndexedUtf8ValueSetIndex<?>) indexSupplier.as(StringValueSetIndex.class);
|
||||
List<Integer> filterValues = new ArrayList<>();
|
||||
List<Integer> nonFilterValues = new ArrayList<>();
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -122,11 +122,6 @@ public class DimensionPredicateFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -145,7 +140,7 @@ public class DimensionPredicateFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -137,11 +137,6 @@ public class FilterPartitionBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -140,11 +140,6 @@ public class FilteredAggregatorBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -158,11 +158,6 @@ public class GroupByTypeInterfaceBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -85,11 +85,6 @@ public class InFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final Iterable<Integer> ints = intGenerator();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -109,7 +104,7 @@ public class InFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
inFilter = new InDimFilter(
|
||||
"dummy",
|
||||
|
|
|
@ -95,9 +95,6 @@ public class JoinAndLookupBenchmark
|
|||
@Param({"500000"})
|
||||
int rows;
|
||||
|
||||
@Param({"0", "16384"})
|
||||
int columnCacheSizeBytes;
|
||||
|
||||
private File tmpDir = null;
|
||||
private QueryableIndex index = null;
|
||||
private Segment baseSegment = null;
|
||||
|
@ -123,8 +120,7 @@ public class JoinAndLookupBenchmark
|
|||
public void setup() throws IOException
|
||||
{
|
||||
tmpDir = FileUtils.createTempDir();
|
||||
ColumnConfig columnConfig = () -> columnCacheSizeBytes;
|
||||
index = JoinTestHelper.createFactIndexBuilder(columnConfig, tmpDir, rows).buildMMappedIndex();
|
||||
index = JoinTestHelper.createFactIndexBuilder(ColumnConfig.DEFAULT, tmpDir, rows).buildMMappedIndex();
|
||||
|
||||
final String prefix = "c.";
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -122,11 +122,6 @@ public class LikeFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -145,7 +140,7 @@ public class LikeFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -138,11 +138,6 @@ public class TopNTypeInterfaceBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.LongsColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
@ -118,10 +119,7 @@ public class ColumnarLongsEncodeDataFromSegmentBenchmark extends BaseColumnarLon
|
|||
File dataFile = new File(dir, getColumnDataFileName(segmentName, columnName));
|
||||
|
||||
if (!dataFile.exists()) {
|
||||
final IndexIO indexIO = new IndexIO(
|
||||
new DefaultObjectMapper(),
|
||||
() -> 0
|
||||
);
|
||||
final IndexIO indexIO = new IndexIO(new DefaultObjectMapper(), ColumnConfig.DEFAULT);
|
||||
try (final QueryableIndex index = indexIO.loadIndex(new File(segmentPath))) {
|
||||
final Set<String> columnNames = new LinkedHashSet<>();
|
||||
columnNames.add(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -105,10 +106,7 @@ public class IndexMergeBenchmark
|
|||
InjectableValues.Std injectableValues = new InjectableValues.Std();
|
||||
injectableValues.addValue(ExprMacroTable.class, ExprMacroTable.nil());
|
||||
JSON_MAPPER.setInjectableValues(injectableValues);
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
}
|
||||
|
||||
@Setup
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -74,10 +75,7 @@ public class IndexPersistBenchmark
|
|||
static {
|
||||
NullHandling.initializeForTests();
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
}
|
||||
|
||||
|
|
|
@ -174,11 +174,6 @@ public class GroupByBenchmark
|
|||
),
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.druid.segment.IndexMergerV9;
|
|||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -136,10 +137,7 @@ public class ScanBenchmark
|
|||
|
||||
static {
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
}
|
||||
|
||||
|
|
|
@ -143,11 +143,6 @@ public class SearchBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -136,11 +136,6 @@ public class TimeseriesBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -134,11 +134,6 @@ public class TopNBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -160,11 +160,6 @@ public class TimeCompareBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -1478,7 +1478,6 @@ Processing properties set on the MiddleManager will be passed through to Peons.
|
|||
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|
||||
|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`druid.processing.fifo`|Enables the processing queue to treat tasks of equal priority in a FIFO manner.|`true`|
|
||||
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
|
||||
|`druid.processing.intermediaryData.storage.type`|Storage type for intermediary segments of data shuffle between native parallel index tasks. <br />Set to `local` to store segment files in the local storage of the MiddleManager or Indexer. <br />Set to `deepstore` to use configured deep storage for better fault tolerance during rolling updates. When the storage type is `deepstore`, Druid stores the data in the `shuffle-data` directory under the configured deep storage path. Druid does not support automated cleanup for the `shuffle-data` directory. You can set up cloud storage lifecycle rules for automated cleanup of data at the `shuffle-data` prefix location.|`local`|
|
||||
|
@ -1628,7 +1627,6 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|`druid.processing.formatString`|Indexer processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|
||||
|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|
||||
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
|
||||
|
||||
|
@ -1738,7 +1736,6 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|
||||
|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|
||||
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
|
||||
|
||||
|
@ -1917,7 +1914,6 @@ The broker uses processing configs for nested groupBy queries.
|
|||
|`druid.processing.buffer.poolCacheInitialCount`|initializes the number of buffers allocated on the intermediate results pool. Note that pool can create more buffers if necessary.|`0`|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|`druid.processing.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|
||||
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
|
||||
|`druid.processing.merge.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`|
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.msq.kernel.WorkOrder;
|
|||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.server.security.AuthConfig;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
|
@ -68,7 +69,7 @@ public class WorkerChatHandlerTest
|
|||
public void setUp()
|
||||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
IndexIO indexIO = new IndexIO(mapper, () -> 0);
|
||||
IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
mapper,
|
||||
indexIO,
|
||||
|
|
|
@ -61,6 +61,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
||||
|
@ -137,7 +138,7 @@ public class CalciteMSQTestsHelper
|
|||
)
|
||||
);
|
||||
ObjectMapper testMapper = MSQTestBase.setupObjectMapper(dummyInjector);
|
||||
IndexIO indexIO = new IndexIO(testMapper, () -> 0);
|
||||
IndexIO indexIO = new IndexIO(testMapper, ColumnConfig.DEFAULT);
|
||||
SegmentCacheManager segmentCacheManager = null;
|
||||
try {
|
||||
segmentCacheManager = new SegmentCacheManagerFactory(testMapper).manufacturate(temporaryFolder.newFolder(
|
||||
|
|
|
@ -122,6 +122,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
|
@ -356,7 +357,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
);
|
||||
|
||||
ObjectMapper secondMapper = setupObjectMapper(secondInjector);
|
||||
indexIO = new IndexIO(secondMapper, () -> 0);
|
||||
indexIO = new IndexIO(secondMapper, ColumnConfig.DEFAULT);
|
||||
|
||||
try {
|
||||
segmentCacheManager = new SegmentCacheManagerFactory(secondMapper).manufacturate(tmpFolder.newFolder("test"));
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.druid.msq.kernel.QueryDefinition;
|
|||
import org.apache.druid.msq.querykit.DataSegmentProvider;
|
||||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.NoopRowIngestionMeters;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
|
||||
|
@ -114,10 +115,7 @@ public class MSQTestWorkerContext implements WorkerContext
|
|||
@Override
|
||||
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber)
|
||||
{
|
||||
IndexIO indexIO = new IndexIO(
|
||||
mapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
mapper,
|
||||
indexIO,
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexMergerV9Factory;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
|
||||
import org.apache.druid.segment.loading.LocalLoadSpec;
|
||||
|
@ -82,10 +83,7 @@ public class TestUtils
|
|||
public TestUtils()
|
||||
{
|
||||
this.jsonMapper = new DefaultObjectMapper();
|
||||
indexIO = new IndexIO(
|
||||
jsonMapper,
|
||||
() -> 0
|
||||
);
|
||||
indexIO = new IndexIO(jsonMapper, ColumnConfig.DEFAULT);
|
||||
indexMergerV9Factory = new IndexMergerV9Factory(
|
||||
jsonMapper,
|
||||
indexIO,
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
|
@ -179,10 +180,7 @@ public class AppenderatorsTest
|
|||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
|
@ -184,10 +185,7 @@ public class BatchAppenderatorsTest
|
|||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
|
@ -574,7 +572,7 @@ public class BatchAppenderatorsTest
|
|||
.config(config)
|
||||
.joinableFactory(NoopJoinableFactory.INSTANCE)
|
||||
.jsonMapper(mapper)
|
||||
.indexIO(new IndexIO(new ObjectMapper(), () -> 0))
|
||||
.indexIO(new IndexIO(new ObjectMapper(), ColumnConfig.DEFAULT))
|
||||
.indexMergerV9(indexMergerV9)
|
||||
.taskReportFileWriter(new NoopTestTaskReportFileWriter())
|
||||
.authorizerMapper(AuthTestUtils.TEST_AUTHORIZER_MAPPER)
|
||||
|
|
|
@ -104,6 +104,7 @@ import org.apache.druid.segment.SimpleQueryableIndex;
|
|||
import org.apache.druid.segment.column.BaseColumn;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -1983,7 +1984,7 @@ public class CompactionTaskTest
|
|||
Map<DataSegment, File> segmentFileMap
|
||||
)
|
||||
{
|
||||
super(mapper, () -> 0);
|
||||
super(mapper, ColumnConfig.DEFAULT);
|
||||
|
||||
queryableIndexMap = Maps.newHashMapWithExpectedSize(segmentFileMap.size());
|
||||
for (Entry<DataSegment, File> entry : segmentFileMap.entrySet()) {
|
||||
|
|
4
pom.xml
4
pom.xml
|
@ -236,7 +236,7 @@
|
|||
<enabled>false</enabled>
|
||||
</snapshots>
|
||||
</repository>
|
||||
|
||||
|
||||
<!-- Only used by core, but moved to root for parallel build dependency resolution -->
|
||||
<repository>
|
||||
<id>sigar</id>
|
||||
|
@ -246,7 +246,7 @@
|
|||
</snapshots>
|
||||
</repository>
|
||||
</repositories>
|
||||
|
||||
|
||||
<pluginRepositories>
|
||||
<pluginRepository>
|
||||
<id>${repoOrgId}</id>
|
||||
|
|
|
@ -142,13 +142,6 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem
|
|||
return DEFAULT_NUM_MERGE_BUFFERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Config(value = "${base_path}.columnCache.sizeBytes")
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
@Config(value = "${base_path}.indexes.skipValueRangeIndexScale")
|
||||
public double skipValueRangeIndexScale()
|
||||
|
|
|
@ -65,10 +65,10 @@ import org.apache.druid.segment.data.IndexedIterable;
|
|||
import org.apache.druid.segment.data.ListIndexed;
|
||||
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
||||
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.FloatNumericColumnSupplier;
|
||||
import org.apache.druid.segment.serde.LongNumericColumnSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -109,7 +109,7 @@ public class IndexIO
|
|||
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
|
||||
Preconditions.checkNotNull(columnConfig, "null ColumnConfig");
|
||||
ImmutableMap.Builder<Integer, IndexLoader> indexLoadersBuilder = ImmutableMap.builder();
|
||||
LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig);
|
||||
LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler());
|
||||
for (int i = 0; i <= V8_VERSION; i++) {
|
||||
indexLoadersBuilder.put(i, legacyIndexLoader);
|
||||
}
|
||||
|
@ -364,7 +364,6 @@ public class IndexIO
|
|||
metrics.put(metric, holder);
|
||||
}
|
||||
|
||||
Map<String, GenericIndexed<String>> dimValueLookups = new HashMap<>();
|
||||
Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups = new HashMap<>();
|
||||
Map<String, VSizeColumnarMultiInts> dimColumns = new HashMap<>();
|
||||
Map<String, GenericIndexed<ImmutableBitmap>> bitmaps = new HashMap<>();
|
||||
|
@ -379,8 +378,6 @@ public class IndexIO
|
|||
fileDimensionName
|
||||
);
|
||||
|
||||
// Duplicate the first buffer since we are reading the dictionary twice.
|
||||
dimValueLookups.put(dimension, GenericIndexed.read(dimBuffer.duplicate(), GenericIndexed.STRING_STRATEGY));
|
||||
dimValueUtf8Lookups.put(dimension, GenericIndexed.read(dimBuffer, GenericIndexed.UTF8_STRATEGY));
|
||||
dimColumns.put(dimension, VSizeColumnarMultiInts.readFromByteBuffer(dimBuffer));
|
||||
}
|
||||
|
@ -410,7 +407,6 @@ public class IndexIO
|
|||
dataInterval,
|
||||
timestamps,
|
||||
metrics,
|
||||
dimValueLookups,
|
||||
dimValueUtf8Lookups,
|
||||
dimColumns,
|
||||
bitmaps,
|
||||
|
@ -432,12 +428,10 @@ public class IndexIO
|
|||
static class LegacyIndexLoader implements IndexLoader
|
||||
{
|
||||
private final IndexIOHandler legacyHandler;
|
||||
private final ColumnConfig columnConfig;
|
||||
|
||||
LegacyIndexLoader(IndexIOHandler legacyHandler, ColumnConfig columnConfig)
|
||||
LegacyIndexLoader(IndexIOHandler legacyHandler)
|
||||
{
|
||||
this.legacyHandler = legacyHandler;
|
||||
this.columnConfig = columnConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -452,21 +446,18 @@ public class IndexIO
|
|||
.setType(ValueType.STRING)
|
||||
.setHasMultipleValues(true)
|
||||
.setDictionaryEncodedColumnSupplier(
|
||||
new DictionaryEncodedColumnSupplier(
|
||||
index.getDimValueLookup(dimension),
|
||||
index.getDimValueUtf8Lookup(dimension),
|
||||
new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
index.getDimValueUtf8Lookup(dimension)::singleThreaded,
|
||||
null,
|
||||
Suppliers.ofInstance(index.getDimColumn(dimension)),
|
||||
columnConfig.columnCacheSizeBytes()
|
||||
Suppliers.ofInstance(index.getDimColumn(dimension))
|
||||
)
|
||||
);
|
||||
GenericIndexed<ImmutableBitmap> bitmaps = index.getBitmapIndexes().get(dimension);
|
||||
ImmutableRTree spatialIndex = index.getSpatialIndexes().get(dimension);
|
||||
builder.setIndexSupplier(
|
||||
new DictionaryEncodedStringIndexSupplier(
|
||||
new StringUtf8ColumnIndexSupplier<>(
|
||||
new ConciseBitmapFactory(),
|
||||
index.getDimValueLookup(dimension),
|
||||
index.getDimValueUtf8Lookup(dimension),
|
||||
index.getDimValueUtf8Lookup(dimension)::singleThreaded,
|
||||
bitmaps,
|
||||
spatialIndex
|
||||
),
|
||||
|
|
|
@ -42,7 +42,6 @@ public class MMappedIndex
|
|||
final Interval dataInterval;
|
||||
final CompressedColumnarLongsSupplier timestamps;
|
||||
final Map<String, MetricHolder> metrics;
|
||||
final Map<String, GenericIndexed<String>> dimValueLookups;
|
||||
final Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups;
|
||||
final Map<String, VSizeColumnarMultiInts> dimColumns;
|
||||
final Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes;
|
||||
|
@ -55,7 +54,6 @@ public class MMappedIndex
|
|||
Interval dataInterval,
|
||||
CompressedColumnarLongsSupplier timestamps,
|
||||
Map<String, MetricHolder> metrics,
|
||||
Map<String, GenericIndexed<String>> dimValueLookups,
|
||||
Map<String, GenericIndexed<ByteBuffer>> dimValueUtf8Lookups,
|
||||
Map<String, VSizeColumnarMultiInts> dimColumns,
|
||||
Map<String, GenericIndexed<ImmutableBitmap>> invertedIndexes,
|
||||
|
@ -68,7 +66,6 @@ public class MMappedIndex
|
|||
this.dataInterval = dataInterval;
|
||||
this.timestamps = timestamps;
|
||||
this.metrics = metrics;
|
||||
this.dimValueLookups = dimValueLookups;
|
||||
this.dimValueUtf8Lookups = dimValueUtf8Lookups;
|
||||
this.dimColumns = dimColumns;
|
||||
this.invertedIndexes = invertedIndexes;
|
||||
|
@ -97,11 +94,6 @@ public class MMappedIndex
|
|||
return metrics.get(metric);
|
||||
}
|
||||
|
||||
public GenericIndexed<String> getDimValueLookup(String dimension)
|
||||
{
|
||||
return dimValueLookups.get(dimension);
|
||||
}
|
||||
|
||||
public GenericIndexed<ByteBuffer> getDimValueUtf8Lookup(String dimension)
|
||||
{
|
||||
return dimValueUtf8Lookups.get(dimension);
|
||||
|
|
|
@ -21,7 +21,23 @@ package org.apache.druid.segment.column;
|
|||
|
||||
public interface ColumnConfig
|
||||
{
|
||||
int columnCacheSizeBytes();
|
||||
ColumnConfig DEFAULT = new ColumnConfig() {};
|
||||
|
||||
ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig()
|
||||
{
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValuePredicateIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap
|
||||
|
|
|
@ -31,41 +31,12 @@ import java.util.NoSuchElementException;
|
|||
|
||||
public final class IndexedStringDruidPredicateIndex<TDictionary extends Indexed<String>> implements DruidPredicateIndex
|
||||
{
|
||||
static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValuePredicateIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
};
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final TDictionary dictionary;
|
||||
private final Indexed<ImmutableBitmap> bitmaps;
|
||||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
|
||||
public IndexedStringDruidPredicateIndex(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
Indexed<ImmutableBitmap> bitmaps
|
||||
)
|
||||
{
|
||||
this(bitmapFactory, dictionary, bitmaps, ALWAYS_USE_INDEXES, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
public IndexedStringDruidPredicateIndex(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
|
|
|
@ -47,23 +47,6 @@ public final class IndexedUtf8LexicographicalRangeIndex<TDictionary extends Inde
|
|||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
|
||||
public IndexedUtf8LexicographicalRangeIndex(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
Indexed<ImmutableBitmap> bitmaps,
|
||||
boolean hasNull
|
||||
)
|
||||
{
|
||||
this(
|
||||
bitmapFactory,
|
||||
dictionary,
|
||||
bitmaps,
|
||||
hasNull,
|
||||
IndexedStringDruidPredicateIndex.ALWAYS_USE_INDEXES,
|
||||
Integer.MAX_VALUE
|
||||
);
|
||||
}
|
||||
|
||||
public IndexedUtf8LexicographicalRangeIndex(
|
||||
BitmapFactory bitmapFactory,
|
||||
TDictionary dictionary,
|
||||
|
|
|
@ -1,807 +0,0 @@
|
|||
/*
|
||||
* 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.segment.column;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.AbstractDimensionSelector;
|
||||
import org.apache.druid.segment.DimensionSelectorUtils;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.ColumnarMultiInts;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
import org.apache.druid.segment.data.SingleIndexedInt;
|
||||
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||
import org.apache.druid.segment.historical.HistoricalDimensionSelector;
|
||||
import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorInspector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<String>
|
||||
{
|
||||
@Nullable
|
||||
private final ColumnarInts column;
|
||||
@Nullable
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final Indexed<String> dictionary;
|
||||
private final Indexed<ByteBuffer> dictionaryUtf8;
|
||||
|
||||
public StringDictionaryEncodedColumn(
|
||||
@Nullable ColumnarInts singleValueColumn,
|
||||
@Nullable ColumnarMultiInts multiValueColumn,
|
||||
Indexed<String> dictionary,
|
||||
Indexed<ByteBuffer> dictionaryUtf8
|
||||
)
|
||||
{
|
||||
this.column = singleValueColumn;
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.dictionary = dictionary;
|
||||
this.dictionaryUtf8 = dictionaryUtf8;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return hasMultipleValues() ? multiValueColumn.size() : column.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return column == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSingleValueRow(int rowNum)
|
||||
{
|
||||
return column.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getMultiValueRow(int rowNum)
|
||||
{
|
||||
return multiValueColumn.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return dictionary.get(id);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns the value for a particular dictionary id as UTF-8 bytes.
|
||||
*
|
||||
* The returned buffer is in big-endian order. It is not reused, so callers may modify the position, limit, byte
|
||||
* order, etc of the buffer.
|
||||
*
|
||||
* The returned buffer points to the original data, so callers must take care not to use it outside the valid
|
||||
* lifetime of this column.
|
||||
*
|
||||
* @param id id to lookup the dictionary value for
|
||||
*
|
||||
* @return dictionary value for the given id, or null if the value is itself null
|
||||
*/
|
||||
@Nullable
|
||||
public ByteBuffer lookupNameUtf8(int id)
|
||||
{
|
||||
return dictionaryUtf8.get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return dictionary.indexOf(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
return dictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HistoricalDimensionSelector makeDimensionSelector(
|
||||
final ReadableOffset offset,
|
||||
@Nullable final ExtractionFn extractionFn
|
||||
)
|
||||
{
|
||||
abstract class QueryableDimensionSelector extends AbstractDimensionSelector
|
||||
implements HistoricalDimensionSelector, IdLookup
|
||||
{
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
/*
|
||||
This is technically wrong if
|
||||
extractionFn != null && (extractionFn.getExtractionType() != ExtractionFn.ExtractionType.ONE_TO_ONE ||
|
||||
!extractionFn.preservesOrdering())
|
||||
However current behavior allows some GroupBy-V1 queries to work that wouldn't work otherwise and doesn't
|
||||
cause any problems due to special handling of extractionFn everywhere.
|
||||
See https://github.com/apache/druid/pull/8433
|
||||
*/
|
||||
return getCardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
final String value = StringDictionaryEncodedColumn.this.lookupName(id);
|
||||
return extractionFn == null ? value : extractionFn.apply(value);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ByteBuffer lookupNameUtf8(int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsLookupNameUtf8()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return extractionFn == null ? this : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
if (extractionFn != null) {
|
||||
throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function");
|
||||
}
|
||||
return StringDictionaryEncodedColumn.this.lookupId(name);
|
||||
}
|
||||
}
|
||||
|
||||
if (hasMultipleValues()) {
|
||||
class MultiValueDimensionSelector extends QueryableDimensionSelector
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
return multiValueColumn.get(offset.getOffset());
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow(int offset)
|
||||
{
|
||||
return multiValueColumn.get(offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(@Nullable String value)
|
||||
{
|
||||
return DimensionSelectorUtils.makeValueMatcherGeneric(this, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||
{
|
||||
return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
return defaultGetObject();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("multiValueColumn", multiValueColumn);
|
||||
inspector.visit("offset", offset);
|
||||
inspector.visit("extractionFn", extractionFn);
|
||||
}
|
||||
}
|
||||
return new MultiValueDimensionSelector();
|
||||
} else {
|
||||
class SingleValueQueryableDimensionSelector extends QueryableDimensionSelector
|
||||
implements SingleValueHistoricalDimensionSelector
|
||||
{
|
||||
private final SingleIndexedInt row = new SingleIndexedInt();
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
row.setValue(getRowValue());
|
||||
return row;
|
||||
}
|
||||
|
||||
public int getRowValue()
|
||||
{
|
||||
return column.get(offset.getOffset());
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow(int offset)
|
||||
{
|
||||
row.setValue(getRowValue(offset));
|
||||
return row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRowValue(int offset)
|
||||
{
|
||||
return column.get(offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final @Nullable String value)
|
||||
{
|
||||
if (extractionFn == null) {
|
||||
final int valueId = lookupId(value);
|
||||
if (valueId >= 0) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return getRowValue() == valueId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", StringDictionaryEncodedColumn.this);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return BooleanValueMatcher.of(false);
|
||||
}
|
||||
} else {
|
||||
// Employ caching BitSet optimization
|
||||
return makeValueMatcher(Predicates.equalTo(value));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||
{
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
final int id = getRowValue();
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final boolean matches = predicate.apply(lookupName(id));
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
}
|
||||
return matches;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", StringDictionaryEncodedColumn.this);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
return lookupName(getRowValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", column);
|
||||
inspector.visit("offset", offset);
|
||||
inspector.visit("extractionFn", extractionFn);
|
||||
}
|
||||
}
|
||||
return new SingleValueQueryableDimensionSelector();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset)
|
||||
{
|
||||
final class StringVectorSelector extends StringSingleValueDimensionVectorSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
super(column, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return getCardinality();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(final int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupName(id);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ByteBuffer lookupNameUtf8(int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(@Nullable final String name)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupId(name);
|
||||
}
|
||||
}
|
||||
|
||||
return new StringVectorSelector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(final ReadableVectorOffset offset)
|
||||
{
|
||||
final class MultiStringVectorSelector extends StringMultiValueDimensionVectorSelector
|
||||
{
|
||||
|
||||
public MultiStringVectorSelector()
|
||||
{
|
||||
super(multiValueColumn, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return getCardinality();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(final int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupName(id);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public ByteBuffer lookupNameUtf8(int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupNameUtf8(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(@Nullable final String name)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupId(name);
|
||||
}
|
||||
}
|
||||
return new MultiStringVectorSelector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
if (!hasMultipleValues()) {
|
||||
final class StringVectorSelector extends StringVectorObjectSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
super(column, offset);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupName(id);
|
||||
}
|
||||
}
|
||||
|
||||
return new StringVectorSelector();
|
||||
} else {
|
||||
final class MultiStringVectorSelector extends MultiValueStringVectorObjectSelector
|
||||
{
|
||||
public MultiStringVectorSelector()
|
||||
{
|
||||
super(multiValueColumn, offset);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return StringDictionaryEncodedColumn.this.lookupName(id);
|
||||
}
|
||||
}
|
||||
|
||||
return new MultiStringVectorSelector();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
CloseableUtils.closeAll(
|
||||
dictionary instanceof Closeable ? (Closeable) dictionary : null /* Dictionary may be CachingIndexed */,
|
||||
column,
|
||||
multiValueColumn
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringSingleValueDimensionVectorSelector
|
||||
implements SingleValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarInts column;
|
||||
private final ReadableVectorOffset offset;
|
||||
private final int[] vector;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringSingleValueDimensionVectorSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
this.offset = offset;
|
||||
this.vector = new int[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] getRowVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return vector;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsLookupNameUtf8()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringMultiValueDimensionVectorSelector
|
||||
implements MultiValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final IndexedInts[] vector;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringMultiValueDimensionVectorSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.offset = offset;
|
||||
this.vector = new IndexedInts[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts[] getRowVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return vector;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
final int currentOffset = offset.getStartOffset();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(i + currentOffset);
|
||||
}
|
||||
} else {
|
||||
final int[] offsets = offset.getOffsets();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(offsets[i]);
|
||||
}
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsLookupNameUtf8()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*/
|
||||
public abstract static class StringVectorObjectSelector implements VectorObjectSelector
|
||||
{
|
||||
private final ColumnarInts column;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final int[] vector;
|
||||
private final Object[] strings;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringVectorObjectSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
this.offset = offset;
|
||||
this.vector = new int[offset.getMaxVectorSize()];
|
||||
this.strings = new Object[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return strings;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
strings[i] = lookupName(vector[i]);
|
||||
}
|
||||
id = offset.getId();
|
||||
|
||||
return strings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public abstract String lookupName(int id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*/
|
||||
public abstract static class MultiValueStringVectorObjectSelector implements VectorObjectSelector
|
||||
{
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final IndexedInts[] vector;
|
||||
private final Object[] strings;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public MultiValueStringVectorObjectSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.offset = offset;
|
||||
this.vector = new IndexedInts[offset.getMaxVectorSize()];
|
||||
this.strings = new Object[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public abstract String lookupName(int id);
|
||||
|
||||
@Override
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return strings;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
final int currentOffset = offset.getStartOffset();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(i + currentOffset);
|
||||
}
|
||||
} else {
|
||||
final int[] offsets = offset.getOffsets();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(offsets[i]);
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
IndexedInts ithRow = vector[i];
|
||||
if (ithRow.size() == 0) {
|
||||
strings[i] = null;
|
||||
} else if (ithRow.size() == 1) {
|
||||
strings[i] = lookupName(ithRow.get(0));
|
||||
} else {
|
||||
List<String> row = new ArrayList<>(ithRow.size());
|
||||
// noinspection SSBasedInspection
|
||||
for (int j = 0; j < ithRow.size(); j++) {
|
||||
row.add(lookupName(ithRow.get(j)));
|
||||
}
|
||||
strings[i] = row;
|
||||
}
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return strings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -39,6 +39,7 @@ import org.apache.druid.segment.historical.HistoricalDimensionSelector;
|
|||
import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
||||
import org.apache.druid.segment.nested.NestedCommonFormatColumn;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorInspector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
|
@ -47,19 +48,18 @@ import org.apache.druid.utils.CloseableUtils;
|
|||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* {@link DictionaryEncodedColumn<String>} for a column which has only a UTF-8 dictionary, no String dictionary.
|
||||
* {@link DictionaryEncodedColumn<String>} for a column which has a {@link ByteBuffer} based UTF-8 dictionary.
|
||||
* <p>
|
||||
* This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than lacking a
|
||||
* String dictionary.
|
||||
* <p>
|
||||
* Implements {@link NestedCommonFormatColumn} so it can be used as a reader for single value string specializations
|
||||
* of {@link org.apache.druid.segment.AutoTypeColumnIndexer}.
|
||||
*/
|
||||
public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColumn<String>,
|
||||
NestedCommonFormatColumn
|
||||
public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColumn<String>, NestedCommonFormatColumn
|
||||
{
|
||||
@Nullable
|
||||
private final ColumnarInts column;
|
||||
|
@ -364,7 +364,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(final ReadableVectorOffset offset)
|
||||
{
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
final class StringVectorSelector extends StringSingleValueDimensionVectorSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
|
@ -404,7 +404,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
@Override
|
||||
public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(final ReadableVectorOffset offset)
|
||||
{
|
||||
final class MultiStringVectorSelector extends StringDictionaryEncodedColumn.StringMultiValueDimensionVectorSelector
|
||||
final class MultiStringVectorSelector extends StringMultiValueDimensionVectorSelector
|
||||
{
|
||||
public MultiStringVectorSelector()
|
||||
{
|
||||
|
@ -446,7 +446,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
if (!hasMultipleValues()) {
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector
|
||||
final class StringVectorSelector extends StringVectorObjectSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
|
@ -462,7 +462,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
}
|
||||
return new StringVectorSelector();
|
||||
} else {
|
||||
final class MultiStringVectorSelector extends StringDictionaryEncodedColumn.MultiValueStringVectorObjectSelector
|
||||
final class MultiStringVectorSelector extends MultiValueStringVectorObjectSelector
|
||||
{
|
||||
public MultiStringVectorSelector()
|
||||
{
|
||||
|
@ -497,4 +497,309 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum
|
|||
{
|
||||
return new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringSingleValueDimensionVectorSelector
|
||||
implements SingleValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarInts column;
|
||||
private final ReadableVectorOffset offset;
|
||||
private final int[] vector;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringSingleValueDimensionVectorSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
this.offset = offset;
|
||||
this.vector = new int[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public int[] getRowVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return vector;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsLookupNameUtf8()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*
|
||||
* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}.
|
||||
*/
|
||||
public abstract static class StringMultiValueDimensionVectorSelector
|
||||
implements MultiValueDimensionVectorSelector, IdLookup
|
||||
{
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final IndexedInts[] vector;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringMultiValueDimensionVectorSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.offset = offset;
|
||||
this.vector = new IndexedInts[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts[] getRowVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return vector;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
final int currentOffset = offset.getStartOffset();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(i + currentOffset);
|
||||
}
|
||||
} else {
|
||||
final int[] offsets = offset.getOffsets();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(offsets[i]);
|
||||
}
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return vector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsLookupNameUtf8()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*/
|
||||
public abstract static class StringVectorObjectSelector implements VectorObjectSelector
|
||||
{
|
||||
private final ColumnarInts column;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final int[] vector;
|
||||
private final Object[] strings;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public StringVectorObjectSelector(
|
||||
ColumnarInts column,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.column = column;
|
||||
this.offset = offset;
|
||||
this.vector = new int[offset.getMaxVectorSize()];
|
||||
this.strings = new Object[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return strings;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
column.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
column.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
strings[i] = lookupName(vector[i]);
|
||||
}
|
||||
id = offset.getId();
|
||||
|
||||
return strings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public abstract String lookupName(int id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Base type for a {@link VectorObjectSelector} for a dictionary encoded {@link ColumnType#STRING}
|
||||
* built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods.
|
||||
*/
|
||||
public abstract static class MultiValueStringVectorObjectSelector implements VectorObjectSelector
|
||||
{
|
||||
private final ColumnarMultiInts multiValueColumn;
|
||||
private final ReadableVectorOffset offset;
|
||||
|
||||
private final IndexedInts[] vector;
|
||||
private final Object[] strings;
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
public MultiValueStringVectorObjectSelector(
|
||||
ColumnarMultiInts multiValueColumn,
|
||||
ReadableVectorOffset offset
|
||||
)
|
||||
{
|
||||
this.multiValueColumn = multiValueColumn;
|
||||
this.offset = offset;
|
||||
this.vector = new IndexedInts[offset.getMaxVectorSize()];
|
||||
this.strings = new Object[offset.getMaxVectorSize()];
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public abstract String lookupName(int id);
|
||||
|
||||
@Override
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return strings;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
final int currentOffset = offset.getStartOffset();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(i + currentOffset);
|
||||
}
|
||||
} else {
|
||||
final int[] offsets = offset.getOffsets();
|
||||
final int numRows = offset.getCurrentVectorSize();
|
||||
|
||||
for (int i = 0; i < numRows; i++) {
|
||||
// Must use getUnshared, otherwise all elements in the vector could be the same shared object.
|
||||
vector[i] = multiValueColumn.getUnshared(offsets[i]);
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
IndexedInts ithRow = vector[i];
|
||||
if (ithRow.size() == 0) {
|
||||
strings[i] = null;
|
||||
} else if (ithRow.size() == 1) {
|
||||
strings[i] = lookupName(ithRow.get(0));
|
||||
} else {
|
||||
List<String> row = new ArrayList<>(ithRow.size());
|
||||
// noinspection SSBasedInspection
|
||||
for (int j = 0; j < ithRow.size(); j++) {
|
||||
row.add(lookupName(ithRow.get(j)));
|
||||
}
|
||||
strings[i] = row;
|
||||
}
|
||||
}
|
||||
|
||||
id = offset.getId();
|
||||
return strings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,158 +0,0 @@
|
|||
/*
|
||||
* 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.segment.data;
|
||||
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.ToIntFunction;
|
||||
|
||||
public class CachingIndexed<T> implements CloseableIndexed<T>
|
||||
{
|
||||
private static final int INITIAL_CACHE_CAPACITY = 16384;
|
||||
|
||||
private static final Logger log = new Logger(CachingIndexed.class);
|
||||
|
||||
private final Indexed<T> delegate;
|
||||
private final ToIntFunction<T> sizeFn;
|
||||
@Nullable
|
||||
private final SizedLRUMap<Integer, T> cachedValues;
|
||||
|
||||
/**
|
||||
* Creates a CachingIndexed wrapping the given GenericIndexed with a value lookup cache
|
||||
*
|
||||
* CachingIndexed objects are not thread safe and should only be used by a single thread at a time.
|
||||
* CachingIndexed objects must be closed to release any underlying cache resources.
|
||||
*
|
||||
* @param delegate the Indexed to wrap with a lookup cache.
|
||||
* @param sizeFn function that determines the size in bytes of an object
|
||||
* @param lookupCacheSize maximum size in bytes of the lookup cache if greater than zero
|
||||
*/
|
||||
public CachingIndexed(Indexed<T> delegate, final ToIntFunction<T> sizeFn, final int lookupCacheSize)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.sizeFn = sizeFn;
|
||||
|
||||
if (lookupCacheSize > 0) {
|
||||
log.debug("Allocating column cache of max size[%d]", lookupCacheSize);
|
||||
cachedValues = new SizedLRUMap<>(INITIAL_CACHE_CAPACITY, lookupCacheSize);
|
||||
} else {
|
||||
cachedValues = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return delegate.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public T get(int index)
|
||||
{
|
||||
if (cachedValues != null) {
|
||||
final T cached = cachedValues.getValue(index);
|
||||
if (cached != null) {
|
||||
return cached;
|
||||
}
|
||||
|
||||
final T value = delegate.get(index);
|
||||
cachedValues.put(index, value, sizeFn.applyAsInt(value));
|
||||
return value;
|
||||
} else {
|
||||
return delegate.get(index);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int indexOf(@Nullable T value)
|
||||
{
|
||||
return delegate.indexOf(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSorted()
|
||||
{
|
||||
return delegate.isSorted();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<T> iterator()
|
||||
{
|
||||
return delegate.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
if (cachedValues != null) {
|
||||
log.debug("Closing column cache");
|
||||
cachedValues.clear();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("cachedValues", cachedValues != null);
|
||||
inspector.visit("delegate", delegate);
|
||||
}
|
||||
|
||||
private static class SizedLRUMap<K, V> extends LinkedHashMap<K, Pair<Integer, V>>
|
||||
{
|
||||
private final int maxBytes;
|
||||
private int numBytes = 0;
|
||||
|
||||
SizedLRUMap(int initialCapacity, int maxBytes)
|
||||
{
|
||||
super(initialCapacity, 0.75f, true);
|
||||
this.maxBytes = maxBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean removeEldestEntry(Map.Entry<K, Pair<Integer, V>> eldest)
|
||||
{
|
||||
if (numBytes > maxBytes) {
|
||||
numBytes -= eldest.getValue().lhs;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public void put(K key, @Nullable V value, int size)
|
||||
{
|
||||
final int totalSize = size + 48; // add approximate object overhead
|
||||
numBytes += totalSize;
|
||||
super.put(key, new Pair<>(totalSize, value));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public V getValue(Object key)
|
||||
{
|
||||
final Pair<Integer, V> sizeValuePair = super.get(key);
|
||||
return sizeValuePair == null ? null : sizeValuePair.rhs;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -40,7 +40,7 @@ import org.apache.druid.segment.IdLookup;
|
|||
import org.apache.druid.segment.LongColumnSelector;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.Types;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.ColumnarDoubles;
|
||||
|
@ -678,7 +678,7 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
|||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
|
@ -786,7 +786,7 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
|
|||
}
|
||||
};
|
||||
}
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringVectorObjectSelector
|
||||
final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringVectorObjectSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
|
|
|
@ -20,30 +20,16 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnBuilder;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.column.DruidPredicateIndex;
|
||||
import org.apache.druid.segment.column.IndexedStringDictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.IndexedStringDruidPredicateIndex;
|
||||
import org.apache.druid.segment.column.IndexedUtf8LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex;
|
||||
import org.apache.druid.segment.column.LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.NullValueIndex;
|
||||
import org.apache.druid.segment.column.SimpleImmutableBitmapIndex;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
||||
|
@ -53,6 +39,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.VByte;
|
||||
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
|
@ -73,12 +60,10 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
final int columnNameLength = VByte.readInt(bb);
|
||||
final String columnName = StringUtils.fromUtf8(bb, columnNameLength);
|
||||
|
||||
|
||||
if (version == NestedCommonFormatColumnSerializer.V0) {
|
||||
try {
|
||||
final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
|
||||
final GenericIndexed<ByteBuffer> stringDictionary;
|
||||
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
||||
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
|
||||
|
||||
final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||
mapper,
|
||||
|
@ -92,17 +77,19 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
||||
final byte encodingId = stringDictionaryBuffer.get();
|
||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
||||
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
|
||||
dictionarySupplier = FrontCodedIndexed.read(
|
||||
stringDictionaryBuffer,
|
||||
byteOrder
|
||||
);
|
||||
stringDictionary = null;
|
||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
||||
// this provides backwards compatibility should we switch at some point in the future to always
|
||||
// writing dictionaryVersion
|
||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
||||
frontCodedStringDictionarySupplier = null;
|
||||
dictionarySupplier = GenericIndexed.read(
|
||||
stringDictionaryBuffer,
|
||||
GenericIndexed.UTF8_STRATEGY,
|
||||
mapper
|
||||
)::singleThreaded;
|
||||
} else {
|
||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
||||
}
|
||||
|
@ -111,8 +98,11 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
||||
// GenericIndexed version can be correctly read
|
||||
stringDictionaryBuffer.position(dictionaryStartPosition);
|
||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
||||
frontCodedStringDictionarySupplier = null;
|
||||
dictionarySupplier = GenericIndexed.read(
|
||||
stringDictionaryBuffer,
|
||||
GenericIndexed.UTF8_STRATEGY,
|
||||
mapper
|
||||
)::singleThreaded;
|
||||
}
|
||||
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||
mapper,
|
||||
|
@ -138,8 +128,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
size = throwAway.size();
|
||||
}
|
||||
return new ScalarStringColumnAndIndexSupplier(
|
||||
stringDictionary,
|
||||
frontCodedStringDictionarySupplier,
|
||||
dictionarySupplier,
|
||||
ints,
|
||||
valueIndexes,
|
||||
bitmapSerdeFactory,
|
||||
|
@ -155,20 +144,13 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private final GenericIndexed<ByteBuffer> stringDictionary;
|
||||
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
||||
private final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
|
||||
private final Supplier<ColumnarInts> encodedColumnSupplier;
|
||||
private final GenericIndexed<ImmutableBitmap> valueIndexes;
|
||||
private final ImmutableBitmap nullValueBitmap;
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
private final ColumnIndexSupplier stringIndexSupplier;
|
||||
|
||||
private ScalarStringColumnAndIndexSupplier(
|
||||
GenericIndexed<ByteBuffer> stringDictionary,
|
||||
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
|
||||
Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier,
|
||||
Supplier<ColumnarInts> encodedColumnSupplier,
|
||||
GenericIndexed<ImmutableBitmap> valueIndexes,
|
||||
BitmapSerdeFactory serdeFactory,
|
||||
|
@ -176,27 +158,23 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
int numRows
|
||||
)
|
||||
{
|
||||
this.stringDictionary = stringDictionary;
|
||||
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
|
||||
this.dictionarySupplier = dictionarySupplier;
|
||||
this.encodedColumnSupplier = encodedColumnSupplier;
|
||||
this.valueIndexes = valueIndexes;
|
||||
this.bitmapFactory = serdeFactory.getBitmapFactory();
|
||||
this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0);
|
||||
this.columnConfig = columnConfig;
|
||||
this.numRows = numRows;
|
||||
this.stringIndexSupplier = new StringUtf8ColumnIndexSupplier<>(
|
||||
serdeFactory.getBitmapFactory(),
|
||||
dictionarySupplier,
|
||||
valueIndexes,
|
||||
null,
|
||||
columnConfig,
|
||||
numRows
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NestedCommonFormatColumn get()
|
||||
{
|
||||
if (frontCodedStringDictionarySupplier != null) {
|
||||
return new StringUtf8DictionaryEncodedColumn(
|
||||
encodedColumnSupplier.get(),
|
||||
null,
|
||||
frontCodedStringDictionarySupplier.get()
|
||||
);
|
||||
}
|
||||
return new StringUtf8DictionaryEncodedColumn(encodedColumnSupplier.get(), null, stringDictionary.singleThreaded());
|
||||
return new StringUtf8DictionaryEncodedColumn(encodedColumnSupplier.get(), null, dictionarySupplier.get());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -204,44 +182,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (valueIndexes != null) {
|
||||
final Indexed<ImmutableBitmap> singleThreadedBitmaps = valueIndexes.singleThreaded();
|
||||
final Indexed<ByteBuffer> utf8Dictionary = frontCodedStringDictionarySupplier == null
|
||||
? stringDictionary.singleThreaded()
|
||||
: frontCodedStringDictionarySupplier.get();
|
||||
if (clazz.equals(NullValueIndex.class)) {
|
||||
final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap);
|
||||
return (T) (NullValueIndex) () -> nullIndex;
|
||||
} else if (clazz.equals(StringValueSetIndex.class)) {
|
||||
return (T) new IndexedUtf8ValueSetIndex<>(
|
||||
bitmapFactory,
|
||||
utf8Dictionary,
|
||||
singleThreadedBitmaps
|
||||
);
|
||||
} else if (clazz.equals(DruidPredicateIndex.class)) {
|
||||
return (T) new IndexedStringDruidPredicateIndex<>(
|
||||
bitmapFactory,
|
||||
new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary),
|
||||
singleThreadedBitmaps,
|
||||
columnConfig,
|
||||
numRows
|
||||
);
|
||||
} else if (clazz.equals(LexicographicalRangeIndex.class)) {
|
||||
return (T) new IndexedUtf8LexicographicalRangeIndex<>(
|
||||
bitmapFactory,
|
||||
utf8Dictionary,
|
||||
singleThreadedBitmaps,
|
||||
utf8Dictionary.get(0) == null,
|
||||
columnConfig,
|
||||
numRows
|
||||
);
|
||||
} else if (clazz.equals(DictionaryEncodedStringValueIndex.class)
|
||||
|| clazz.equals(DictionaryEncodedValueIndex.class)) {
|
||||
return (T) new IndexedStringDictionaryEncodedStringValueIndex<>(
|
||||
bitmapFactory,
|
||||
new StringEncodingStrategies.Utf8ToStringIndexed(utf8Dictionary),
|
||||
valueIndexes
|
||||
);
|
||||
}
|
||||
return stringIndexSupplier.as(clazz);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -40,8 +40,8 @@ import org.apache.druid.segment.IdLookup;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.FixedIndexed;
|
||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||
|
@ -668,7 +668,7 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
|||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
final class StringVectorSelector extends StringUtf8DictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.druid.segment.data.FrontCodedIndexed;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.GenericIndexedWriter;
|
||||
import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSupplier;
|
||||
import org.apache.druid.segment.data.VSizeColumnarInts;
|
||||
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
||||
|
@ -313,16 +314,21 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
|
||||
final int dictionaryStartPosition = buffer.position();
|
||||
final byte dictionaryVersion = buffer.get();
|
||||
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
|
||||
|
||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
||||
final byte encodingId = buffer.get();
|
||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
||||
readFrontCodedColumn(buffer, builder, rVersion, rFlags, hasMultipleValues);
|
||||
dictionarySupplier = FrontCodedIndexed.read(buffer, byteOrder);
|
||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
||||
// this provides backwards compatibility should we switch at some point in the future to always
|
||||
// writing dictionaryVersion
|
||||
readGenericIndexedColumn(buffer, builder, columnConfig, rVersion, rFlags, hasMultipleValues);
|
||||
dictionarySupplier = GenericIndexed.read(
|
||||
buffer,
|
||||
GenericIndexed.UTF8_STRATEGY,
|
||||
builder.getFileMapper()
|
||||
)::singleThreaded;
|
||||
} else {
|
||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
||||
}
|
||||
|
@ -331,101 +337,13 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
||||
// GenericIndexed version can be correctly read
|
||||
buffer.position(dictionaryStartPosition);
|
||||
readGenericIndexedColumn(buffer, builder, columnConfig, rVersion, rFlags, hasMultipleValues);
|
||||
}
|
||||
}
|
||||
|
||||
private void readGenericIndexedColumn(
|
||||
ByteBuffer buffer,
|
||||
ColumnBuilder builder,
|
||||
ColumnConfig columnConfig,
|
||||
VERSION rVersion,
|
||||
int rFlags,
|
||||
boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
// Duplicate the first buffer since we are reading the dictionary twice.
|
||||
final GenericIndexed<String> rDictionary = GenericIndexed.read(
|
||||
buffer.duplicate(),
|
||||
GenericIndexed.STRING_STRATEGY,
|
||||
builder.getFileMapper()
|
||||
);
|
||||
|
||||
final GenericIndexed<ByteBuffer> rDictionaryUtf8 = GenericIndexed.read(
|
||||
buffer,
|
||||
GenericIndexed.UTF8_STRATEGY,
|
||||
builder.getFileMapper()
|
||||
);
|
||||
|
||||
final WritableSupplier<ColumnarInts> rSingleValuedColumn;
|
||||
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;
|
||||
|
||||
if (hasMultipleValues) {
|
||||
rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags);
|
||||
rSingleValuedColumn = null;
|
||||
} else {
|
||||
rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer);
|
||||
rMultiValuedColumn = null;
|
||||
}
|
||||
|
||||
final String firstDictionaryEntry = rDictionary.get(0);
|
||||
|
||||
DictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier = new DictionaryEncodedColumnSupplier(
|
||||
rDictionary,
|
||||
rDictionaryUtf8,
|
||||
rSingleValuedColumn,
|
||||
rMultiValuedColumn,
|
||||
columnConfig.columnCacheSizeBytes()
|
||||
);
|
||||
|
||||
builder.setHasMultipleValues(hasMultipleValues)
|
||||
.setHasNulls(firstDictionaryEntry == null)
|
||||
.setDictionaryEncodedColumnSupplier(dictionaryEncodedColumnSupplier);
|
||||
|
||||
GenericIndexed<ImmutableBitmap> rBitmaps = null;
|
||||
ImmutableRTree rSpatialIndex = null;
|
||||
if (!Feature.NO_BITMAP_INDEX.isSet(rFlags)) {
|
||||
rBitmaps = GenericIndexed.read(
|
||||
dictionarySupplier = GenericIndexed.read(
|
||||
buffer,
|
||||
bitmapSerdeFactory.getObjectStrategy(),
|
||||
GenericIndexed.UTF8_STRATEGY,
|
||||
builder.getFileMapper()
|
||||
);
|
||||
)::singleThreaded;
|
||||
}
|
||||
|
||||
if (buffer.hasRemaining()) {
|
||||
rSpatialIndex = new ImmutableRTreeObjectStrategy(
|
||||
bitmapSerdeFactory.getBitmapFactory()
|
||||
).fromByteBufferWithSize(buffer);
|
||||
}
|
||||
|
||||
if (rBitmaps != null || rSpatialIndex != null) {
|
||||
builder.setIndexSupplier(
|
||||
new DictionaryEncodedStringIndexSupplier(
|
||||
bitmapSerdeFactory.getBitmapFactory(),
|
||||
rDictionary,
|
||||
rDictionaryUtf8,
|
||||
rBitmaps,
|
||||
rSpatialIndex
|
||||
),
|
||||
rBitmaps != null,
|
||||
rSpatialIndex != null
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void readFrontCodedColumn(
|
||||
ByteBuffer buffer,
|
||||
ColumnBuilder builder,
|
||||
VERSION rVersion,
|
||||
int rFlags,
|
||||
boolean hasMultipleValues
|
||||
)
|
||||
{
|
||||
final Supplier<FrontCodedIndexed> rUtf8Dictionary = FrontCodedIndexed.read(
|
||||
buffer,
|
||||
byteOrder
|
||||
);
|
||||
|
||||
final WritableSupplier<ColumnarInts> rSingleValuedColumn;
|
||||
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;
|
||||
|
||||
|
@ -437,17 +355,16 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
rMultiValuedColumn = null;
|
||||
}
|
||||
|
||||
final boolean hasNulls = rUtf8Dictionary.get().get(0) == null;
|
||||
final boolean hasNulls = dictionarySupplier.get().get(0) == null;
|
||||
|
||||
StringFrontCodedDictionaryEncodedColumnSupplier dictionaryEncodedColumnSupplier =
|
||||
new StringFrontCodedDictionaryEncodedColumnSupplier(
|
||||
rUtf8Dictionary,
|
||||
rSingleValuedColumn,
|
||||
rMultiValuedColumn
|
||||
);
|
||||
final StringUtf8DictionaryEncodedColumnSupplier<?> supplier = new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
dictionarySupplier,
|
||||
rSingleValuedColumn,
|
||||
rMultiValuedColumn
|
||||
);
|
||||
builder.setHasMultipleValues(hasMultipleValues)
|
||||
.setHasNulls(hasNulls)
|
||||
.setDictionaryEncodedColumnSupplier(dictionaryEncodedColumnSupplier);
|
||||
.setDictionaryEncodedColumnSupplier(supplier);
|
||||
|
||||
GenericIndexed<ImmutableBitmap> rBitmaps = null;
|
||||
ImmutableRTree rSpatialIndex = null;
|
||||
|
@ -467,9 +384,9 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
|||
|
||||
if (rBitmaps != null || rSpatialIndex != null) {
|
||||
builder.setIndexSupplier(
|
||||
new StringFrontCodedColumnIndexSupplier(
|
||||
new StringUtf8ColumnIndexSupplier(
|
||||
bitmapSerdeFactory.getBitmapFactory(),
|
||||
rUtf8Dictionary,
|
||||
dictionarySupplier,
|
||||
rBitmaps,
|
||||
rSpatialIndex
|
||||
),
|
||||
|
|
|
@ -1,100 +0,0 @@
|
|||
/*
|
||||
* 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.segment.serde;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.data.CachingIndexed;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.ColumnarMultiInts;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class DictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn<?>>
|
||||
{
|
||||
private final GenericIndexed<String> dictionary;
|
||||
private final GenericIndexed<ByteBuffer> dictionaryUtf8;
|
||||
private final @Nullable Supplier<ColumnarInts> singleValuedColumn;
|
||||
private final @Nullable Supplier<ColumnarMultiInts> multiValuedColumn;
|
||||
private final int lookupCacheSize;
|
||||
|
||||
public DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed<String> dictionary,
|
||||
GenericIndexed<ByteBuffer> dictionaryUtf8,
|
||||
@Nullable Supplier<ColumnarInts> singleValuedColumn,
|
||||
@Nullable Supplier<ColumnarMultiInts> multiValuedColumn,
|
||||
int lookupCacheSize
|
||||
)
|
||||
{
|
||||
this.dictionary = dictionary;
|
||||
this.dictionaryUtf8 = dictionaryUtf8;
|
||||
this.singleValuedColumn = singleValuedColumn;
|
||||
this.multiValuedColumn = multiValuedColumn;
|
||||
this.lookupCacheSize = lookupCacheSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DictionaryEncodedColumn<?> get()
|
||||
{
|
||||
final Indexed<String> cacheWrappedDictionary;
|
||||
final Indexed<ByteBuffer> singleThreadedDictionaryUtf8 = dictionaryUtf8.singleThreaded();
|
||||
|
||||
if (lookupCacheSize > 0) {
|
||||
cacheWrappedDictionary = new CachingIndexed<>(
|
||||
dictionary.singleThreaded(),
|
||||
s -> s == null ? 0 : s.length() * Character.BYTES,
|
||||
lookupCacheSize
|
||||
);
|
||||
} else {
|
||||
cacheWrappedDictionary = dictionary.singleThreaded();
|
||||
}
|
||||
|
||||
if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedDictionaryUtf8)) {
|
||||
return new StringDictionaryEncodedColumn(
|
||||
singleValuedColumn != null ? new CombineFirstTwoValuesColumnarInts(singleValuedColumn.get()) : null,
|
||||
multiValuedColumn != null ? new CombineFirstTwoValuesColumnarMultiInts(multiValuedColumn.get()) : null,
|
||||
CombineFirstTwoEntriesIndexed.returnNull(cacheWrappedDictionary),
|
||||
CombineFirstTwoEntriesIndexed.returnNull(singleThreadedDictionaryUtf8)
|
||||
);
|
||||
} else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedDictionaryUtf8)) {
|
||||
return new StringDictionaryEncodedColumn(
|
||||
singleValuedColumn != null ? singleValuedColumn.get() : null,
|
||||
multiValuedColumn != null ? multiValuedColumn.get() : null,
|
||||
new ReplaceFirstValueWithNullIndexed<>(cacheWrappedDictionary),
|
||||
new ReplaceFirstValueWithNullIndexed<>(singleThreadedDictionaryUtf8)
|
||||
);
|
||||
} else {
|
||||
return new StringDictionaryEncodedColumn(
|
||||
singleValuedColumn != null ? singleValuedColumn.get() : null,
|
||||
multiValuedColumn != null ? multiValuedColumn.get() : null,
|
||||
cacheWrappedDictionary,
|
||||
singleThreadedDictionaryUtf8
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,127 +0,0 @@
|
|||
/*
|
||||
* 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.segment.serde;
|
||||
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.spatial.ImmutableRTree;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.column.DruidPredicateIndex;
|
||||
import org.apache.druid.segment.column.IndexedStringDictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.IndexedStringDruidPredicateIndex;
|
||||
import org.apache.druid.segment.column.IndexedUtf8LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex;
|
||||
import org.apache.druid.segment.column.LexicographicalRangeIndex;
|
||||
import org.apache.druid.segment.column.NullValueIndex;
|
||||
import org.apache.druid.segment.column.SimpleImmutableBitmapIndex;
|
||||
import org.apache.druid.segment.column.SpatialIndex;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.column.Utf8ValueSetIndex;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class DictionaryEncodedStringIndexSupplier implements ColumnIndexSupplier
|
||||
{
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final GenericIndexed<String> dictionary;
|
||||
private final GenericIndexed<ByteBuffer> dictionaryUtf8;
|
||||
@Nullable
|
||||
private final GenericIndexed<ImmutableBitmap> bitmaps;
|
||||
@Nullable
|
||||
private final ImmutableRTree indexedTree;
|
||||
|
||||
public DictionaryEncodedStringIndexSupplier(
|
||||
BitmapFactory bitmapFactory,
|
||||
GenericIndexed<String> dictionary,
|
||||
GenericIndexed<ByteBuffer> dictionaryUtf8,
|
||||
@Nullable GenericIndexed<ImmutableBitmap> bitmaps,
|
||||
@Nullable ImmutableRTree indexedTree
|
||||
)
|
||||
{
|
||||
this.bitmapFactory = bitmapFactory;
|
||||
this.dictionary = dictionary;
|
||||
this.dictionaryUtf8 = dictionaryUtf8;
|
||||
this.bitmaps = bitmaps;
|
||||
this.indexedTree = indexedTree;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (bitmaps != null) {
|
||||
Indexed<String> singleThreadedStrings = dictionary.singleThreaded();
|
||||
Indexed<ByteBuffer> singleThreadedUtf8 = dictionaryUtf8.singleThreaded();
|
||||
Indexed<ImmutableBitmap> singleThreadedBitmaps = bitmaps.singleThreaded();
|
||||
|
||||
if (NullHandling.mustCombineNullAndEmptyInDictionary(singleThreadedUtf8)) {
|
||||
singleThreadedStrings = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedStrings);
|
||||
singleThreadedUtf8 = CombineFirstTwoEntriesIndexed.returnNull(singleThreadedUtf8);
|
||||
singleThreadedBitmaps = CombineFirstTwoEntriesIndexed.unionBitmaps(bitmapFactory, singleThreadedBitmaps);
|
||||
} else if (NullHandling.mustReplaceFirstValueWithNullInDictionary(singleThreadedUtf8)) {
|
||||
singleThreadedStrings = new ReplaceFirstValueWithNullIndexed<>(singleThreadedStrings);
|
||||
singleThreadedUtf8 = new ReplaceFirstValueWithNullIndexed<>(singleThreadedUtf8);
|
||||
}
|
||||
|
||||
if (clazz.equals(NullValueIndex.class)) {
|
||||
final BitmapColumnIndex nullIndex;
|
||||
final ByteBuffer firstValue = singleThreadedUtf8.get(0);
|
||||
if (NullHandling.isNullOrEquivalent(firstValue)) {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0));
|
||||
} else {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
}
|
||||
return (T) (NullValueIndex) () -> nullIndex;
|
||||
} else if (clazz.equals(StringValueSetIndex.class)) {
|
||||
return (T) new IndexedUtf8ValueSetIndex<>(bitmapFactory, singleThreadedUtf8, singleThreadedBitmaps);
|
||||
} else if (clazz.equals(Utf8ValueSetIndex.class)) {
|
||||
return (T) new IndexedUtf8ValueSetIndex<>(bitmapFactory, singleThreadedUtf8, singleThreadedBitmaps);
|
||||
} else if (clazz.equals(DruidPredicateIndex.class)) {
|
||||
return (T) new IndexedStringDruidPredicateIndex<>(bitmapFactory, singleThreadedStrings, singleThreadedBitmaps);
|
||||
} else if (clazz.equals(LexicographicalRangeIndex.class)) {
|
||||
return (T) new IndexedUtf8LexicographicalRangeIndex<>(
|
||||
bitmapFactory,
|
||||
singleThreadedUtf8,
|
||||
singleThreadedBitmaps,
|
||||
NullHandling.isNullOrEquivalent(singleThreadedStrings.get(0))
|
||||
);
|
||||
} else if (clazz.equals(DictionaryEncodedStringValueIndex.class)
|
||||
|| clazz.equals(DictionaryEncodedValueIndex.class)) {
|
||||
return (T) new IndexedStringDictionaryEncodedStringValueIndex<>(
|
||||
bitmapFactory,
|
||||
singleThreadedStrings,
|
||||
singleThreadedBitmaps
|
||||
);
|
||||
}
|
||||
}
|
||||
if (indexedTree != null && clazz.equals(SpatialIndex.class)) {
|
||||
return (T) (SpatialIndex) () -> indexedTree;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import org.apache.druid.segment.column.ColumnBuilder;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
|
@ -128,106 +130,18 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
public Deserializer getDeserializer()
|
||||
{
|
||||
if (isVariantType || logicalType.isArray()) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
|
||||
logicalType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
return new VariantColumnDeserializer();
|
||||
}
|
||||
if (logicalType.is(ValueType.STRING)) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
return new StringColumnDeserializer();
|
||||
}
|
||||
if (logicalType.is(ValueType.LONG)) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
return new LongColumnDeserializer();
|
||||
}
|
||||
if (logicalType.is(ValueType.DOUBLE)) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
return new DoubleColumnDeserializer();
|
||||
}
|
||||
return (buffer, builder, columnConfig) -> {
|
||||
NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
|
||||
logicalType,
|
||||
hasNulls,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig,
|
||||
bitmapSerdeFactory,
|
||||
byteOrder
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
ColumnType simpleType = supplier.getLogicalType();
|
||||
ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType;
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls));
|
||||
builder.setFilterable(true);
|
||||
};
|
||||
return new NestedColumnDeserializer();
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -260,6 +174,129 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
return bitmapSerdeFactory;
|
||||
}
|
||||
|
||||
private class StringColumnDeserializer implements Deserializer
|
||||
{
|
||||
@Override
|
||||
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
||||
{
|
||||
ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
}
|
||||
}
|
||||
|
||||
private class LongColumnDeserializer implements Deserializer
|
||||
{
|
||||
@Override
|
||||
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
||||
{
|
||||
ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
}
|
||||
}
|
||||
|
||||
private class DoubleColumnDeserializer implements Deserializer
|
||||
{
|
||||
@Override
|
||||
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
||||
{
|
||||
ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read(
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
}
|
||||
}
|
||||
|
||||
private class VariantColumnDeserializer implements Deserializer
|
||||
{
|
||||
@Override
|
||||
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
||||
{
|
||||
VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
|
||||
logicalType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
}
|
||||
}
|
||||
|
||||
private class NestedColumnDeserializer implements Deserializer
|
||||
{
|
||||
@Override
|
||||
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
|
||||
{
|
||||
NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
|
||||
logicalType,
|
||||
hasNulls,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig,
|
||||
bitmapSerdeFactory,
|
||||
byteOrder
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
ColumnType simpleType = supplier.getLogicalType();
|
||||
ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType;
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls));
|
||||
builder.setFilterable(true);
|
||||
}
|
||||
}
|
||||
|
||||
public static class SerializerBuilder
|
||||
{
|
||||
private ColumnType logicalType;
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
|||
import org.apache.druid.collections.spatial.ImmutableRTree;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.column.BitmapColumnIndex;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedValueIndex;
|
||||
|
@ -39,17 +40,16 @@ import org.apache.druid.segment.column.SimpleImmutableBitmapIndex;
|
|||
import org.apache.druid.segment.column.SpatialIndex;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier
|
||||
public class StringUtf8ColumnIndexSupplier<TIndexed extends Indexed<ByteBuffer>> implements ColumnIndexSupplier
|
||||
{
|
||||
private final BitmapFactory bitmapFactory;
|
||||
private final Supplier<FrontCodedIndexed> utf8Dictionary;
|
||||
private final Supplier<TIndexed> utf8Dictionary;
|
||||
|
||||
@Nullable
|
||||
private final GenericIndexed<ImmutableBitmap> bitmaps;
|
||||
|
@ -57,17 +57,34 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier
|
|||
@Nullable
|
||||
private final ImmutableRTree indexedTree;
|
||||
|
||||
public StringFrontCodedColumnIndexSupplier(
|
||||
private final ColumnConfig columnConfig;
|
||||
private final int numRows;
|
||||
|
||||
public StringUtf8ColumnIndexSupplier(
|
||||
BitmapFactory bitmapFactory,
|
||||
Supplier<FrontCodedIndexed> utf8Dictionary,
|
||||
Supplier<TIndexed> utf8Dictionary,
|
||||
@Nullable GenericIndexed<ImmutableBitmap> bitmaps,
|
||||
@Nullable ImmutableRTree indexedTree
|
||||
)
|
||||
{
|
||||
this(bitmapFactory, utf8Dictionary, bitmaps, indexedTree, ColumnConfig.ALWAYS_USE_INDEXES, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
public StringUtf8ColumnIndexSupplier(
|
||||
BitmapFactory bitmapFactory,
|
||||
Supplier<TIndexed> utf8Dictionary,
|
||||
@Nullable GenericIndexed<ImmutableBitmap> bitmaps,
|
||||
@Nullable ImmutableRTree indexedTree,
|
||||
@Nullable ColumnConfig columnConfig,
|
||||
int numRows
|
||||
)
|
||||
{
|
||||
this.bitmapFactory = bitmapFactory;
|
||||
this.bitmaps = bitmaps;
|
||||
this.utf8Dictionary = utf8Dictionary;
|
||||
this.indexedTree = indexedTree;
|
||||
this.columnConfig = columnConfig;
|
||||
this.numRows = numRows;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -90,7 +107,8 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier
|
|||
final BitmapColumnIndex nullIndex;
|
||||
final ByteBuffer firstValue = dict.get(0);
|
||||
if (NullHandling.isNullOrEquivalent(firstValue)) {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(singleThreadedBitmaps.get(0));
|
||||
ImmutableBitmap bitmap = singleThreadedBitmaps.get(0);
|
||||
nullIndex = new SimpleImmutableBitmapIndex(bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap);
|
||||
} else {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
}
|
||||
|
@ -105,14 +123,18 @@ public class StringFrontCodedColumnIndexSupplier implements ColumnIndexSupplier
|
|||
return (T) new IndexedStringDruidPredicateIndex<>(
|
||||
bitmapFactory,
|
||||
new StringEncodingStrategies.Utf8ToStringIndexed(dict),
|
||||
singleThreadedBitmaps
|
||||
singleThreadedBitmaps,
|
||||
columnConfig,
|
||||
numRows
|
||||
);
|
||||
} else if (clazz.equals(LexicographicalRangeIndex.class)) {
|
||||
return (T) new IndexedUtf8LexicographicalRangeIndex<>(
|
||||
bitmapFactory,
|
||||
dict,
|
||||
singleThreadedBitmaps,
|
||||
dict.get(0) == null
|
||||
dict.get(0) == null,
|
||||
columnConfig,
|
||||
numRows
|
||||
);
|
||||
} else if (clazz.equals(DictionaryEncodedStringValueIndex.class)
|
||||
|| clazz.equals(DictionaryEncodedValueIndex.class)) {
|
|
@ -22,26 +22,25 @@ package org.apache.druid.segment.serde;
|
|||
import com.google.common.base.Supplier;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.ColumnarMultiInts;
|
||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* {@link DictionaryEncodedColumnSupplier} but for columns using a {@link StringUtf8DictionaryEncodedColumn}
|
||||
* instead of the traditional {@link StringDictionaryEncodedColumn}
|
||||
* Supplier for {@link StringUtf8DictionaryEncodedColumn}
|
||||
*/
|
||||
public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier<DictionaryEncodedColumn<?>>
|
||||
public class StringUtf8DictionaryEncodedColumnSupplier<TIndexed extends Indexed<ByteBuffer>> implements Supplier<DictionaryEncodedColumn<?>>
|
||||
{
|
||||
private final Supplier<FrontCodedIndexed> utf8Dictionary;
|
||||
private final Supplier<TIndexed> utf8Dictionary;
|
||||
private final @Nullable Supplier<ColumnarInts> singleValuedColumn;
|
||||
private final @Nullable Supplier<ColumnarMultiInts> multiValuedColumn;
|
||||
|
||||
public StringFrontCodedDictionaryEncodedColumnSupplier(
|
||||
Supplier<FrontCodedIndexed> utf8Dictionary,
|
||||
public StringUtf8DictionaryEncodedColumnSupplier(
|
||||
Supplier<TIndexed> utf8Dictionary,
|
||||
@Nullable Supplier<ColumnarInts> singleValuedColumn,
|
||||
@Nullable Supplier<ColumnarMultiInts> multiValuedColumn
|
||||
)
|
||||
|
@ -54,7 +53,7 @@ public class StringFrontCodedDictionaryEncodedColumnSupplier implements Supplier
|
|||
@Override
|
||||
public DictionaryEncodedColumn<?> get()
|
||||
{
|
||||
final FrontCodedIndexed suppliedUtf8Dictionary = utf8Dictionary.get();
|
||||
final TIndexed suppliedUtf8Dictionary = utf8Dictionary.get();
|
||||
|
||||
if (NullHandling.mustCombineNullAndEmptyInDictionary(suppliedUtf8Dictionary)) {
|
||||
return new StringUtf8DictionaryEncodedColumn(
|
|
@ -97,7 +97,6 @@ public class DruidProcessingConfigTest
|
|||
Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount());
|
||||
Assert.assertEquals(NUM_PROCESSORS - 1, config.getNumThreads());
|
||||
Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers());
|
||||
Assert.assertEquals(0, config.columnCacheSizeBytes());
|
||||
Assert.assertTrue(config.isFifo());
|
||||
Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir());
|
||||
Assert.assertEquals(BUFFER_SIZE, config.intermediateComputeSizeBytes());
|
||||
|
@ -112,7 +111,6 @@ public class DruidProcessingConfigTest
|
|||
Assert.assertEquals(Integer.MAX_VALUE, config.poolCacheMaxCount());
|
||||
Assert.assertTrue(config.getNumThreads() == 1);
|
||||
Assert.assertEquals(Math.max(2, config.getNumThreads() / 4), config.getNumMergeBuffers());
|
||||
Assert.assertEquals(0, config.columnCacheSizeBytes());
|
||||
Assert.assertTrue(config.isFifo());
|
||||
Assert.assertEquals(System.getProperty("java.io.tmpdir"), config.getTmpDir());
|
||||
Assert.assertEquals(BUFFER_SIZE, config.intermediateComputeSizeBytes());
|
||||
|
@ -138,7 +136,6 @@ public class DruidProcessingConfigTest
|
|||
props.setProperty("druid.processing.buffer.sizeBytes", "1");
|
||||
props.setProperty("druid.processing.buffer.poolCacheMaxCount", "1");
|
||||
props.setProperty("druid.processing.numThreads", "256");
|
||||
props.setProperty("druid.processing.columnCache.sizeBytes", "1");
|
||||
props.setProperty("druid.processing.fifo", "false");
|
||||
props.setProperty("druid.processing.tmpDir", "/test/path");
|
||||
|
||||
|
@ -156,7 +153,6 @@ public class DruidProcessingConfigTest
|
|||
Assert.assertEquals(1, config.poolCacheMaxCount());
|
||||
Assert.assertEquals(256, config.getNumThreads());
|
||||
Assert.assertEquals(64, config.getNumMergeBuffers());
|
||||
Assert.assertEquals(1, config.columnCacheSizeBytes());
|
||||
Assert.assertFalse(config.isFifo());
|
||||
Assert.assertEquals("/test/path", config.getTmpDir());
|
||||
Assert.assertEquals(0, config.getNumInitalBuffersForIntermediatePool());
|
||||
|
|
|
@ -168,11 +168,6 @@ public class AggregationTestHelper implements Closeable
|
|||
mapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
@ -208,11 +203,6 @@ public class AggregationTestHelper implements Closeable
|
|||
mapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
@ -260,11 +250,6 @@ public class AggregationTestHelper implements Closeable
|
|||
mapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
@ -304,11 +289,6 @@ public class AggregationTestHelper implements Closeable
|
|||
mapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
|
|
@ -119,11 +119,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -128,11 +128,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -113,11 +113,6 @@ public class GroupByMultiSegmentTest
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -74,6 +74,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
|
||||
|
@ -116,10 +117,7 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest
|
|||
ExprMacroTable.nil()
|
||||
)
|
||||
);
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
}
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ import org.apache.druid.segment.column.ColumnBuilder;
|
|||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.ListIndexed;
|
||||
import org.apache.druid.segment.data.ObjectStrategy;
|
||||
|
@ -434,7 +434,7 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
|||
ColumnHolder holder = EasyMock.createMock(ColumnHolder.class);
|
||||
EasyMock.expect(mockIndex.getColumnHolder("x")).andReturn(holder).atLeastOnce();
|
||||
|
||||
StringDictionaryEncodedColumn dictionaryEncodedColumn = EasyMock.createMock(StringDictionaryEncodedColumn.class);
|
||||
StringUtf8DictionaryEncodedColumn dictionaryEncodedColumn = EasyMock.createMock(StringUtf8DictionaryEncodedColumn.class);
|
||||
EasyMock.expect(holder.getColumn()).andReturn(dictionaryEncodedColumn).atLeastOnce();
|
||||
|
||||
dictionaryEncodedColumn.close();
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.druid.segment.column.ColumnHolder;
|
|||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -62,7 +62,7 @@ public class ColumnSelectorColumnIndexSelectorTest
|
|||
|
||||
ColumnHolder holder = EasyMock.createMock(ColumnHolder.class);
|
||||
EasyMock.expect(index.getColumnHolder(STRING_DICTIONARY_COLUMN_NAME)).andReturn(holder).anyTimes();
|
||||
StringDictionaryEncodedColumn stringColumn = EasyMock.createMock(StringDictionaryEncodedColumn.class);
|
||||
StringUtf8DictionaryEncodedColumn stringColumn = EasyMock.createMock(StringUtf8DictionaryEncodedColumn.class);
|
||||
EasyMock.expect(holder.getCapabilities()).andReturn(
|
||||
ColumnCapabilitiesImpl.createDefault()
|
||||
.setType(ColumnType.STRING)
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.jackson.SegmentizerModule;
|
|||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
|
@ -58,7 +59,7 @@ public class CustomSegmentizerFactoryTest extends InitializedNullHandlingTest
|
|||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.registerModule(new SegmentizerModule());
|
||||
mapper.registerSubtypes(new NamedType(CustomSegmentizerFactory.class, "customSegmentFactory"));
|
||||
final IndexIO indexIO = new IndexIO(mapper, () -> 0);
|
||||
final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
|
|
|
@ -98,7 +98,7 @@ public class IndexBuilder
|
|||
|
||||
public static IndexBuilder create()
|
||||
{
|
||||
return new IndexBuilder(TestHelper.JSON_MAPPER, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
return new IndexBuilder(TestHelper.JSON_MAPPER, ColumnConfig.ALWAYS_USE_INDEXES);
|
||||
}
|
||||
|
||||
public static IndexBuilder create(ColumnConfig columnConfig)
|
||||
|
@ -108,7 +108,7 @@ public class IndexBuilder
|
|||
|
||||
public static IndexBuilder create(ObjectMapper jsonMapper)
|
||||
{
|
||||
return new IndexBuilder(jsonMapper, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
return new IndexBuilder(jsonMapper, ColumnConfig.ALWAYS_USE_INDEXES);
|
||||
}
|
||||
|
||||
public static IndexBuilder create(ObjectMapper jsonMapper, ColumnConfig columnConfig)
|
||||
|
|
|
@ -118,7 +118,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling
|
|||
@Test
|
||||
public void testV9LoaderThatIgnoresmptyColumns() throws IOException
|
||||
{
|
||||
QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG).load(
|
||||
QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(ColumnConfig.ALWAYS_USE_INDEXES).load(
|
||||
segmentDir,
|
||||
TestHelper.makeJsonMapper(),
|
||||
false,
|
||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
|||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.BitmapValues;
|
||||
|
@ -548,12 +548,12 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest
|
|||
DictionaryEncodedColumn encodedColumn = (DictionaryEncodedColumn) index.getColumnHolder("dim2").getColumn();
|
||||
Object obj;
|
||||
if (encodedColumn.hasMultipleValues()) {
|
||||
Field field = StringDictionaryEncodedColumn.class.getDeclaredField("multiValueColumn");
|
||||
Field field = StringUtf8DictionaryEncodedColumn.class.getDeclaredField("multiValueColumn");
|
||||
field.setAccessible(true);
|
||||
|
||||
obj = field.get(encodedColumn);
|
||||
} else {
|
||||
Field field = StringDictionaryEncodedColumn.class.getDeclaredField("column");
|
||||
Field field = StringUtf8DictionaryEncodedColumn.class.getDeclaredField("column");
|
||||
field.setAccessible(true);
|
||||
|
||||
obj = field.get(encodedColumn);
|
||||
|
|
|
@ -62,26 +62,6 @@ import java.util.stream.IntStream;
|
|||
public class TestHelper
|
||||
{
|
||||
public static final ObjectMapper JSON_MAPPER = makeJsonMapper();
|
||||
public static final ColumnConfig NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValuePredicateIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
};
|
||||
|
||||
public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
|
||||
{
|
||||
|
@ -90,7 +70,7 @@ public class TestHelper
|
|||
|
||||
public static IndexIO getTestIndexIO()
|
||||
{
|
||||
return getTestIndexIO(NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
return getTestIndexIO(ColumnConfig.ALWAYS_USE_INDEXES);
|
||||
}
|
||||
|
||||
public static IndexIO getTestIndexIO(ColumnConfig columnConfig)
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
|||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.segment.IndexIO.V9IndexLoader;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
|
@ -119,7 +120,7 @@ public class V9IndexLoaderTest extends InitializedNullHandlingTest
|
|||
{
|
||||
private CannotDeserializeCountColumnV9IndexLoader()
|
||||
{
|
||||
super(() -> 0);
|
||||
super(ColumnConfig.DEFAULT);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory;
|
|||
import org.apache.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -116,13 +116,12 @@ public class ExtractionDimFilterTest extends InitializedNullHandlingTest
|
|||
public ColumnIndexSupplier getIndexSupplier(String column)
|
||||
{
|
||||
if ("foo".equals(column)) {
|
||||
return new DictionaryEncodedStringIndexSupplier(
|
||||
return new StringUtf8ColumnIndexSupplier<>(
|
||||
factory,
|
||||
GenericIndexed.fromIterable(Collections.singletonList("foo1"), GenericIndexed.STRING_STRATEGY),
|
||||
GenericIndexed.fromIterable(
|
||||
Collections.singletonList(ByteBuffer.wrap(StringUtils.toUtf8("foo1"))),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
GenericIndexed.fromIterable(Collections.singletonList(foo1BitMap), serdeFactory.getObjectStrategy()),
|
||||
null
|
||||
);
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.VSizeColumnarInts;
|
||||
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
||||
import org.apache.druid.segment.selector.TestColumnValueSelector;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -67,8 +67,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes
|
|||
public void testDimensionProcessorMultiValuedDimensionMatchingValue()
|
||||
{
|
||||
// Emulate multi-valued dimension
|
||||
final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY),
|
||||
final StringUtf8DictionaryEncodedColumnSupplier<?> columnSupplier = new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
GenericIndexed.fromIterable(
|
||||
ImmutableList.of(
|
||||
ByteBuffer.wrap(StringUtils.toUtf8("v1")),
|
||||
|
@ -76,10 +75,9 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes
|
|||
ByteBuffer.wrap(StringUtils.toUtf8("v3"))
|
||||
),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
null,
|
||||
() -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))),
|
||||
0
|
||||
() -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1})))
|
||||
);
|
||||
final ValueMatcher matcher = forSelector("v2")
|
||||
.makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true);
|
||||
|
@ -90,8 +88,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes
|
|||
public void testDimensionProcessorMultiValuedDimensionNotMatchingValue()
|
||||
{
|
||||
// Emulate multi-valued dimension
|
||||
final DictionaryEncodedColumnSupplier columnSupplier = new DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(ImmutableList.of("v1", "v2", "v3"), GenericIndexed.STRING_STRATEGY),
|
||||
final StringUtf8DictionaryEncodedColumnSupplier<?> columnSupplier = new StringUtf8DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(
|
||||
ImmutableList.of(
|
||||
ByteBuffer.wrap(StringUtils.toUtf8("v1")),
|
||||
|
@ -99,10 +96,9 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes
|
|||
ByteBuffer.wrap(StringUtils.toUtf8("v3"))
|
||||
),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
null,
|
||||
() -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1}))),
|
||||
0
|
||||
() -> VSizeColumnarMultiInts.fromIterable(ImmutableList.of(VSizeColumnarInts.fromArray(new int[]{1})))
|
||||
);
|
||||
final ValueMatcher matcher = forSelector("v3")
|
||||
.makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true);
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.druid.segment.SimpleAscendingOffset;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.VSizeColumnarInts;
|
||||
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8DictionaryEncodedColumnSupplier;
|
||||
import org.apache.druid.segment.vector.NilVectorSelector;
|
||||
import org.apache.druid.segment.vector.NoFilterVectorOffset;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -38,50 +38,44 @@ import java.nio.ByteBuffer;
|
|||
|
||||
public class ValueMatchersTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private DictionaryEncodedColumnSupplier supplierSingleConstant;
|
||||
private DictionaryEncodedColumnSupplier supplierSingle;
|
||||
private DictionaryEncodedColumnSupplier supplierMulti;
|
||||
private StringUtf8DictionaryEncodedColumnSupplier<?> supplierSingleConstant;
|
||||
private StringUtf8DictionaryEncodedColumnSupplier<?> supplierSingle;
|
||||
private StringUtf8DictionaryEncodedColumnSupplier<?> supplierMulti;
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
supplierSingleConstant = new DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY),
|
||||
supplierSingleConstant = new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
GenericIndexed.fromIterable(
|
||||
ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
() -> VSizeColumnarInts.fromArray(new int[]{0}),
|
||||
null,
|
||||
0
|
||||
null
|
||||
);
|
||||
supplierSingle = new DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(ImmutableList.of("value", "value2"), GenericIndexed.STRING_STRATEGY),
|
||||
supplierSingle = new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
GenericIndexed.fromIterable(
|
||||
ImmutableList.of(
|
||||
ByteBuffer.wrap(StringUtils.toUtf8("value")),
|
||||
ByteBuffer.wrap(StringUtils.toUtf8("value2"))
|
||||
),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
() -> VSizeColumnarInts.fromArray(new int[]{0, 0, 1, 0, 1}),
|
||||
null,
|
||||
0
|
||||
null
|
||||
);
|
||||
supplierMulti = new DictionaryEncodedColumnSupplier(
|
||||
GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY),
|
||||
supplierMulti = new StringUtf8DictionaryEncodedColumnSupplier<>(
|
||||
GenericIndexed.fromIterable(
|
||||
ImmutableList.of(ByteBuffer.wrap(StringUtils.toUtf8("value"))),
|
||||
GenericIndexed.UTF8_STRATEGY
|
||||
),
|
||||
)::singleThreaded,
|
||||
null,
|
||||
() -> VSizeColumnarMultiInts.fromIterable(
|
||||
ImmutableList.of(
|
||||
VSizeColumnarInts.fromArray(new int[]{0, 0}),
|
||||
VSizeColumnarInts.fromArray(new int[]{0})
|
||||
)
|
||||
),
|
||||
0
|
||||
)
|
||||
);
|
||||
}
|
||||
@Test
|
||||
|
|
|
@ -188,7 +188,7 @@ public class JoinTestHelper
|
|||
|
||||
public static IndexBuilder createFactIndexBuilder(final File tmpDir) throws IOException
|
||||
{
|
||||
return createFactIndexBuilder(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG, tmpDir, -1);
|
||||
return createFactIndexBuilder(ColumnConfig.ALWAYS_USE_INDEXES, tmpDir, -1);
|
||||
}
|
||||
|
||||
public static IndexBuilder createFactIndexBuilder(
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback;
|
|||
import org.apache.druid.segment.SimpleAscendingOffset;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.column.BaseColumn;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
|
||||
|
@ -102,7 +103,7 @@ public class BroadcastSegmentIndexedTableTest extends InitializedNullHandlingTes
|
|||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.registerModule(new SegmentizerModule());
|
||||
final IndexIO indexIO = new IndexIO(mapper, () -> 0);
|
||||
final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.segment.IndexSpec;
|
|||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.join.table.BroadcastSegmentIndexedTable;
|
||||
import org.apache.druid.segment.join.table.IndexedTable;
|
||||
|
@ -67,7 +68,7 @@ public class BroadcastJoinableMMappedQueryableSegmentizerFactoryTest extends Ini
|
|||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
mapper.registerModule(new SegmentizerModule());
|
||||
final IndexIO indexIO = new IndexIO(mapper, () -> 0);
|
||||
final IndexIO indexIO = new IndexIO(mapper, ColumnConfig.DEFAULT);
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||
|
|
|
@ -46,11 +46,6 @@ public class SegmentizerFactoryTest
|
|||
mapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 777;
|
||||
}
|
||||
}
|
||||
);
|
||||
mapper.setInjectableValues(
|
||||
|
|
|
@ -100,11 +100,6 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
|
||||
private static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
|
|
|
@ -66,12 +66,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
private static final int ROW_COUNT = 10;
|
||||
static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
|
@ -1470,12 +1464,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
|
|||
{
|
||||
ColumnConfig twentyPercent = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
|
|
|
@ -52,7 +52,7 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan
|
|||
@Test
|
||||
public void testStringColumnWithNullValueSetIndex() throws IOException
|
||||
{
|
||||
DictionaryEncodedStringIndexSupplier indexSupplier = makeStringWithNullsSupplier();
|
||||
StringUtf8ColumnIndexSupplier<?> indexSupplier = makeStringWithNullsSupplier();
|
||||
StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class);
|
||||
Assert.assertNotNull(valueSetIndex);
|
||||
|
||||
|
@ -102,7 +102,7 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan
|
|||
checkBitmap(bitmap);
|
||||
}
|
||||
|
||||
private DictionaryEncodedStringIndexSupplier makeStringWithNullsSupplier() throws IOException
|
||||
private StringUtf8ColumnIndexSupplier<?> makeStringWithNullsSupplier() throws IOException
|
||||
{
|
||||
ByteBuffer stringBuffer = ByteBuffer.allocate(1 << 12);
|
||||
ByteBuffer byteBuffer = ByteBuffer.allocate(1 << 12);
|
||||
|
@ -164,10 +164,9 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan
|
|||
writeToBuffer(bitmapsBuffer, bitmapWriter);
|
||||
|
||||
GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.read(bitmapsBuffer, roaringFactory.getObjectStrategy());
|
||||
return new DictionaryEncodedStringIndexSupplier(
|
||||
return new StringUtf8ColumnIndexSupplier<>(
|
||||
roaringFactory.getBitmapFactory(),
|
||||
GenericIndexed.read(stringBuffer, GenericIndexed.STRING_STRATEGY),
|
||||
GenericIndexed.read(byteBuffer, GenericIndexed.UTF8_STRATEGY),
|
||||
GenericIndexed.read(byteBuffer, GenericIndexed.UTF8_STRATEGY)::singleThreaded,
|
||||
bitmaps,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -49,9 +49,7 @@ import java.nio.ByteBuffer;
|
|||
|
||||
public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final String COLUMN_NAME = "missing";
|
||||
private static final ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0);
|
||||
private static final ColumnConfig A_CONFIG = () -> 0;
|
||||
|
||||
@Test
|
||||
public void testSerde() throws JsonProcessingException
|
||||
|
@ -68,7 +66,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
final ColumnCapabilities columnCapabilities = builder.build().getCapabilities();
|
||||
Assert.assertTrue(Types.is(columnCapabilities, ValueType.DOUBLE));
|
||||
Assert.assertTrue(columnCapabilities.hasNulls().isTrue());
|
||||
|
@ -85,7 +83,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
|
@ -107,7 +105,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
|
@ -135,7 +133,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
|
@ -154,7 +152,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
|
@ -175,7 +173,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
|
@ -199,7 +197,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest
|
|||
{
|
||||
final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance());
|
||||
final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, A_CONFIG);
|
||||
partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT);
|
||||
ColumnHolder holder = builder.build();
|
||||
Assert.assertNull(holder.getIndexSupplier());
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
|||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
|
||||
|
@ -95,7 +95,7 @@ public class DummyStringVirtualColumn implements VirtualColumn
|
|||
return DimensionSelector.constant(null);
|
||||
}
|
||||
|
||||
StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||
StringUtf8DictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||
|
||||
DimensionSelector baseDimensionSelector = stringCol.makeDimensionSelector(
|
||||
offset,
|
||||
|
@ -145,7 +145,7 @@ public class DummyStringVirtualColumn implements VirtualColumn
|
|||
return NilColumnValueSelector.instance();
|
||||
}
|
||||
|
||||
StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||
StringUtf8DictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||
return stringCol.makeColumnValueSelector(offset);
|
||||
} else {
|
||||
return null;
|
||||
|
@ -222,13 +222,13 @@ public class DummyStringVirtualColumn implements VirtualColumn
|
|||
return new byte[0];
|
||||
}
|
||||
|
||||
private StringDictionaryEncodedColumn toStringDictionaryEncodedColumn(BaseColumn column)
|
||||
private StringUtf8DictionaryEncodedColumn toStringDictionaryEncodedColumn(BaseColumn column)
|
||||
{
|
||||
if (!(column instanceof StringDictionaryEncodedColumn)) {
|
||||
if (!(column instanceof StringUtf8DictionaryEncodedColumn)) {
|
||||
throw new IAE("I can only work with StringDictionaryEncodedColumn");
|
||||
}
|
||||
|
||||
return (StringDictionaryEncodedColumn) column;
|
||||
return (StringUtf8DictionaryEncodedColumn) column;
|
||||
}
|
||||
|
||||
private DimensionSelector disableValueMatchers(DimensionSelector base)
|
||||
|
|
|
@ -54,7 +54,7 @@ public class LocalDataStorageDruidModuleTest
|
|||
new LocalDataStorageDruidModule(),
|
||||
(Module) binder -> {
|
||||
binder.bind(new TypeLiteral<List<StorageLocation>>(){}).toInstance(ImmutableList.of());
|
||||
binder.bind(ColumnConfig.class).toInstance(() -> 0);
|
||||
binder.bind(ColumnConfig.class).toInstance(ColumnConfig.DEFAULT);
|
||||
binder.bind(StorageLocationSelectorStrategy.class)
|
||||
.toInstance(new RandomStorageLocationSelectorStrategy(ImmutableList.of()));
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMerger;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.AppendableIndexSpec;
|
||||
import org.apache.druid.segment.incremental.ParseExceptionHandler;
|
||||
import org.apache.druid.segment.incremental.RowIngestionMeters;
|
||||
|
@ -183,10 +184,7 @@ public class ClosedSegmensSinksBatchAppenderatorTester implements AutoCloseable
|
|||
);
|
||||
metrics = new FireDepartmentMetrics();
|
||||
|
||||
IndexIO indexIO = new IndexIO(
|
||||
objectMapper,
|
||||
() -> 0
|
||||
);
|
||||
IndexIO indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
IndexMergerV9 indexMerger = new IndexMergerV9(
|
||||
objectMapper,
|
||||
indexIO,
|
||||
|
|
|
@ -91,11 +91,6 @@ public class DefaultOfflineAppenderatorFactoryTest
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 25 * 1024 * 1024;
|
||||
}
|
||||
}
|
||||
);
|
||||
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
||||
|
|
|
@ -159,11 +159,6 @@ public class OpenAndClosedSegmentsAppenderatorTester implements AutoCloseable
|
|||
objectMapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -157,11 +157,6 @@ public class StreamAppenderatorTester implements AutoCloseable
|
|||
objectMapper,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.segment.IndexMergerV9;
|
|||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
|
||||
import org.apache.druid.segment.TestIndex;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.join.BroadcastTableJoinableFactory;
|
||||
import org.apache.druid.segment.join.JoinConditionAnalysis;
|
||||
|
@ -116,7 +117,7 @@ public class SegmentManagerBroadcastJoinIndexedTableTest extends InitializedNull
|
|||
new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local"))
|
||||
);
|
||||
|
||||
indexIO = new IndexIO(objectMapper, () -> 0);
|
||||
indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
objectMapper.setInjectableValues(
|
||||
new InjectableValues.Std().addValue(LocalDataSegmentPuller.class, segmentPuller)
|
||||
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
|
||||
import org.apache.druid.segment.loading.LocalLoadSpec;
|
||||
|
@ -97,7 +98,7 @@ public class SegmentManagerThreadSafetyTest
|
|||
new SimpleModule().registerSubtypes(new NamedType(LocalLoadSpec.class, "local"), new NamedType(TestSegmentizerFactory.class, "test"))
|
||||
)
|
||||
.setInjectableValues(new Std().addValue(LocalDataSegmentPuller.class, segmentPuller));
|
||||
indexIO = new IndexIO(objectMapper, () -> 0);
|
||||
indexIO = new IndexIO(objectMapper, ColumnConfig.DEFAULT);
|
||||
segmentCacheDir = temporaryFolder.newFolder();
|
||||
segmentDeepStorageDir = temporaryFolder.newFolder();
|
||||
segmentCacheManager = new SegmentLocalCacheManager(
|
||||
|
|
|
@ -761,11 +761,6 @@ public class DumpSegment extends GuiceRunnable
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 25 * 1024 * 1024;
|
||||
}
|
||||
}
|
||||
);
|
||||
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
||||
|
|
|
@ -114,11 +114,6 @@ public class ValidateSegments extends GuiceRunnable
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 25 * 1024 * 1024;
|
||||
}
|
||||
}
|
||||
);
|
||||
binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class);
|
||||
|
|
Loading…
Reference in New Issue