前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >Elasitcsearch 底层系列 Lucene 内核解析之Point索引

Elasitcsearch 底层系列 Lucene 内核解析之Point索引

原创
作者头像
老生姜
修改2018-11-26 14:52:45
3.3K1
修改2018-11-26 14:52:45
举报
文章被收录于专栏:Elasticsearch实验室

1. 背景

       Luene是一款高性能、可扩展的信息检索库,可实现对文档元信息、文档内容的搜索功能。用户可以使用Lucene 或 基于Lucene开发的成熟产品Nutch/Solr/Elasticsearch等,快速构建搜索服务,如文件搜索、网页搜索等。在Lucene概览中,我们初步介绍了其底层的核心存储文件,本文主要介绍其中的数值索引(Point索引)部分,分析数值索引的文件结构及其读写流程。

2. 简介

       在早期版本中,Luene并没有针对数值设置专属的字段类型,因此数值也是当做字符串存储的,所有字段都是字符串类型,倒排索引均由Trie-Tree数据结构实现。这种索引结构对于精确的term查询比较友好,可以快速返回结果。而对于数值类型的范围查询,效率就比较低了。考虑到数值类型的字段常用于范围比较,从Lucene 6.0版本开始,引入针对数值类型的新索引数据结构BKD-Tree,用于优化Lucene中范围查询的性能。由于这一索引结构最初用于地理坐标场景,因此被命名为Point索引。

2.1 BKD-Tree简介

       Point索引基于BKD-Tree(Block K-Dimension Balanced Tree)实现。类似LSM-Tree,BKD-Tree为一组KDB-Tree(K-Dimension Balanced Tree)的集合。Lucene的一个Index由多个Segment组成,每个Segment中每个数值字段的索引即为一个KDB-Tree。而在Segment Merge的过程中,多个KDB-Tree会进行合并,生成一个较大的KDB-Tree。

       KDB-Tree实际是一棵特殊的多维度B+Tree,和传统B+Tree只包含一个维度略有不同,KDB-Tree会按照多个维度持续切分,生成整个树结构。这里采用常规的构建方式,以二维平面点(x,y)的集合A(2,3)、B(5,4)、C(9,6)、D(4,7)、E(8,1)、F(7,2)为例,假设叶子节点最多包含2个平面点,2阶KDB-Tree的构建过程如下:

  • 样例中平面点共6个,每当2阶KDB-Tree深度增加一层,叶子节点最多可包含的平面点个数缩小一倍,叶子节点个数也会对应增加一倍。这里当叶子节点数为4时,其中的平面点数不超过2。
  • KDB-Tree从根节点开始构建,选择x作为切分维度,如上点集合在x维从小到大排序为A(2,3)、D(4,7)、B(5,4)、F(7,2)、E(8,1)、C(9,6),取第(0+6+1)/2=3个平面点的横坐标x=7用于切分,下标[0, 3)的平面点A(2,3)、D(4,7)、B(5,4)挂在x=7的左子树,下标[3, 6)的平面点F(7,2)、E(8,1)、C(9,6)挂在x=7的右子树(注:其中0为平面点起始下标,6为右边界下标但不包含)。
  • 继续构建x=7节点的左子树,选择y作为切分维度,平面点在y维从小到大排序为A(2,3)、B(5,4)、D(4,7),取(0+3+1)/2=2个平面点的横坐标y=7用于切分,A(2,3)、B(5,4)挂在其左子树,D(4,7)挂在其右子树。
  • 继续构建x=7节点的右子树,选择y作为切分维度,平面点在y维从小到大排序为E(8,1)、F(7,2)、C(9,6),取(3+6+1)/2=5个平面点的横坐标y=6用于切分,E(8,1)、F(7,2)挂在其左子树,C(9,6)挂在其右子树。
  • 至此KDB-Tree已包含4个叶子节点,构建完成。结果如下:

       在构建KDB-Tree的过程中,一个重要的步骤是切分维度的选择,常见选择方式为:

  • 差值法:计算各维度的差值范围,选择差值最大的维度,上述示例采用此法。
  • 方差法:计算各维度的方差,方差越大分布越分散,方差越小分布越集中。从方差大的维度开始切分可以取得很好的切分效果及平衡性。

       上述是对BKD-Tree的简要介绍,方便读者建立对BKD-Tree的直观印象,如果希望了解更多BKD-Tree、KDB-Tree相关内容,可参考相应论文。由于Lucene未对BKD-Tree和KDB-Tree进行明确的概念区分,为了和源码一致,本文在后续介绍中会统一使用名词BKD-Tree。

