IncrementalIndex
IncrementalIndex类中有两个重要的成员,分别是metricDescs
和dimensionDescs
:
private final Map<String, MetricDesc> metricDescs;
private final Map<String, DimensionDesc> dimensionDescs;
metricDescs和dimensionDescs在IncrementalIndex的构造函数中被初始化。
MetricDesc
每个MetricDesc中有几个重要的成员:
private final int index; // metric序号
private final String name; // metric名字
private final String type: // metric类型
private final ColumnCapabilitiesImpl capabilities // metric能力
MetricDesc的构造函数:
public MetricDesc(int index, AggregatorFactory factory)
{
this.index = index;
this.name = factory.getName();
String typeInfo = factory.getTypeName();
this.capabilities = new ColumnCapabilitiesImpl();
if ("float".equalsIgnoreCase(typeInfo)) {
capabilities.setType(ValueType.FLOAT);
this.type = typeInfo;
} else if ("long".equalsIgnoreCase(typeInfo)) {
capabilities.setType(ValueType.LONG);
this.type = typeInfo;
} else if ("double".equalsIgnoreCase(typeInfo)) {
capabilities.setType(ValueType.DOUBLE);
this.type = typeInfo;
} else {
capabilities.setType(ValueType.COMPLEX);
this.type = ComplexMetrics.getSerdeForType(typeInfo).getTypeName();
}
}
每个AggregatorFactory的实例都有一个名字,通过getTypeName()方法获取。比如CountAggregatorFactory的getTypeName()方法返回"long",HyperUniquesAggregatorFactory的getTypeName()方法返回"hyperUnique"。
如果对AggregatorFactory调用getTypeName()返回的名字不是"float"、"long"、"double"之一,name这个AggregatorFactory的类型是复杂类型,比如HyperUniquesAggregatorFactory。
在IncrementalIndex中通过如下代码构造每个metric的MetricDesc和MetricDescs:
for (AggregatorFactory metric : metrics) {
MetricDesc metricDesc = new MetricDesc(metricDesc.size(), metric);
metricDescs.put(metricDesc.getName(), metricDesc);
}
DimensionDesc
每个DimensionDesc中有几个重要成员:
private final int index; // dimension序号
private final String name; // dimnesion名字
private final ColumnCapabilitiesImpl capabilities // dimension能力
private final DimensionHandler handler;
private final DimensionIndexer indexer;
DimensionHandler
DimensionHandler对象封装了特定于某一个dimension的索引,列合并/创建,以及查询操作。这些操作由通过DimensionHandler方法创建的对象(DimensionIndexer通过makeIndexer创建,DimensionMerger通过makeMerger创建,DimensionColumnReader)handle。每个DimensionHandler对象都特定于一个单独的dimension。
DimensionIndexer
每个dimension对应一个DimensionIndexer,用于在内存中处理注入的行。
ColumnCapabilitiesImpl
在IncrementalIndex的构造函数中定义了每个dimension的capalibities:
ColumnCapabilitiesImpl capabilities = makeCapabilitiesFromValueType(type);
private ColumnCapabilitiesImpl makeCapabilitiesFromValueType(ValueType type)
{
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
capabilities.setDictionaryEncoded(type == ValueType.STRING);
capabilities.setHasBitmapIndexes(type == ValueType.STRING);
capabilities.setType(type);
return capabilities
}
可见只有string类型的dimension才支持字典编码
和位图索引
。
设置是否支持位图索引:
capabilities.setHasBitmapIndexes(dimSchema.hasBitmapIndex());
只有string类型的dimension才支持字典编码。
根据不同的capabilities生成不同的DimensionHandler:
DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(
dimName,
capabilities,
dimSchema.getMultiValueHanding()
);
public static DimensionHandler getHandlerFromCapabilities(
String dimensionName,
ColumnCapabilities capabilities,
MultiValueHandling multiValueHandling
)
{
if (capabilities == null) {
return new StringDimensionHandler(dimensionName, multiValueHandling, true);
}
multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHanding;
if (capabilities.getType() == ValueType.STRING) {
if (!capabilities.isDictionaryEncoded()) {
throw new IAE("String column must have dictionary encoding.");
}
}
if (capabilities.getType() == ValueType.LONG) {
return new LongDimensionHandler(dimensionName);
}
if (capabilities.getType() == ValueType.FLOAT) {
return new FloatDimensionHandler(dimensionName);
}
if (capabilities.getType() == ValueType.DOUBLE) {
return new DoubleDimensionHandler(dimensionName);
}
// Return a StringDimensionHandler by default (null columns will be treated as String typed)
return new StringDimensionHandler(dimensionName, multiValueHandling, true);
}
向IncrementalIndex中写入一行数据
解析出的一行数据(这里认为一行数据的实际类型为MapBasedInputRow
)最终会调用IncrementalIndex的toIncrementalIndexRow(InputRow row)方法向IncrementalIndex中加入一条数据。
对于一行数据中的某一列的值,会调用:
Object dimsKey = null;
dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent(
row.getRow(dimension),
true
);
这里row.getRow(dimension)就是解析出特定的一行数据中,dimension名字对应的的值。
indexer是在DimensionHandler中类型为DimensionIndexer的成员。这里我们只考虑String类型的Dimension,因此这里indexer的实例类型是StringDimensionIndexer。
下面我们来看一下StringDimensionIndexer的processRowValsToUnsortedEncodedKeyComponent方法。
在StringDimensionIndexer中有一个重要的内部类DimensionDictionary
。其中有两个重要的成员:
private final Object2IntMap<String> valueToId = new Object2IntOpenHashMap<>();
private final List<String> idToValue = new ArrayList<>();
valueToId
存储了值到id的对应关系。idToValue
存储了id到值的对应关系,id就是List的下标。
processRowValsToUnsortedEncodedKeyComponent方法
在processRowValsToUnsortedEncodedKeyComponent方法中:
如果传入的dimension的值是null的话,则会调用DimensionDictionary的getId方法:
final int nullId = dimLookup.getId(null);
这里如果是第一次遇到null值,则返回-1。
然后返回dimension的值编码后的值(这里是index、序号,比如第1行和第50行的数据可能返回的都是7):
encodedDimensionValues = nullId == ABSENT_VALUE_ID ? new int[]{dimLookup.add(null)} : new int[]{nullId};
如果nullId是-1(首次遇到特定dimension值为null的情况),这时调用DimensionDictionary的add方法将null值加入idToValue
这个List中,设置idForNull为idToValue.size()并返回这个的id(idForNull是null值在idToValue中的下标或索引);如果nullId不为-1,则说明不是首次遇到特定dimension值为null的情况,这时直接返回nullId(也是idForNull的值)。
传入的dimension的值是个List,这种情况我们先不做分析,只考虑单值的情况。
传入的dimension的值为单值,则调用DimensionDictionary的add方法:
encodedDimensionValues = new int[]{dimLoojup.add(emptyToNullIfNeeded(dimValues))};
在add方法中,首先看valueToId
中有没有这个值,如果有的话,直接返回这个值对应的id,如果没有,则调用idToValue.size()设置这个值在idToValue中的索引,然后将这个值和对应的索引写入valueToId,并把这个值加入到idToValue中:
final int index = idToValue.size();
valueToId.put(originalValue, index);
idToValue.add(originalValue);
然后设置特定dimension当前的minValue和maxValue,最后返回index:
minValue = minValue == null || minValue.compareTo(originalValue) > 0 ? originalValue : minValue;
maxValue = maxValue == null || maxValue.compareTo(originalValue) < 0 ? originalValue : maxValue;
return index;
processRowValsToUnsortedEncodedKeyComponent最终返回的是当前行的特定列的值在valueToId中的id,也就是在idToValue中的索引。
需要记住的是,每个dimension对应一个DimensionDesc,每个DimensionDesc中有一个DimensionIndexer,每个DimensionIndexer中 有一个DimensionDictionary,每个DimensionDictionary中有一个valueToId和一个IdToValue。
这里给个例子,如果当前有10行数据,它们的维度dim列的值为'a','b','c','b','d','e','a','a','b','f',那么在这10列数据都调用processRowValsToUnsortedEncodedKeyComponent之后,idToValue中的值为[a, b, c, d, e, f],valueToId中的值为{'a'->0, 'b'->1, 'c'->2, 'd'->3, 'e'->4, 'f'->5},processRowValsToUnsortedEncodedKeyComponent返回值分别为{0},{1},{2},{1},{3},{4},{0},{0},{1},{5}。
回到toIncrementalIndexRow方法,对这一行数据的每个dimension都调用processRowValsToUnsortedEncodedKeyComponent返回一个index数组(单值的话数组中只有一个元素),单后设置dims的值:
Object[] dims;
dims[desc.getIndex()] = dimsKey;
然后构造一个incrementalIndexRow实例:
IncrementalIndexRow incrementalIndexRow = IncrementalIndexRow.createTimeAndDimswithDimsKeySize(
Math.max(truncated, minTimestamp),
dims,
dimensionDescList,
dimsKeySize
);
其中truncated是根据注入spec中的granularitySpec中指定的queryGranularity的值截断的时间戳。例如一行数据中的time字段的值为2019-08-14T17:55:34,如果queryGranularity是NONE,则不截断,如果为minute,则截断为2019-08-14T17:55:00,如果为day,则截断为2019-08-14T00:00:00。
minTimestamp是当前segment起始的timestamp。
最后返回一个IncrementIndexRowResult实例:
return new IncrementalIndexRowResult(incrementalIndexRow, parseExceptionMessages);
FactsHolder
每个IncrementalIndex都有一个FactsHolder类型的成员,这里我们假设在注入的spec中的granularitySpec中指定了rollup为true(默认就为true),则这里的FactsHolder实际类型为RollupFactsHolder。
上面生成了IncrementalIndexRowResult实例之后,调用addToFacts:
final AddToFactsResult addToFactsResult = addToFacts(
row,
incrementalIndexResult.getIncrementalIndexRow(),
in,
rowSupplier,
skipMaxRowsInMemoryCheck
);
在OnheapIncrementalIndex中,有
protected AddToFacts(
InputRow row,
IncrementalIndexRow key,
ThreadLocal<InputRow> rowContainer,
Supplier<InputRow> rowSupplier,
boolean skipMaxRowsInMemoryCheck
)
从AggregatorFactory产出Aggregator
在addToFacts中,首先对metrics(类型是AggregatorFactory数组)产出Aggregator数组:
Aggregator[] aggs;
aggs = new Aggregator[metrics.length];
factorizeAggs(metrics, aggs, rowContainer, row);
对每个AggregatorFactory调用factorize产出Aggregator。比如对CountAggregatorFactory
产出CountAggregator
,对HyperUniquesAggregatorFactory
产出HyperUniquesAggregator
。
对产出的每个Aggregator调用aggregate方法计算当前的metric值。对于CountAggregator,它的aggregate方法定义如下:
public void aggregate()
{
++count;
}
很简单也就对它的count成员加1;
对于HyperUniquesAggregator,它的aggregate方法如下:
public void aggregate()
{
Object object = selector.getObject();
if (object == null) {
return;
}
if (collector == null) {
collector = HyperLogLogCollector.makeLatestCollector()l
}
collector.fold((HyperLogLogCollector) object);
}
这里首先调用selector的getObject()方法,selector的类型实际上是IncrementalIndex中的makeColumnSelectorFactory方法返回的IncrementalIndexInputRowColumnSelectorFactory实例中makeColumnValueSelector方法返回的ColumnValueSelector实例。它的getObject()方法调用的是ComplexMetricExtractor的extractValue方法:
public Object getObject() { return extract.extractValue(in.get(), column, agg); }
在HyperUniquesSerde的getExtractor返回的ComplexMetricExtractor实例的extractValue方法中,实际返回的事?一个HyperLogLogCollector实例:
public HyperLogLogCollector extractValue(InputRow inputRow, String metricName) { Object rawValue = inputRow.getRow(metricName); if (rawValue instanceOf HyperLogLogCollector) { return (HyperLogLogCollector) rawValue; } else { HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); List<String> dimValues = inputRow.getDimension(metricName); if (dimValues == null) { return collector; } for (String dimensionValue : dimValues) { collector.add(hyperLogLogHash.hash(dimensionValue)); } return collector; } }
调用HyperLogLogCollector的fold方法,也就是将一行数据的HyperUniquesAggregator的HyperLogLogCollector合并到同一个HyperLogLogCollector对象上。
接着获取当前的rowIndex:
final int rowIndex = indexIncrement.getAndIncrement();
private final AtomicInteger indexIncrement = new AtomicInteger(0);
这里的rowIndex代表的值是注入的实际行数,而不是最终在segment中存储的行数。
最后将调用了aggregate方法之后的所有metrics对应的Aggregator设置到IncrementalIndex的成员aggregators上:
concurrentSet(rowIndex, aggs);
protected void concurrentSet(int offset, Aggregator[] value)
{
return aggregators.put(offset, value);
}
private final ConcurrentHashMap<Integer, Aggregator[]> aggregators = new ConcurrentMap<>();
也就是aggregators中每条记录就是注入的一行数据中的所有metric的Aggregator。
注入了多少行,就有多少个Aggregator数组。
FactsHolder的facts成员填充
facts成员在RollupFactsHolder中的声明如下:
private final ConcurrentMap<IncrementalIndexRow, IncrementalIndexRow> facts
接下来获取当前IncrementalIndexRow在FactsHolder的facts成员中的索引:
final int prev = facts.putIfAbsent(key, rowIndex);
public int putIfAbsent(IncrementalIndexRow key, int rowIndex)
{
key.setRowIndex(rowIndex);
IncrementalIndexRow prev = facts.putIfAbsent(key, key);
return prev == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : prev.getRowIndex();
}
可见如果当前IncrementalIndexRow(也可认为是当前行)在facts中不存在时返回-1,存在则返回它在facts中的索引。
为什么新的一行数据会在facts中存在呢?例如有两行数据:"2019-01-01T12:30:30,male,beijing"和'2019-01-01T21:24:10,male,beijing',如果queryGranularity为day,则在facts中这两行数据可以作为同一行数据存储。但是在aggregator中会有2组Aggregator数组。
如果putIfAbsent返回-1,则证明是新的一行数据(没有预聚合上),这是numEntries加1。*numEntries中保存的是segment中实际存储的行数*。
如果putIfAbsent没有返回-1,则证明预聚合生效了,也就是之前已经有相同的一行数据(timestamp根据queryGranularity做了截断)。这时先将和当前行相同的行(说行不准确,应该是IncrementalIndexRow)的自然索引也就是prev值对应的Aggregator数据取出来:
aggs = concurrentGet(prev);
protected Aggregator[] concurrentGet(int offset)
{
return aggregators.get(offset);
}
然后将当前行的每个Aggregator聚合到前一步取出的Aggregator上,这一步是真正实现了预聚合:
parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row);
最后从aggregators中删除当前行的Aggregator数组(因为已经预聚合完毕了,这一行的Aggregator就是多余的,需要删掉):
concurrentRemove(rowIndex);
protected void concurrentRemove(int offset)
{
aggregators.remove(offset);
}
最后从addToFacts方法返回一个AddToFactsResult实例:
return new AddToFactsResult(numEntries.get(), siteInBytes.get(), parseExceptionMessages);
这里的numEntries是当前IncrementalIndex中实际存在的行数。
最后在addToFacts的调用方法,也就是IncrementalIndex的add方法中,返回一个IncrementalIndexAddResult实例:
return new IncrementalIndexAddResult(
addToFactsResult.getRowCount(),
addToFactsResult.getBytesInMemory(),
parseException
);
生成并持久化一个Segment
在一个IncrementalIndex中写入足够多的数据之后,就需要将这些索引数据写入一个Segment。如果当前的IncrementalIndex中的行数大于在tuningConfig
中指定的maxRowsPerSegment(默认为5,000,000行)就需要将当前索引中的数据写入一个新的Segment。
在IndexMergerV9的persist方法中,会调用merge方法:
return merge(
Collections.singletonList(
new IncrementalIndexAdapter(
dataInterval,
index,
indexSpec.getBitmapSerdeFactory().getBitmapFactory()
),
false, // rollup, no neet to rollup again
index.getMetricAggs(), // AggregatorFactory[]
outDir,
indexSpec,
progress,
segmentWriteOutMediumFactory
)
);
merge方法的签名如下:
private File merge(
List<IndexableAdapter> indexes,
final boolean rollup,
final AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
这里对于参数需要说明的是一下几点:
- indexes - 是IndexableAdapter的列表,上面在调用merge方法时传入的事IncrementalIndexAdapter的singleton list。生成的IncrementalIndexAdapter后面我会详细分析;
- metricAggs - 传入的是AggregatorFactory的数组(从注入spec的metricsSpec来);
- indexSpec - 来自在tuningConfig中指定的IndexSpec对象,包括bitmap字段(BitmapSerdeFactory,默认为ConciseBitmapSerdeFactory),dimensionCompression字段(CompressionStrategy,默认为LZ4),metriCompression字段(CompressionStrategy,默认为LZ4),longEncoding(CompressionFactory.LongEncodingStrategy,默认为LongEncodingStrategy.LONGS);
- segmentWriteOutMediumFactory - 默认为null。
传入merge方法的indexes参数
indexes参数的类型为List<IndexableAdapter>,从上面看到传入的实际类型是单个IncrementalIndexAdapter对象的列表。
上面初始化了IncrementalIndexAdapter:
new IncrementalIndexAdapter(
dataInterval,
index,
indexSpec.getBitmapSerdeFactory().getBitmapFactory()
)
IncrementalIndexAdapter的构造函数:
public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex<?> index, BitmapFactory bitmapFactory) {
this.dataInterval = dataInterval;
this.index = index;
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
accessors = dimensions
.stream()
.collect(Collectors.toMap(IncrementalIndex.DimensionDesc::getName, DimensionAccessor::new));
processRows(index, bitmapFactory, dimensions);
}
这里传入的BitmapFactory的实际类型默认为ConciseBitmapFactory。
IncrementalIndexAdapter的accessors成员声明为:
private final Map<String, DimensionAccessor> accessors;
也就是一个dimension对应一个DimensionAccessor。
DimensionAccessor的构造函数:
public DimensionAccessor(IncrementalIndex.DimensionDesc dimensionDesc)
{
this.dimensionDesc = dimensionDesc;
this.indexer = dimensionDesc.getIndexer();
if (dimensionDesc.getCapabilities().hasBitmapIndexes()) {
this.invertedIndexes = new MutableBitmap[indexer.getCardinality() + 1];
} else {
this.invertedIndexes = null;
}
}
private MutableBitmap[] invertedIndexes;
如果一个DimensionDesc中的CapabilitiesImpl的hasBitmapIndexes调用返回true(只有String类型的dimension才会有位图索引),则初始化invertedIndexes成员为一个MutableBitmap数组,数组大小为索引中的distinct value number。StringDimensionIndexer的getCardinality调用:
public int getCardinality()
{
return dimLookup.size();
}
而DimensionDictionary的size()方法返回的是idToValue
这个List的大小。
IndexableAdapter中,每个dimension都有一个DimensionAccessor,每个DimensionAccessor中都有一个MutableBitamp数组,数组中的每个元素都是一个MutableBitmap的实现,代表这个dimension中每一个distinct value的倒排索引(MutableBitmap数组的下标为distinct value值的索引,MutableBitmap中保存的是行号)。
processRows方法
在IncrementalIndexAdapter构造函数的最后,会调用processRows方法填充每一列中所有distinct value的bitmap(倒排索引)
private void processRows(
IncrementalIndex<?> index,
BitmapFactory bitmapFactory,
List<IncrementalIndex.DimensionDesc> dimensions
)
{
int rowNum = 0;
for (IncrementalIndexRow row : index.getFacts().persistIterable()) {
final Object[] dims = row.getDims();
for (IncrementalIndex.DimensionDesc dimension : dimensions) {
final int dimIndex = dimension.getIndex();
DimensionAccessor accessor = accessors.get(dimension.getName());
...
final ColumnCapabilities capabilities = dimension.getCapabilities();
if (capabilities.hasBitmapIndexes()) {
final MutableBitmap[] bitmapIndexes = access.invertedIndexes;
indexer.fillBitmapsFromUnsortedEncodedKeyComponent(dims[dimIndex], rowNum, bitmapIndexes, bitmapFactory);
}
}
++rowNum;
}
}
这里的rowNum
是当前行号,dims[dimIndex]
是是当前rowNum行dimIndex列这个cell的索引值(也就是DimensionDictionary中idToValue
中特定值的下标)。
StringDimensionIndexer中fillBitmapsFromUnsortedEncodedKeyComponent方法定义如下:
public void fillBitmapsFromUnsortedEncodedKeyComponent(
int[] key,
int rowNum,
MutableBitmap[] bitmapIndexes,
BitmapFactory factory
)
{
if (!hasBitmapIndexes) {
throw new UnsupportedOperationException("This column does not include bitmap indexes");
}
for (int dimValIdx : key) {
if (bitmapIndexes[dimValIdx] == null) {
bitmapIndexes[dimValIdx] = factory.makeEmptyMutableBitmap();
}
bitmapIndexes[dimValIdx].add(rowNum);
}
}
调用makeIndexFIles方法生成Segment文件
在merge方法中,最终调用makeIndexFiles方法并返回:
return makeIndexFIles(
indexes,
sortedMetricAggs,
outDir,
progress,
mergedDimensions,
mergedMetrics,
rowMergerFn,
true, // fillRowNumConversions
indexSpec,
segmentWriteOutMediumFactory
);
private File makeIndexFiles(
final List<IndexableAdapter> adapters,
final @Nullable AggregatorFactory[] metricAggs,
final File outDir,
final ProgressIndicator progress,
final List<String> mergedDimensions,
final List<String> mergedMetrics,
final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn,
final boolean fillRowNumConversions,
final IndexSpec indexSpec,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
这里rowMergerFun的参数值为MergingRowIterator::new
。MergingRowIterator是TimeAbndDimsIterator的子类。
对于kafka-indexing-service注入的任务来说,basePersistDirectory是用Files.createTempDir()方法创建的,也就是"java.io.tmpdir"指定的目录(默认为var/tmp)下创建的System.currentTimeMills() + "-" + {n},例如"var/tmp/1565779238486-0",
那么outDir为"var/tmp/1565779238486-0/{identifier}/{fireHydrant_count}"
初始化FileSmoosher
final FileSmoosher v9Smoosher = new FileSmoosher(outDir);
FileUtils.forceMkdir(outDir);
初始化SegmentWriteOutMediumFactory
SegmentWriteOutMediumFactory omf = segmentWriteOutMediumFactory != null ? segmentWriteOutMediumFactory:
: defaultSegmentWriteOutMediumFactory;
makeIndexFiles传入的segmentWriteOutMediumFactory为null,因此这里的omf的值defaultSegmentWriteOutMediumFactory是IndexMergerV9的构造函数参数,默认为TmpFileSegmentWriteOutMediumFactory
。
初始化SegmentWriteOutMedium
SegmentWriteOutMedium segmentWriteOutMedium = omf.makeSegmentWriteOutMedium(outDir);
这里调用的是TmpFileSegmentWriteOutMediumFactory的makeSegmentWriteOutMedium方法。返回一个TmpFileSegmentWriteOutMedium实例。
创建并写入version.bin文件
Files.asByteSink(new File(outDir, "version.bin")).write(Ints.toByteArray(IndexIO.V9_VERSION));
将代表IndexIO.V9_VERSION值(0x9)的字节数组写入version.bin文件。
创建并写入factory.json文件
try (FileOutputStream fos = new FileOutputStream(new File(outDir, "factory.json"))) {
mapper.writeValue(fos, new MMappedQueryableSegmentizerFactory(indexIO));
}
写入字典表
在writeDimValuesAndSetupDimConversion方法中, 有
for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) {
mergers.get(dimIndex).writeMergedValueDictionary(indexes);
}
只考虑维度为String的情况的话,这里实际调用的是StringDimensionMergerV9
的writeMergedValueDictionary方法。
特定dimension的distinct value写入字典表
首先定义文件名为{dimensionName}.dim_values
:
String dictFilename = StringUtils.format("%s.dim_values", dimensionName);
然后初始化一个名为dictionaryWriter(StringDimensionMergerV9的全局成员)的GenericIndexerWriter
:
dictionaryWriter = new GenericIndexerWriter<>(segmentWriteOutMeidum, dictFilename, GenericIndexed.STRING_STRATEGY);
这里传入的STRING_STRATEGY是在GenericIndexed类中国定义的static成员。
接着调用上面初始化的GenericIndexerWriter的open()方法:
dictionaryWriter.open();
open()方法实际上是初始化了GenericIndexerWriter的headerOut
和valuesOut
成员,都初始化成了FileWriteOutBytes实例(通过调用TmpFileSegmentWriteOutMedium的makeWriteOutBytes()方法)。
每个FileWriteOutBytes实例中都有一个file
和ch
成员,file是名为peonxxxxx.tmp的File,ch是file对应的FileChannel
。除此之外,FileWriteOutBytes中还有一个类型为ByteBuffer大小为4096字节的buffer
成员,用于保存每次提交的ByteBuffer或者字节数组byte[],通过flush()方法将buffer中的值flush到ch中,然后清空buffer。
{
headerOut = segmentWriteOutMedium.makeWriteOutBytes();
valuesOut = segmentWriteOutMedium.makeWriteOutBytes();
}
headerOut用于写入当前valuesOut的长度(字节个数),每次写入当前valuesOut的长度;
valuesOut用于写入dimension的值(字符串对应的字节数组)。
然后调用StringDimensionMergerV9的writeDictionary方法:
writeDictionary(dimValueLookup);
这里的dimValueLookup的实际类型是在StringDimensionIndexer中定义的getSortedIndexedValues()方法返回的CloseableIndexed<String>
类型的实例。
private void writeDictionary(Iterable<String> dictionaryValues) IOException
{
for (String value : dictionaryValues) {
dictionaryWriter.write(value);
value = NullHandling.emptyToNUllIfNeeded(value);
if (dictionarySize == 0) {
firstDictionaryValue = value;
}
dictionarySize++;
}
}
这里传入的实际类型是CloseableIndexed<String>,CloseableIndexed继承Indexed<String>, Indexed<T>继承Iterable。getSortedIndexedValues()返回的CloseableIndexed<String>实例定义了iterator()方法,返回的是
IndexedIterable
的iterator()方法返回的Iterator<String>定义。在这个定义的next()方法中,调用了getSortedIndexedValues()方法返回的CloseableIndexed<String>实例定义的get方法:public String get(int index) { return getActualValue(index, true); }
getActualValue方法在StringDimensionIndexer中定义:
private String getActualValue(int intermediateValue, boolean idSoted) { if (idSorted) { return sortedLookup().getValueFromSortedId(intermediateValue); } else { return dimLookup.getValue(intermediateValue); } }
sortedLookup()方法调用实际调用的是DimensionDictionary的sort()方法,返回一个SortedDimensionDictionary实例。在调用DimensionDictionary的sorte()方法的过程中,完成了3件事:
- 初始化SortedDimensionDictionary实例中类型为List<String>的sortedVals成员(使用DimensionDictionary中的idToValue成员进行初始化),sortedVals中的值按照字母序升序排列;
- 初始化SortedDimensionDictionary实例中类型为int[]的idToIndex成员,idToIndex的下标是DimensionDictionary中的idToValue成员的下标,值是sortedVals成员的下标;
- 初始化SortedDimensionDictionary实例中类型为int[]的indexToId成员,indexToIdx的下表是sortedVals成员的下标,值是idToValue的下标。
public SortedDimensionDictionary(List<String> idToValue, int length) { Object2IntSortedMap<String> sortedMap = new Object2IntRBTreeMap<>>(Comparators.naturalNullsFirst()); for (int id = 0; id < length; ++id) { String value = idToValue.get(id); sortedMap.put(value, id); } this.sortedVals = Lists.newArrayList(sortedMap.keySet()); this.idToIndex = new int[length]; this.indexToId = new int[length]; int index = 0; for (IntIterator iterator = sortedMap.values().iterator(); iterator.hasNext(); ) { int id = iterator.nextInt(); idToIndex[id] = index; indexToId[index] = id; index++; } }
然后调用SortedDimensionDictionary的getValueFromSortedId方法返回index对应的值:
public String getValueFromSortedId(int index) { return sortedVals.get(index); }
回到writeDictionary方法,这里面主要是调用了GenericIndexerWriter<String>的write方法将数据写入valuesOut
和headerOut
。
对于valuesOut:
- 先写入一个int型的标志位,如果当前要写入的值为null则写入-1,否则写入0;
- 写入实际的值,这里调用的是STRING_STATEGY(类型为ObjectStrategy<String>)的writeTo方法,将实际的dimension的字符串值转成字节数组,然后写入valuesOut(类型为FileWriteOutBytes)的buffer(类型为ByteBuffer)成员,在此期间如果buffer满了则flush到ch中。
对于headerOut:
- 写入当前valuesOut中ch的大小(也就是FileChannel当前的长度)。
将所有的dimension value都写入之后,valuesOut代表的FileWriteOutBytes中的FileChannel如下图:
headerOut代表的FileWriteOutBytes中的FileChannel如下图:
设置dimConversions
dimConversions是StringDimensionMergerV9的成员,类型为ArrayList<IntBuffer>。对于单个indexer(IndexableAdapter)的情况来说,此时dimConversions中只有一个元素null。
设置ColumnarIntsSerializer
在writeMergedValueDictionary方法的最后,调用:
setupEncodedValueWriter();
初始化一个ColumnarIntsSerializer,用来缓存IncrementalIndexRow的facts中的每一行数据(int数组):
String filenameBase = StringUtils.format("%s.forward_dim", dimensionName);
...
encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create(
segmentWriteOutMedium,
filenameBase,
cardinality,
compressionStrategy
);
在CompressedVSizeColumnarIntsSerializer的构造函数中,会构造chunkFactor成员,这个成员的作用是和每个Int占用字节大小相乘,得到准确的buffer size,构造chunFactor的方法如下:
- 根据特定dimension的cardinality,判断每个值的id占用几个字节,如果cardinality小于等于0xFF,则每个值的id只需要一个字节;如果cardinality小于等于0xFFFF,则每个值的id需要2个字节;如果cardinality小于等于0xFFFF,则每个值的id需要3个字节;其余情况需要4个字节;
- 用buffer大小(默认为65536字节,64KB)除以第一步得到的每个value对应的id需要的字节数,算出实际能存储多少id;
- 找到比第二步得到的值小的最近的2的幂作为chunkFactor(Integer.highestOneBit(maxSizePer))。
此外,CompressedVSizeColumnarIntsSerializer还有一个类型为GenericIndexedWriter<ByteBuffer>
的成员flattener,用于写入压缩后的dimension数据。如下初始化:
GenericIndexedWriter.ofCompressedByteBuffers(
segmentWriteOutMedium,
filenameBase,
compression,
sizePer(maxValue, chunkFactor)
)
static GenericIndexedWriter<ByteBuffer> ofCompressedByteBuffers(
final SegmentWriteOutMedium segmentWriteOutMedium,
final String filenameBase,
final CompressionStrategy compressionStrategy,
final int bufferSize
)
{
GenericIndexedWriter<ByteBuffer> writer = new GenericIndexedWriter<>(
segmentWriteOutMedium,
filenameBase,
compressedByteBuffersWriteObjectStrategy(compressionStrategy, bufferSize, segmentWriteOutMedium.getCloser())
);
writer.objectsSorted = false;
return writer;
}
GenericIndexedWriter中的compressedByteBuffersWriteObjectsStrategy返回一个Object<ByteBuffer>实例,在这个实例中定义了一个类型为CompressionStrategy.Compressor的成员compressor,它的实际类型(对于默认的LZ4压缩算法)为LZ4Compressor;除此之外还有一个类型为ByteBuffer的compressedDataBuffer成员,用来承载压缩后的字节数据:
private final CompressionStrategy.Compressor compressor = compressionStrategy.getCompressor();
private final ByteBuffer compressedDataBuffer = compressor.allocateOutBuffer(bufferSize, closer);
在setupEncodedValueWriter()方法调用的最后,调用上面初始化的类型为CompressedVSizeColumnarIntsSerializer的变量encodedValueSerializer的open()方法:
encodedValueSerializer.open();
open()方法打开encodedValueSerializer,实际上调用的是CompressedVSizeColumnarIntsSerializer的类型为GenericIndexedWriter<ByteBuffer>的flattener成员的open方法。上面已经说过GenericIndexedWriter<T>的open()方法初始化了headerOut
和valuesOut
。
设置TimeAndDimsIterator
调用makeMergedTimeAndDimsIterator方法:
final TimeAndDimsIterator timeAndDimsIterator = makeMergedTimeAndDimsIterator(
adapters,
mergedDimensions,
mergedMetrics,
rowMergerFn,
handlers,
mergers
);
首先构造一个元素类型为TransformableRowIterator的List,由于我们的index(IndexableAdapter)只有一个,因此这个List中只会有一个TransformableRowIterator。
调用IndexableAdapter的getRows()方法构造TransformableRowIterator。这里的adapter的实际类型为IncrementalIndexAdapter,它的getRows方法返回一个IncrementalIndexRowIterator实例:
public TransformableRowIterator getRows()
{
return new IncrementalIndexRowIterator(index);
}
IncrementalIndexRowIterator用来遍历IncrementalIndexRow集合,这里的index是IncrementalIndex。
在IncrementalIndexRowIterator的构造方法中,初始化了timeAndDimsIterator
、currentRowPointer
和markedRowPointer
三个成员:
IncrementalIndexRowIterator(IncrementIndex<?> incrementalIndex)
{
this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator();
this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter);
this.markedRowPointer = makeRowPointer(incrementalIndex, markedRowHolder, new RowNumCounter);
}
当前currentRowHolder为空,它的currEntry成员就是当前遍历到的IncrementalIndexRow:
private final IncrementalIndexRowHolder currentRowHolder = new IncrementalIndexRowHolder();
public class IncrementalIndexRowHolder implements LongColumnSelector
{
private IncrementalIndexRow currEntry = null;
...
}
makePointer方法范慧慧一个RowPointer实例:
private static RowPointer makeRowPointer(
IncrementalIndex<?> incrementalIndex,
IncrementalIndexRowHolder rowHolder,
RowNumCounter rowNumCounter
)
{
ColumnSelectorFactory columnSelectorFactory =
new IncrementalIndexColumnSelectorFactory(incrementalIndex, VirtualColumns.EMPTY, false, rowHolder);
ColumnValueSelector[] dimensionSelectors = incrementalIndex
.getDimensions()
.stream()
.map(dim -> {
ColumnValueSelector selectorWithUnsortedValues = columnSelectorFactory.makeColumnValueSelector(dim.getName());
return dim.getIndexer().convertUnsortedValuesToSorted(selectorWithUnsortedValues);
})
.toArray(ColumnValueSelector[]::new);
List<DimensionHandler> dimensionHandlers = incrementalIndex
.getDimensions()
.stream()
.map(IncrementalIndex.DimensionDesc::getHandler)
.collect(Collectors.toList());
ColumnValueSelector[] metricSelectors = incrementalIndex
.getMetricNames()
.stream()
.map(columnSelectorFactory::makeColumnValueSelector)
.toArray(ColumnValueSelector[]::new);
return new RowPointer(
rowHolder,
dimensionSelectors,
dimensionHandlers,
metricSelectors,
incrementalIndex.getMetricNames().
rowNumCounter
);
}
public RowPointer(
ColumnValueSelector timestampSelector,
ColumnValueSelector[] dimensionSelectors,
List<DimensionHandler> dimensionHandlers,
List<String> metricNames,
IntSupplier rowNumPointer
)
{
super(timestampSelector, dimensionSelectors, dimensionHandlers, metricSelectors, metricNames);
this.rowNumPointer = rowNumPointer;
}
传入RowPointer构造函数的第一个参数rowHolder的实际类型是IncrementalIndexRowHolder,它继承了LongColumnSelector,LongColumnValueSelector又继承了ColumnValueSelector<Long>。IncrementalIndexRowHolder中的getLong方法返回的是当前IncrementalIndexRow中的timestamp:
public long getLong()
{
return currEntry.getTimestamp();
}
初始化IncrementalIndexRowIterator之后,将它加入TransformableRowIterator的List:
perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers));
最后,调用rowMergerFn的apply方法。rowMergerFn这里是MergingRowIterator::new
:
return rowMergerFn.apply(perIndexRowIterators);
MergingRowIterator(List<TransformableRowIterator> iterators)
{
iterators.forEach(closer::register);
originalIterators = new TransformableRowIterator[iterators.size()];
pQueue = IntStream
.range(0, iterators.size())
.filter(indexNum -> iterators.get(indexNum).moveToNext())
.mapToObj(indexNum -> {
TransformableRowIterator rowIterator = iterators.get(indexNum);
rowIterator.getPointer().setIndexNum(indexNum);
originalIterators[indexNum] = rowIterator;
return rowIterator;
})
.toArray(RowIterator[]::new);
equalToChild = new boolean[pQueue.length];
heapify();
initEqualToChildStates();
}
IncrementalIndexRowIterator的moveToNext()方法:
public boolean moveToNext()
{
if (!timeAndDimsIterator.hasNext()) {
return false;
}
currentRowHolder.set(timeAndDimsIterator.next());
currentRowNumCounter.increment();
return true;
}
如果当前IncrementalIndex(实际是IncrementalIndex的FactsHolder中的Map<IncrementalIndexRow, IncrementalIndexRow>)中没有数据,则返回false;如果有数据,则设置currentRowHolder去hold第一行IncrementalIndexRow,currentRowNumCounter自增1变为0,返回true。
设置timeWriter
调用setupTimeWriter:
final GenericColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec);
在setupTimeWriter方法中,创建一个LongColumnSerializer并返回:
GenericColumnSerializer timeWriter = createLongColumnSerializer(
segmentWriteOutMedium,
"little_end_time",
indexSpec
);
return LongColumnSerializer.create(
segmentWriteOutMedium,
columnName,
indexSpec.getMetricCompression(),
indexSpec.getLongEncoding()
);
然后调用timeWriter的open()方法:
public void open() throws IOException
{
writer = CompressionFactory.getLongSerializer(
segmentWriteOutMedium,
StringUtils.format("%s.long_column", filenameBase),
byteOrder,
encoding,
compression
);
writer.open();
}
writer是LongColumnSerializer的成员:
private ColumnarLongsSerializer writer;
writer的实际类型是BlockLayoutColumnarLongsSerializer:
public static ColumnarLongsSerializer getLongSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder order,
LongEncodingStrategy encodingStrategy,
CompressionStrategy compressionStrategy
)
{
if (encodingStrategy == LongEncodingStrategy.AUTO) {
...
} else if (encodingStrategy == LongEncodingStrategy.LONGS) {
if (compressionStrategy [[ CompressionStrategy.NONE) {
...
} else {
return new BlockLayoutColumnarLongsSerializer(
segmentWriteOutMedium,
filenameBase,
order,
new LongLongEncodingWriter(order),
compressionStrategy
);
}
} else {
...
}
}
BlockLayoutColumnarLongsSerializer中有一个类型为GenericIndexedWriter<ByteBuffer>
的名为flattener的成员,和CompressedVSizeColumnarIntsSerializer
中的flattener成员类型。都是调用GenericIndexedWriter的ofCompressedByteBuffers初始化的:
this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(segmentWriteOutMedium, filenameBase, compression, bufferSize);
然后调用BlockLayoutColumnarLongsSerializer的open()方法,这个方法调用flattener的open()方法初始化GenericIndexedWriter中类型为FileWriteOutBytes的headerOut
和valueOut
。
最后在setupTimeWriter中,返回类型为LongColumnSerializer的timeWriter。
设置metricWriters
final ArrayList<GenericColumnSerializer> metricWriters =
setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec);
假设我们有2个metrics,一个ValueType为LONG,另一个为COMPLEX(HyperUnique),则前者的metricWriter为LongColumnSerializer
,后者的metricWriter:
final String typeName = metricTypeNames.get(metric);
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName);
writer = serde.getSerializer(segmentWriteOutMedium, metric);
那对于HyperUnique类型的metric来说,serde的类型是HyperUniquesSerde
,它的getSerializer方法返回LargeColumnSupportedComplexColumnSerializer
:
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return LargeColumnSupportedComplexColumnSerializer.create(segementWriteOutMedium, column, this.getObjectStrategy());
}
随后调用metricWriter的open()方法,对于Long类型的metric来说,调用的是LongColumnSerializer的open方法,也就类型为BlockLayoutColumnarLongsSerializer的成员writer的open方法,也就是BlockLayoutColumnarLongsSerializer的成员flattener的open方法;对于HyperUnique类型的metric来说,用的是LargeColumnSupportedComplexColumnSerializer的open方法,也就是类型为GenericIndexedWriter<HyperLogLogCollector>的成员writer的open方法。
遍历数据集 & 写入列数据
调用mergeIndexesAndWriteColumns:
List<IntBuffer> rowNumConversions = mergeIndexesAndWriteColumns(
adapters,
progress,
timeAndDimsIterator,
timeWriter,
metricWriters,
mergers,
fillRowNumConversions
)
在mergeIndexesAndWriteColumns方法中,
首先初始化rowNumConversions列表
List<IntBuffer> rowNumConversions = null;
if (fillRowNumConversions) {
rowNumConversions = new ArrayList<>(adapters.size());
for (IndexableAdapter adapter : adapters) {
int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr,, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr));
}
}
这里只有一个adapter,因此rowNumConversions列表中只有一个IntBuffer,这个IntBuffer中每个int都是-1,这个IntBuffer的size是当前IndexableAdapter中IncrementalIndex中FactsHolder中类型为Map<IncrementalIndexRow, IncrementalIndexRow>的size。
遍历index中的每一行并写入每一行中的时间戳timestamp
while (timeAndDimsIterator.moveToNext()) {
...
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
timeWriter.serialize(timeAndDims.timestampSelector);
...
}
timeAndDimsPointer的实际类型是MergingRowIterator
,timeAndDims的实际类型是RowPointer,它的timestampSelector成员的实际类型是IncrementalIndexRowHolder,继承了LongColumnSelector。
timeWriter的实际类型是LongColumnSerializer,它的serialize方法:
public void serialize(ColumnValueSelector<?> selector) throws IOException
{
writer.add(selector.getLong());
}
上面说了selector的实际类型是IncrementalIndexRowHolder,调用它的getLong()方法返回的事当前IncrementalIndexRow中的timestamp。
LongColumnSerializer的writer成员的实际类型是BlockLayoutColumnarLongsSerializer,它的add方法将当前的timestamp写入它的endBuffer成员,最终当写入的byte数和numInsertedForNextFlush相等时,将endBuffer全部写入类型为GenericIndexedWriter<Bytebuffer>的flattener成员的valuesOut和headerOut中:
public void add(long value) throws IOException
{
if (endBuffer == null) {
throw new IllegalStateException("written out already");
}
if (numInserted == numInsertedForNextFlush) {
numInsertedForNextFlush += sizePer;
writer.flush();
endBuffer.flip();
flattener.write(endBuffer);
endBuffer.clear();
writer.setBuffer(endBuffer);
}
writer.write(value);
++numInserted;
}
BlockLayoutColumnarLongsSerializer的writer成员类型是LongsLongEncodingWriter,它的write方法:
public void write(long value) throws IOException
{
if (outBuffer != null) {
outBuffer.putLong(value);
}
}
这里的outBuffer是在BlockLayoutColumnarLongsSerializer构造函数中调用LongsLongEncodingWriter的setBuffer方法设置上去的,outbuffer实际上是BlockLayoutColumnarLongsSerializer的endBuffer成员:
endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), segmentWriteOutMedium.getCloser()).order(byteOrder);
在GenericIndexedWriter<ByteBuffer>的write方法中,会基于传入的ByteBuffer调用stragegy的writeTo方法:
strategy.writeTo(objectToWrite, valuesOut)
这里的strategy是一个ObjectStrategy的实例,它的writeTo方法是在GenericIndexedWriter<T>中定义的compressedByteBuffersWriteObjectStrategy方法中返回的ObjectStrategy<ByteBuffer>实现中定义的:
public void writeTo(ByteBuffer val, WriteOutBytes out) throws IOException
{
compressedDataBuffer.clear();
int valPos = val.position();
out.write(compressor.compress(val, compressedDataBuffer));
val.position(valPos);
}
这里用LZ4Compressor的compress方法将未压缩的ByteBuffer转换成压缩的ByteBuffer,然后将压缩或的ByteBuffer写入valuesOut(类型为FileWriteOutBytes)的buffer,最终写入ch(FileChannel)。
最终,segment的时间列对应的FileChannel如下:
对于valuesOut:
对于headerOut:
遍历index中的每一行并写入每一行中的metrics
while (timeAndDimsIterator.moveToNext()) {
...
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
...
for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) {
metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex));
}
...
}
timeAndDims.getMetricSelector(metricIndex)对于CountAggregator
metric返回的是LongMetricColumnSelector
,LongColumnSerializer调用serialize之后,对应的FileChannel如下:
对于valuesOut:
对于headerOut:
对于timeAndDims.getMetricSelector(metricIndex)对于HyperUniquesAggregator
metric返回的是ObjectMetricColumnSelector
。LargeColumnSupportedComplexColumnSerializer的serialize方法:
public void serialize(ColumnValueSelector<? extends T> selector) throws IOException
{
writer.write(selector.getObject());
}
ObjectMetricColumnSelector的getObject()方法返回一个HyperLogLogCollector实例。writer是LargeColumnSupportedComplexColumnSerializer的成员,类型是GenericIndexedWriter<HyperLogLogCollector>
,它的strategy成员在HyperUniquesSerde中定义,类型为ObjectStrategy<HyperLogLogCollector>
,它的toBytes方法定义如下:
public byte[] toBytes(HyperLogLogCollector collector) {
if (collector == null) {
return new byte[]{};
}
ByteBuffer val = collector.toByteBuffer();
byte[] retVal = new byte[val.remaining()];
val.asReadOnlyBuffer().get(retVal);
return retVal;
}
ObjectStrategy的writeTo方法:
default void writeTo(T val, WriteOutBytes out) throws IOException
{
byte[] bytes = toBytes(val);
if (bytes != null) {
out.write(bytes);
}
}
LargeColumnSupportedComplexColumnSerializer调用serialize之后,对应的FileChannel如下:
对于valuesOut:
对于headerOut:
遍历index中的每一行并写入每一行中的dimensions
while (timeAndDimsIterator.moveToNext()) {
...
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
...
for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) {
DimensionMerger merger = mergers.get(dimIndex);
if (merger.canSkip()) {
continue;
}
merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex));
}
...
}
每个dimension都对应一个DimensionMerger(这里为StringDimensionMergerV9)。StringDimensionMergerV9中的processMergedRow方法中有:
public void processMergedRow(ColumnValueSelector selector) throws IOException
{
IndexedInts row = getRow(selector);
...
int value = row.size() == 0 ? 0 : row.get(0);
((SingleValueColumnarIntsSerializer) encodedValueSerializer).addValue(value);
...
}
encodedValueSerializer是StringDimensionMergerV9的成员,实际类型为CompressedVSizeColumnarIntsSerializer,它的addValue方法:
public void addValue(int val) throws IOException
{
if (endBuffer == null) {
throw new IllegalStateException("written out already");
}
if (@endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(endBuffer);
endBuffer.clear();
}
intBuffer.putInt(0, val);
if (isBigEndian) {
endBuffer.put(intBuffer.array(), Integer.BYTES - numBytes, numBytes);
} else {
endBuffer.put(intBuffer.array(), 0, numBytes);
}
numInserted++;
}
对IncrementalIndex中的facts中的所有IncrementalIndexRow调用完addValue之后,一个特定dimension对应的FileChannel:
对于valuesOut:
对于headerOut:
设置rowNumConversions
...
int rowCount = 0
...
while (timeAndDimsIterator.moveToNext()) {
...
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
...
RowPointer rowPointer = (RowPointer) timeAndDims;
IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum());
int rowNum = rowPointer.getRowNum();
while (conversionBuffer.position() < rowNum) {
conversionBuffer.put(INVALID_ROW);
}
conversionBuffer.put(rowCount);
...
}
...
if (rowNumConversions != null) {
for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind();
}
}
在这之后,rowNumConversions中的IntBuffer中的每个Integer的值都是它的当前位置。如IntBuffer中的offset为0的int值为0,offset为1的int值为1,以此类推。
持久化时间列
makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec);
在makeTimeColumn方法中,首先创建一个ColumnDescriptor:
final ColumnDescriptor serdeficator = ColumnDescriptor
.builder()
.setValueType(ValueType.LONG)
.addSerde(createLongColumnPartSerde(timeWriter, indexSpec))
.build();
createLongColumnPartSerde创建一个LongNumericColumnPartSerde
:
static ColumPartSerde createLongColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec)
{
...
return LongNumericColumnPartSerde.serializerBuilder()
.withByteOrder(IndexIO.BYTE_ORDER)
.withDelegate(serializer)
.build();
}
然后调用makeColumn:
makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator);
private void makeColumn(
final FileSmoosher v9Smoosher,
final String columnName,
final ColumnDescriptor serdeficator
) throws IOException
{
ZeroCopyByteArrayOutputStream specBytes = new ZeroCopyByteArrayOutputStream();
serializerUtils.writeS†ring(specBytes, mapper.writeValueAsString(serdeficator));
try (SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
columnName,
specBytes.size() + serdificator.getSerializedSize()
)) {
specByytes.writeTo(channel);
serdificator.writeTo(channel, v9Smoosher);
}
}
ZeroCopyByteArrayOutputStream集成了ByteArrayOutputStream,只重写了writeTo方法。serializerUtils的writeString方法:
public <T extends OutputStream> void writeString(T out, String name) throws IOException
{
byte[] nameBytes = StringUtils.toUTF8(name);
writeInt(out, nameBytes.length);
out.write(nameBytes);
}
调用writeString之后,specBytes中的内容:
第一个slot为序列化后的ColumnDescriptor json字符串to bytes之后的字节数;
第二个slot为序列化后的ColumnDescriptor json字符串{"valueType":"LONG","hasMultipleValues":false,"parts":[{"type":"long","byteOrder":"LITTLE_ENDIAN"}]}
specBytes.size() + serdificator.getSerializedSize()得到需要写入到smoosh文件的字节数。其中serdificator的getSerializedSize()方法返回的至实际上就是LongNumericColumnPartSerde调用getSerializer()方法返回的LongColumnSerializer(timeWriter)调用getSerializedSize()返回的值:
public long getSerializedSize() throws IOException
{
long size = 0;
for (ColumnPartSerde part : parts) {
size += part.getSerializer().getSerializedSize();
}
return size;
}
LongColumnSerializer的getSerializedSize()方法返回它的类型为BlockLayoutColumnarLongsSerializer的writer成员的getSerializedSize()方法的返回值,BlockLayoutColumnarLongsSerializer的getSerializedSize()方法:
public long getSerializedSize() throws IOException
{
writeEndBuffer();
return metaSerdeHelper.size(this) + flattener.getSerializedSize();
}
writeEndBuffer()方法将endBuffer中的内容写入flattener。
metaSerdeHelper:
private static final MetaSerdeHelper<BlockLayoutColumnarLongsSerializer> metaSerdeHelper = MetaSerdeHelper
.firstWriteByte((BlockLayoutColumnarLongsSerializer x) -> CompressedColumnarLongsSupplier.VERSION)
.writeInt(x -> x.numInserted)
.writeInt(x -> x.sizePer)
.writeSomething(CompressionFactory.longEncodingWriter(x -> x.writer, x -> x.compression));
因此metaSerdeHelper.size(this)返回1 + 4 + 4 + 1 = 10
flattener(GenericIndexedWriter)的getSerializedSize方法:
public long getSerializedSize() throws IOException
{
if (requireMultipleFiles) {
return multiFileMetaSerdeHelper.size(this);
} else {
return singleFileMetaSerdeHelper.size(this) + headerOut.size() + valuesOut.size();
}
}
后略。。。等有时间了接着写。。。。。。。。
**粗体** _斜体_ [链接](http://example.com) `代码` - 列表 > 引用
。你还可以使用@
来通知其他用户。