2.2 数值索引的基本原理

       数值索引依赖于BKD-Tree加速范围查询,基本原理是利用BKD-Tree类似B+Tree的索引功能。我们继续结合2.1节的示例做介绍,当我们需要查询x∈[3, 6), y∈[2, 5)范围内的所有平面点时,我们的查询逻辑如下:

  • 首先从根节点开始,由切分维度x=7得知,所有x<7的平面点在左子树,右子树排除,进入左子树继续遍历。
  • 当前节点为非叶子节点,由切分维度y=7得知,所有y<7的平面点在左子树,右子树排除,进入左子树继续遍历。
  • 当前节点为叶子节点,停止遍历,取出其中平面点进行数值比较,发现B(5,4)满足,A(2,3)被排除。

3. 文件结构

       Lucene的Point索引以BKD-Tree实现,每一个字段的Point Index是一棵独立的BKD-Tree,持久化到磁盘上已.dim和.dii文件存储,整体文件结构如下:

注:绿色箭头代表数据结构展开,红色箭头代表文件偏移(指针)

  • .dii:记录每个字段的Point索引在.dim文件中的偏移,可理解为索引的索引。
    • count:字段个数。
    • offset:某字段Point Index的偏移。
  • .dim:存储Point索引(BKD-Tree)的持久化数据。
    • field point index:每个字段的Point索引,多个字段顺序存储。
      • leaf block:BKD-Tree的叶子节点,存储point value到doc id的映射。
        • doc ids:每个叶子节点中doc id的集合。
        • dim prefix:存储Point Value各维度的公共前缀,降低存储成本。
        • point values:按doc id顺序,存储叶子节点point value的集合
      • packed index:按中序遍历方式存储BKD-Tree的非叶子节点,每个节点包含切分维度、偏移等信息。
        • left block offset:当前节点的最左叶子节点的偏移,用于快速跳至叶子节点起始位置。
        • split dim:切分维度的下标。
        • split value:切分维度的值。
        • left total bytes:当前节点左子树占用的总字节数,结合left block offset,可快速跳至右子树的起始位置。

       当用户对某字段进行条件查询时,可以先通过.dii获取该字段的Point索引(BKD-Tree)偏移,然后在.dim中定位BKD-Tree的非叶子节点(packed index),按照切分维度信息遍历BKD-Tree得到符合条件的叶子节点,最后读取叶子节点过滤得到最终的doc id集合。

       本节初步介绍Point索引的整体结构,读者先建立一个初步印象,下面会结合读写流程详细介绍Point索引的构建和查询。

4. 读写流程

       Point索引读写的核心是对BKD-Tree的构建和查询,而BKD-Tree是多维度平衡树,在Lucene使用过程中,我们常使用的场景为一维(如整型字段)、二维(如地理坐标类型字段)。下面我们以二维场景为例,结合3中介绍的文件结构,详细介绍多维场景下Point索引的读写流程。

4.1 写入流程

       我们知道,Lucene在处理写入请求时,首先对写入数据进行预处理并缓存在内存中,然后周期性的从内存刷向磁盘,生成Segment。数值索引作为核心存储的一部分,处理流程也是如此。下面我们逐步介绍:

4.1.1 写入数据到内存
  • 用户通过Lucene的IndexWriter.addDocument()等接口发送写入请求,这些请求会经过如下转发过程,最终到达DefaultIndexingChain.processDocument()接口中:
  • DefaultIndexingChain.processDocument()遍历处理doc中的所有字段:
代码语言:txt
复制
for (IndexableField field : docState.doc) {
    fieldCount = processField(field, fieldGen, fieldCount);
}
  • processField()依次处理该字段的store/docvalue/point等属性的写入,如果字段的Point维度数量不为0,代表该字段有Point索引,则进行Point索引写入处理:
代码语言:txt
复制
if (fieldType.pointDimensionCount() != 0) {
    if (fp == null) {
        fp = getOrAddField(fieldName, fieldType, false);
    }
    indexPoint(fp, field);
}
  • 其中,getOrAddField()用于获取保存内存数据的PerField类对象,所有数值索引数据会临时保存在其中,后面介绍flush生成segment时会依据其中数据,生成KDB-Tree并落盘。indexPoint()将Point数据写入内存,具体通过PerField类对象的成员pointValuesWriter完成:
代码语言:txt
复制
fp.pointValuesWriter.addPackedValue(docState.docID, field.binaryValue());
  • PointValuesWriter.addPackedValue()的实现方式很简单,把doc id和point value分别保存到整型数组和字节数组中:
代码语言:txt
复制
bytes.append(value);
docIDs[numPoints] = docID;
4.1.2 flush生成Segment
  • 当内存使用达到阈值 或者 刷新周期已达到 等场景发生时,DefaultIndexingChain.flush()被触发,会将内存中的临时数据写入到磁盘:
代码语言:txt
复制
public Sorter.DocMap flush(SegmentWriteState state) throws IOException, AbortingException {
    ……
    writePoints(state, sortMap);
    ……
}
  • WritePoints()会遍历保存在内存中的PerField类对象,如果字段的Point维度数不为0(pointValuesWriter 不为空),则开始该字段的Point索引落盘过程:
代码语言:txt
复制
for (int i=0;i<fieldHash.length;i++) {
    PerField perField = fieldHash[i];
    while (perField != null) {
        if (perField.pointValuesWriter != null) {
            ……
            if (pointsWriter == null) {
                pointsWriter = fmt.fieldsWriter(state);
            }
            perField.pointValuesWriter.flush(state, sortMap, pointsWriter);
            perField.pointValuesWriter = null;
        }
        ……
        perField = perField.next;
    }
}
  • 其中,当落盘第一个字段时,Lucene60PointsWriter类对象pointsWriter为null,会进行pointsWriter的构建,触发Point索引文件.dim的创建,并写入文件头信息:
代码语言:txt
复制
String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
                                                     writeState.segmentSuffix,
                                                     Lucene60PointsFormat.DATA_EXTENSION);
dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
boolean success = false;
try {
    CodecUtil.writeIndexHeader(dataOut,
                               Lucene60PointsFormat.DATA_CODEC_NAME,
                               Lucene60PointsFormat.DATA_VERSION_CURRENT,
                               writeState.segmentInfo.getId(),
                               writeState.segmentSuffix);
    success = true;
}
  • 参考4.1.1介绍,每个字段的实际数据保存在PerField.pointValuesWriter中,故通过PointValuesWriter.flush()刷新生成segment。在PointValuesWriter.flush()中,基于MutablePointsReader类构建了一个Reader对象,用于读取内存数据写入Lucene60PointsWriter类对象writer中:
代码语言:txt
复制
MutablePointsReader values = new MutablePointsReader() {
    ……
}
……
writer.writeField(fieldInfo, values);
  • 而Lucene60PointsWriter会通过BKDWriter类对象writer,完成当前字段Point索引的写入,并记录该字段Point索引在dim文件中的偏移:
代码语言:txt
复制
final long fp = writer.writeField(dataOut, fieldInfo.name, (MutablePointsReader) values);
if (fp != -1) {
    indexFPs.put(fieldInfo.name, fp);
}
  • BKDWriter结合字段的维度数量,开始构建BKD-Tree,整体实现逻辑类似2.1小节中的介绍,但Lucene中设定叶子节点最多有1024个point value。在正式构建BKD-Tree之前,先计算应有的叶子节点数量,从而保证单个叶子节点中包含的point value不超过1024个:
代码语言:txt
复制
long innerNodeCount = 1;
while (countPerLeaf > maxPointsInLeafNode) {
    countPerLeaf = (countPerLeaf+1)/2;
    innerNodeCount *= 2;
}

int numLeaves = Math.toIntExact(innerNodeCount);
  • 然后,遍历所有维度,获取所有维度的最大、最小值,用于后续选择合适的维度划分生成子树:
代码语言:txt
复制
Arrays.fill(minPackedValue, (byte) 0xff);
Arrays.fill(maxPackedValue, (byte) 0);
for (int i = 0; i < Math.toIntExact(pointCount); ++i) {
    reader.getValue(i, scratchBytesRef1);
    for(int dim=0;dim<numDims;dim++) {
        int offset = dim*bytesPerDim;
        if (StringHelper.compare(bytesPerDim, scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, minPackedValue, offset) < 0) {
            System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, minPackedValue, offset, bytesPerDim);
        }
        if (StringHelper.compare(bytesPerDim, scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, maxPackedValue, offset) > 0) {
            System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset + offset, maxPackedValue, offset, bytesPerDim);
        }
    }
}
  • 完成叶子结点和维度信息统计后,从根节点开始,递归调用build()构造BKD-Tree,完成一个字段Point索引的构造及dim文件写入,这部分细节笔者在4.1.3小结展开介绍。BKD-Tree构造完成后,返回该字段Point索引在dim文件中的偏移:
代码语言:txt
复制
final int[] parentSplits = new int[numDims];
build(1, numLeaves, reader, 0, Math.toIntExact(pointCount), out,
      minPackedValue, maxPackedValue, parentSplits,
      splitPackedValues, leafBlockFPs,
      new int[maxPointsInLeafNode]);
long indexFP = out.getFilePointer();
writeIndex(out, Math.toIntExact(countPerLeaf), leafBlockFPs, splitPackedValues);
return indexFP;
  • 当遍历完成所有字段的Point索引写入后,通过Lucene60PointsWriter.finish(),为dim文件写入footer信息并关闭。同时生成dii文件,记录每个字段的Point索引在dim文件中的偏移,用于定位索引使用:
代码语言:txt
复制
CodecUtil.writeIndexHeader(indexOut,
                           Lucene60PointsFormat.META_CODEC_NAME,
                           Lucene60PointsFormat.INDEX_VERSION_CURRENT,
                           writeState.segmentInfo.getId(),
                           writeState.segmentSuffix);
int count = indexFPs.size();
indexOut.writeVInt(count);
for(Map.Entry<String,Long> ent : indexFPs.entrySet()) {
    FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(ent.getKey());
    if (fieldInfo == null) {
        throw new IllegalStateException("wrote field=\"" + ent.getKey() + "\" but that field doesn't exist in FieldInfos");
    }
    indexOut.writeVInt(fieldInfo.number);
    indexOut.writeVLong(ent.getValue());
}
CodecUtil.writeFooter(indexOut);
4.1.3 BKD-Tree构建过程

       BKD-Tree的构建是一个递归过程,从根节点开始构建,选择合适的维度持续进行拆分,直到产生足够的叶子节点,保证每个叶子节点包含的point value不超过1024个。对于叶子节点,则把其包含的point做整理后,写入dim存储,并记录该节点起始位置。详细构造流程如下:

  • 对于非叶子节点,首先根据各维度最大值、最小值、父节点的切分维度等信息,选择合适的维度进行切分:
代码语言:txt
复制
final int splitDim = split(minPackedValue, maxPackedValue, parentSplits);
  • 对该节点包含的point进行二分切割,使得在切分维度上,mid左边的数据全部小于右边:
代码语言:txt
复制
final int mid = (from + to + 1) >>> 1;
int commonPrefixLen = bytesPerDim;
for (int i = 0; i < bytesPerDim; ++i) {
    if (minPackedValue[splitDim * bytesPerDim + i] != maxPackedValue[splitDim * bytesPerDim + i]) {
        commonPrefixLen = i;
        break;
    }
}
MutablePointsReaderUtils.partition(maxDoc, splitDim, bytesPerDim, commonPrefixLen,
        reader, from, to, mid, scratchBytesRef1, scratchBytesRef2);
  • 获取切分维度的中值,结合当前节点的最大值、最小值,生成子节点的最大值、最小值:
代码语言:txt
复制
final int address = nodeID * (1+bytesPerDim);
splitPackedValues[address] = (byte) splitDim;
reader.getValue(mid, scratchBytesRef1);
System.arraycopy(scratchBytesRef1.bytes, 
                 scratchBytesRef1.offset + splitDim * bytesPerDim,
                 splitPackedValues, address + 1, bytesPerDim);
byte[] minSplitPackedValue = Arrays.copyOf(minPackedValue, packedBytesLength);
byte[] maxSplitPackedValue = Arrays.copyOf(maxPackedValue, packedBytesLength);
System.arraycopy(scratchBytesRef1.bytes,
                 scratchBytesRef1.offset + splitDim * bytesPerDim,
                 minSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
System.arraycopy(scratchBytesRef1.bytes,
                 scratchBytesRef1.offset + splitDim * bytesPerDim,
                 maxSplitPackedValue, splitDim * bytesPerDim, bytesPerDim);
  • 递归调用,生成左右子树:
代码语言:txt
复制
// 递归遍历构造左右子树
parentSplits[splitDim]++;
build(nodeID * 2, leafNodeOffset, reader, from, mid, out,
        minPackedValue, maxSplitPackedValue, parentSplits,
        splitPackedValues, leafBlockFPs, spareDocIds);
build(nodeID * 2 + 1, leafNodeOffset, reader, mid, to, out,
        minSplitPackedValue, maxPackedValue, parentSplits,
        splitPackedValues, leafBlockFPs, spareDocIds);
parentSplits[splitDim]--;
  • 按照此逻辑持续进行切分,当到达叶子节点后结束递归,开始整理叶子节点包含的数据,并进行实际的写入过程。首先获取叶子节点中所有Point各维度的公共前缀长度:
代码语言:txt
复制
      Arrays.fill(commonPrefixLengths, bytesPerDim);
      reader.getValue(from, scratchBytesRef1);
      for (int i = from + 1; i < to; ++i) {
        reader.getValue(i, scratchBytesRef2);
        for (int dim=0;dim<numDims;dim++) {
          final int offset = dim * bytesPerDim;
          for(int j=0;j<commonPrefixLengths[dim];j++) {
            if (scratchBytesRef1.bytes[scratchBytesRef1.offset+offset+j] != scratchBytesRef2.bytes[scratchBytesRef2.offset+offset+j]) {
              commonPrefixLengths[dim] = j;
              break;
            }
          }
        }
      }
  • 对于各维度,排除公共前缀,选择第一个字节唯一值最少的维度,用于在叶子结点内部进行数据排序:
代码语言:txt
复制
MutablePointsReaderUtils.sortByDim(sortedDim, bytesPerDim, commonPrefixLengths,
          reader, from, to, scratchBytesRef1, scratchBytesRef2);
  • 然后先收集docId列表,写入dim文件,具体实现中,会根据docId范围进行基础的变长写入:
代码语言:txt
复制
int[] docIDs = spareDocIds;
for (int i = from; i < to; ++i) {
    docIDs[i - from] = reader.getDocID(i);
}
writeLeafBlockDocs(scratchOut, docIDs, 0, count);
  • 紧接着,写入各维度的公共前缀,即为使用前缀压缩办法降低索引存储开销:
代码语言:txt
复制
reader.getValue(from, scratchBytesRef1);
System.arraycopy(scratchBytesRef1.bytes, scratchBytesRef1.offset, scratch1, 0, packedBytesLength);
writeCommonPrefixes(scratchOut, commonPrefixLengths, scratch1);
  • 最后,排除索引项的前缀部分,把索引项剩余内如写入dim文件,完成叶子节点的写入:
代码语言:txt
复制
writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, count, sortedDim, packedValues);
  • 之后,把BKD-Tree的非叶子节点 及 维度切分信息写入dim文件,完成一个字段的BKD-Tree的写入:
代码语言:txt
复制
writeIndex(out, Math.toIntExact(countPerLeaf), leafBlockFPs, splitPackedValues);

4.2 查询流程

       Lucene中常见的数值类型有Int、Long、Float、Double等,针对数值类型进行等值或条件查询时,如果利用Point索引进行过滤,则会通过如下流程获取到满足查询条件的Doc Id集合:

       其中Scorer对象包含满足查询条件的Doc Id集合。下面以Int类型为例,结合上图描述的流程,具体介绍查询是如何从Point索引中获取结果集的:

  • 首先,对于Int类型的等值查询、范围查询,会在IntPoint类中统一转化为范围查询:
代码语言:txt
复制
public static Query newExactQuery(String field, int value) {
    return newRangeQuery(field, value, value);
}
  • 其中,newRangeQuery会把查询转化为数值类型公用的PointRangeQuery:
代码语言:txt
复制
return new PointRangeQuery(field, pack(lowerValue).bytes, pack(upperValue).bytes, lowerValue.length) {
    @Override
    protected String toString(int dimension, byte[] value) {
        return Integer.toString(decodeDimension(value, 0));
    }
};
  • PointRangeQuery通过createWeight()获取ConstantScoreWeight匿名类对象weight,然后weight通过scorerSupplier()获取ScorerSupplier匿名类对象scorerSupplier,再通过scorerSupplier的get()函数获取Scorer对象,下面为get()的核心部分:
代码语言:txt
复制
values.intersect(field, visitor);
DocIdSetIterator iterator = result.build().iterator();
return new ConstantScoreScorer(weight, score(), iterator);
  • 其中,values为Lucene60PointsReader类对象,通过intersect()读取Point索引,获取满足条件的Doc Id集合,然后封返回Scorer。values.intersect()会获取对应字段的BKDReader,遍历BKD-Tree获取满足条件的文档集合。
代码语言:txt
复制
BKDReader bkdReader = getBKDReader(fieldName);
……
bkdReader.intersect(visitor);
  • BKDReader.intersect()会从BKD-Tree的根节点开始遍历,比较当前子树的取值区间cellMinPacked, cellMaxPacked 和 查询条件区间的关系,根据区间关系进行处理,常见处理分支如代码注释:
代码语言:txt
复制
Relation r = state.visitor.compare(cellMinPacked, cellMaxPacked);
if (r == Relation.CELL_OUTSIDE_QUERY) {
    // 1. 区间不相交,没有符合条件的文档,该子树被优化裁剪掉
} else if (r == Relation.CELL_INSIDE_QUERY) {
    // 2. 查询区间包含子树取值区间,所有文档均匹配,加入结果集
    addAll(state, false);
} else if (state.index.isLeafNode()) {    
    // 3. 查询区间和子树取值区间相交,且当前节点是叶子节点,读取叶子节点内容进行过滤
    if (state.index.nodeExists()) {
        int count = readDocIDs(state.in, state.index.getLeafBlockFP(), state.scratchDocIDs);
        visitDocValues(state.commonPrefixLengths, state.scratchPackedValue, state.in, state.scratchDocIDs, count, state.visitor);
    }
} else {
	// 4. 查询区间和子树取值区间相交,且当前节点是非叶子节点,会进行递归遍历
}
  • 对于第4种分支情况,BKDReader.intersect()会计算左右子树的取值区间,然后递归处理左右子树:
代码语言:txt
复制
int splitDim = state.index.getSplitDim();
byte[] splitPackedValue = state.index.getSplitPackedValue();
BytesRef splitDimValue = state.index.getSplitDimValue();
// 1. 计算左子树取值区间,递归处理左子树
System.arraycopy(cellMaxPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushLeft();
intersect(state, cellMinPacked, splitPackedValue);
state.index.pop();
……
// 2. 计算右子树取值区间,递归处理右子树
System.arraycopy(cellMinPacked, 0, splitPackedValue, 0, packedBytesLength);
System.arraycopy(splitDimValue.bytes, splitDimValue.offset, splitPackedValue, splitDim*bytesPerDim, bytesPerDim);
state.index.pushRight();
intersect(state, splitPackedValue, cellMaxPacked);
state.index.pop();
  • 对于第3种分支情况,BKDReader.readDocIDs()会结合leaf block偏移,读取所有其中的Doc Id集合:
代码语言:txt
复制
in.seek(blockFP);
// How many points are stored in this leaf cell:
int count = in.readVInt();
if (version < BKDWriter.VERSION_COMPRESSED_DOC_IDS) {
  DocIdsWriter.readInts32(in, count, docIDs);
} else {
  DocIdsWriter.readInts(in, count, docIDs);
}
  • 然后每个Doc Id读取point value,通过PointRangeQuery$ConstantScoreWeight$IntersectVisitor的visit()方法进行条件过滤,保留符合条件的Doc Id:
代码语言:txt
复制
public void visit(int docID, byte[] packedValue) {
    for(int dim=0;dim<numDims;dim++) {
        int offset = dim*bytesPerDim;
        if (StringHelper.compare(bytesPerDim, packedValue, offset, lowerPoint, offset) < 0) {
            return;  // point value比查询条件左边界lowerPoint小
        }
        if (StringHelper.compare(bytesPerDim, packedValue, offset, upperPoint, offset) > 0) {
            return;  // point value比查询条件右边界lowerPoint大
        }
    }

    adder.add(docID);
}
  • 当BKD-Tree遍历完时,即可获得符合条件的Doc Id集合。

4.3 相关说明

       到这里,我们已经完成Point索引读写流程的介绍,这里对读写过程中的一些特殊点做如下说明:

  • 维度:本文以二维场景做说明,方便读者理解更一般化的Point索引处理流程。但在实际场景中,我们更多的接触的是一维场景,即便是ES中整形字段包含多值的情况,也是被按值拆分为多个point,因此仍属于一维场景。在实际实现中,Lucene对一维场景做了优化,但总体思路不变,感兴趣的读者可以阅读相应源码。
  • 内存开销:在写入时,Point索引会先缓存在内存中,再周期性的刷新到磁盘,这里对内存有一定的开销,但由于ES会根据内存使用触发刷新,索引内存开销并不是明显问题。但是在多Segment merge过程中,由于历史Segment可能很大,其Point索引达到几百MB甚至GB级,Lucene在实现过程中,会根据需要进行落盘处理,避免过高的内存开销。

5. 小结

       本文主要介绍Point索引的基本概念及其底层存储结构,并结合Point的写入、查询流程进行详细解析。Lucene写入/查询的总体流程、Term索引/行存储/列存储等核心数据结构在本文中暂未提及,后续会有文章作详细介绍。

原创声明:本文系作者授权腾讯云开发者社区发表,未经许可,不得转载。

如有侵权,请联系 cloudcommunity@tencent.com 删除。

原创声明:本文系作者授权腾讯云开发者社区发表,未经许可,不得转载。

如有侵权,请联系 cloudcommunity@tencent.com 删除。

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
目录
  • 1. 背景
  • 2. 简介
    • 2.1 BKD-Tree简介
      • 2.2 数值索引的基本原理
      • 3. 文件结构
      • 4. 读写流程
        • 4.1 写入流程
          • 4.1.1 写入数据到内存
          • 4.1.2 flush生成Segment
          • 4.1.3 BKD-Tree构建过程
        • 4.2 查询流程
          • 4.3 相关说明
          • 5. 小结
          相关产品与服务
          文件存储
          文件存储(Cloud File Storage,CFS)为您提供安全可靠、可扩展的共享文件存储服务。文件存储可与腾讯云服务器、容器服务、批量计算等服务搭配使用,为多个计算节点提供容量和性能可弹性扩展的高性能共享存储。腾讯云文件存储的管理界面简单、易使用,可实现对现有应用的无缝集成;按实际用量付费,为您节约成本,简化 IT 运维工作。
          领券
          问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档