从源码看 Lucene 的文档写入流程

政采云技术团队.png

土根1.png

概念一览

在阅读源码前,这里先对源码中比较重要、常用的几个类做一个简单介绍,以便有一个初步概念,更容易的理解整个流程。

流程图导出 (18).png

IndexWriter:Lucene 写入文档主入口,除此之外,还承担了索引的创建和维护工作。

DocumentsWriter:该类被用来添加文档,直接写入 segment 文件。DocumentsWriter 允许多个线程同时写入数据,每个线程都会从 DocumentsWriterPerThreadPool 中获得一个 ThreadState 对象。

DocumentsWriterPerThreadPool:后文及源码中简称为 DWPTP ,内部维护了一个 List<ThreadState>,用来实现 ThreadState 池。

ThreadState:从 DWPTP 中获得,当 ThreadState 使用完后,会回到 DWPTP 中等待下一次使用。每一个 ThreadState 都持有一个 DocumentsWriterPerThread 的引用,DocumentsWriterPerThread 开始 flush 时,失去引用。

DocumentsWriterPerThread:后文及源码中简称为 DWPT,该类用于将索引实际处理成文件,并生成 segment。

整体流程

在粗略了解 Lucene 文档写入过程中比较核心的几个类后,我们跟着源码来探究一下他们具体做了什么。

流程图导出 (13).png

updateDocument流程图
  /**
   * Lucene写入/更新文档
   * @param docs 待写入的文档
   * @param analyzer 分词器
   * @param delTerm 待删除待文档
   * @return
   * @throws IOException
   * @throws AbortingException
   */  
	long updateDocument(final Iterable<? extends IndexableField> doc, final Analyzer analyzer,
      final Term delTerm) throws IOException, AbortingException {

    boolean hasEvents = preUpdate();
    // 从 DWPTP 中获取 ThreadState
    final ThreadState perThread = flushControl.obtainAndLock();

    final DocumentsWriterPerThread flushingDWPT;
    long seqNo;
    try {
      // 校验索引状态(不能为 close)
      ensureOpen();
      // 校验ThreadState是否持有DWPT,没有的话初始化一个
      ensureInitialized(perThread);
      assert perThread.isInitialized();
      final DocumentsWriterPerThread dwpt = perThread.dwpt;
      final int dwptNumDocs = dwpt.getNumDocsInRAM();
      try {
        // 写入文档
        seqNo = dwpt.updateDocument(doc, analyzer, delTerm); 
      } catch (AbortingException ae) {
        // 若写入异常
        // 回滚相关参数 并回收 ThreadState 和 DWPT
        flushControl.doOnAbort(perThread);
        dwpt.abort();
        throw ae;
      } finally {
        // 由于每次写入都有可能触发flush 因此内存中的文档数量需要重新计算
        numDocsInRAM.addAndGet(dwpt.getNumDocsInRAM() - dwptNumDocs);
      }
      final boolean isUpdate = delTerm != null;
      // 收集满足 flush 条件的线程
      // 比对 flush 策略
      // 若内存中的文档数超过最大文档数(默认 -1 不开启) 或着 内存中的文档大小超过最大文档大小(默认16mb)
      // 则标记 flushPending 等待 flush
      flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate);

      assert seqNo > perThread.lastSeqNo: "seqNo=" + seqNo + " lastSeqNo=" + perThread.lastSeqNo;
      perThread.lastSeqNo = seqNo;

    } finally {
      // 回收 ThreadState
      perThreadPool.release(perThread);
    }

    // postUpdate flush 数据(如果需要的话)
    if (postUpdate(flushingDWPT, hasEvents)) {
      seqNo = -seqNo;
    }   
    return seqNo;
  }

preUpdate 写入文档前的工作

首先是 preUpdate 方法,见名知意,该方法做了一些文档写入前的准备工作。

主要做了两件事:

  • 校验索引状态(不能为 close)
  • 检查 flushQueue 中是否有待 flush 的 DWPT(有则 flush)
  • 检查是否需要阻塞(1秒)

流程图导出 (14).png

preUpdate流程图
  private boolean preUpdate() throws IOException, AbortingException {
    // 首先校验要写入的索引是开启的
    ensureOpen();
    boolean hasEvents = false;
    // 判断flush队列中是否有可 flush 的 DWPT
    if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
      do {
        DocumentsWriterPerThread flushingDWPT;
        // 循环取出 flush 的 DWPT
        while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
          // flush
          hasEvents |= doFlush(flushingDWPT);
        }
        
        // 根据文档写入速度和 flush 速度决定是否阻塞写入 
        flushControl.waitIfStalled();
      } while (flushControl.numQueuedFlushes() != 0); 
    }
    return hasEvents;
  }

  /**
	* 获取下一条满足 flush 条件的 DWPT
	* 若 flush 队列中有现成的,直接取出
	* 若没有,则通过dwptp中被标记为等待 flush(flushPending) 的 ThreadState 获取 DWPT
	*/
  DocumentsWriterPerThread nextPendingFlush() {
    int numPending;
    boolean fullFlush;
    synchronized (this) {
      final DocumentsWriterPerThread poll;
      if ((poll = flushQueue.poll()) != null) {
        updateStallState();
        return poll;
      }
      fullFlush = this.fullFlush;
      numPending = this.numPending;
    }
    if (numPending > 0 && !fullFlush) {
      final int limit = perThreadPool.getActiveThreadStateCount();
      for (int i = 0; i < limit && numPending > 0; i++) {
        final ThreadState next = perThreadPool.getThreadState(i);
        if (next.flushPending) {
          final DocumentsWriterPerThread dwpt = tryCheckoutForFlush(next);
          if (dwpt != null) {
            return dwpt;
          }
        }
      }
    }
    return null;
  }
 
	/**
	* 根据文档写入速度和 flush 速度决定是否阻塞写入
	* 若 flush 产生堆积,则阻塞写入 1 秒
	* 以防止写入过快时,内存增长过快
	* activeBytes:添加/更新的 byte 数
	* flushBytes:等待 flush 的 byte 数
	* limit = 2* ramBufferSizeMB = 32mb
	*/
  void waitIfStalled() {
    final boolean stall = (activeBytes + flushBytes) > limit && activeBytes < limit;
    if (stalled) {
      synchronized (this) {
        if (stalled) {
          try {
            incWaiters();
            wait(1000);
            decrWaiters();
          } catch (InterruptedException e) {
            throw new ThreadInterruptedException(e);
          }
        }
      }
    }
  }

可以看到,为了避免 flush 队列出现堆积,Lucene 在每次写入文档前都会判断是否有未 flush 的 DWPT,并在 flush 之后,通过新增文档和待 flush 文档的大小判断 flush 是否出现堆积,决定是否阻塞写入1秒。很明显,此处Lucene的文档最快写入速度受限于 ramBufferSizeMB 的大小。实际上 ramBufferSizeMB 还被用来判断是否需要 flush,这一点在后文的 doAfterDocument 中会有体现。因此,适当的提高 ramBufferSizeMB 大小有助于提高文档写入速度。

obtainAndLock 获取处理文档的 DWPT

流程图导出 (19).png

getAndLock流程图

可以看到,dwptp 维护了一个空闲 ThreadState 列表 freeList。

当空闲列表 freeList 为空,此时只能新生成一个 ThreadState。

当空闲列表 freeList 不为空时,优先从列表末尾(后进先出)取一个 ThreadState。

若此 ThreadState 不包含 DWPT 的引用,则从freeList寻找一个 DWPT 不为空的 ThreadState。

这是为了尽量优先使用被使用过的 DWPT,以便 DWPT 尽快达到 flush 阈值后释放内存生成段。

  ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter) {
      ThreadState threadState = null;
      synchronized (this) {
        if (freeList.isEmpty()) {
          return newThreadState();
        } else {
          threadState = freeList.remove(freeList.size()-1);
          if (threadState.dwpt == null) {
            for(int i=0;i<freeList.size();i++) {
              ThreadState ts = freeList.get(i);
              if (ts.dwpt != null) {
                freeList.set(i, threadState);
                threadState = ts;
                break;
              }
            }
          }
        }
      }
      threadState.lock();
      return threadState;
  }

updateDocument 写入数据

流程图导出 (15).png

updateDocument流程图
/**
* 写入文档
* docs:要写入的文档
* analyzer:分词器
* delTerm:要删除的文档
*/
  public long updateDocument(Iterable<? extends IndexableField> doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
    testPoint("DocumentsWriterPerThread addDocument start");
    assert deleteQueue != null;
    // 判断是否超过 Lucene 的最大文档数(Integer.MAX_VALUE - 128)
    // 即 2147483519
    reserveOneDoc();
    docState.doc = doc;
    docState.analyzer = analyzer;
    docState.docID = numDocsInRAM;

    boolean success = false;
    try {
      try {
        // 文档处理器处理文档 本期不再展开
        consumer.processDocument();
      } finally {
        docState.clear();
      }
      success = true;
    } finally {
      if (!success) {
        // 如果写入失败 则标记删除
        deleteDocID(docState.docID);
        numDocsInRAM++;
      }
    }

    // 处理需删除的 term
    // 添加到删除队列
    return finishDocument(delTerm);
  }

从这里可以看到,Lucene 支持的最大文档数是 2147483519,放到 ES 上就是单个分片支持的最大文档数是 2147483519。

doOnAbort 处理异常

  /**
  * 若文档写入出错
  * 则需回滚相关参数
  */
  synchronized void doOnAbort(ThreadState state) {
    try {
      // 若 flush 时失败,回滚 flushBytes,否则回滚 activeBytes
      if (state.flushPending) {
        flushBytes -= state.bytesUsed;
      } else {
        activeBytes -= state.bytesUsed;
      }
      // 重新计算内存
      assert assertMemory();
      // ThreadState 丢弃对 DWPT 的引用
      perThreadPool.reset(state);
    } finally {
      // 重新计算是否需要阻塞
      // final boolean stall = (activeBytes + flushBytes) > limit && activeBytes < limit
      updateStallState();
    }
  }

doAfterDocument 按策略收集 flush 的 dwpt

流程图导出 (16).png

doAfterDocument流程图
  /**
  * 判断、收集满足 flush 条件的 DWPT
  * @param perThread
  * @param isUpdate
  * @return
  */
  synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread, boolean isUpdate) {
    try {
      // 统计文档占用的内存
      commitPerThreadBytes(perThread);
      if (!perThread.flushPending) {
        // 标记符合 flush 条件的 ThreadState
        // delete策略:判断 deleteRamByteUsed 是否达到 ramBufferSizeMB,满足则标记 flushDeletes
        // insert策略:如果设置了 flushOnDocCount,则判断文档数是否达到 maxBufferedDocs
        //						否则判断 activeBytes 与 deleteRamByteUsed 的和是否达到 ramBufferSizeMB,满足则标记 flushPending
        // update策略:同时判断 delete 策略 + insert 策略
        if (isUpdate) {
          flushPolicy.onUpdate(this, perThread);
        } else {
        flushPolicy.onInsert(this, perThread);
        }
        // 单个 DWPT 无法处理超过 2048mb 的数据,因此当文档体积累计到一个阈值 hardMaxBytesPerDWPT(默认1945MB)后
        // 会标记为 flushPending 强制flush
        if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
          setFlushPending(perThread);
        }
        }
      // 返回被标记为 flushPending 的 DWPT
      return checkout(perThread, false);
    } finally {
      // 重新计算是否需要阻塞
      boolean stalled = updateStallState();
      assert assertNumDocsSinceStalled(stalled) && assertMemory();
    }
  }

insert策略

这里支持两种判断策略:

  • 按文档数量(maxBufferedDocs)判断
  • 默认 按文档大小(ramBufferSizeMB)判断

首先判断dwpt内文档数量是否超过上限(maxBufferedDocs(默认-1不判断)),超过的话标记 DWPT 需要 flush。

若 maxBufferedDocs=-1,则根据 RAM 使用情况(默认)判断是否需要 flush。

当 DWPT 内新增和删除的文档大小总和 > ramBufferSizeMB(默认 16MB)时,则标记需要 flush。

  /**
   * insert策略收集需要 flush 的 DWPT
   * @param control
   * @param state
   */
  public void onInsert(DocumentsWriterFlushControl control, ThreadState state) {
    if (flushOnDocCount()
        && state.dwpt.getNumDocsInRAM() >= indexWriterConfig
            .getMaxBufferedDocs()) {
      control.setFlushPending(state);
    } else if (flushOnRAM()) {
      final long limit = (long) (indexWriterConfig.getRAMBufferSizeMB() * 1024.d * 1024.d);
      final long totalRam = control.activeBytes() + control.getDeleteBytesUsed();
      if (totalRam >= limit) {
        if (infoStream.isEnabled("FP")) {
          infoStream.message("FP", "trigger flush: activeBytes=" + control.activeBytes() + " deleteBytes=" + control.getDeleteBytesUsed() + " vs limit=" + limit);
        }
        markLargestWriterPending(control, state, totalRam);
      }
    }
  }

delete策略

当被删除的文档大小超过 ramBufferSizeMB(默认16MB) 时,则标记需要flush以清除这些文档。

  /**
   * delete策略收集需要 flush 的 DWPT
   * @param control
   * @param state
   */
  public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
    if ((flushOnRAM() && control.getDeleteBytesUsed() > 1024*1024*indexWriterConfig.getRAMBufferSizeMB())) {
      control.setApplyAllDeletes();
      if (infoStream.isEnabled("FP")) {
        infoStream.message("FP", "force apply deletes bytesUsed=" + control.getDeleteBytesUsed() + " vs ramBufferMB=" + indexWriterConfig.getRAMBufferSizeMB());
      }
    }
  }

update策略

即 insert 策略 + delete 策略

  public void onUpdate(DocumentsWriterFlushControl control, ThreadState state) {
    onInsert(control, state);
    onDelete(control, state);
  }

postUpdate flush数据

流程图导出 (17).png

postUpdate流程图
  /**
  * 冻结待删除待文档并添加到删除队列
  * 根据 DWPT 状态判断是否需要 flush
  * @param flushingDWPT
  * @param hasEvents
  * @return
  * @throws IOException
  * @throws AbortingException
  */
  private boolean postUpdate(DocumentsWriterPerThread flushingDWPT, boolean hasEvents) throws IOException,   AbortingException {
    // 将所有待删除的数据加入到 flushTicket 队列中
    // private final Queue<FlushTicket> queue = new LinkedList<>();
    // queue.add(new GlobalDeletesTicket(deleteQueue.freezeGlobalBuffer(null)));
    hasEvents |= applyAllDeletes(deleteQueue);
    if (flushingDWPT != null) {
      hasEvents |= doFlush(flushingDWPT);
    } else {
      final DocumentsWriterPerThread nextPendingFlush = flushControl.nextPendingFlush();
      if (nextPendingFlush != null) {
        hasEvents |= doFlush(nextPendingFlush);
      }
    }

    return hasEvents;
  }
  /**
  * flush
  * @param flushingDWPT
  * @return
  * @throws IOException
  * @throws AbortingException
  */
  private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException, AbortingException {
      boolean hasEvents = false;
      while (flushingDWPT != null) {
        hasEvents = true;
        boolean success = false;
        SegmentFlushTicket ticket = null;
        try {
          assert currentFullFlushDelQueue == null
              || flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: "
              + currentFullFlushDelQueue + "but was: " + flushingDWPT.deleteQueue
              + " " + flushControl.isFullFlush();
          try {
            //  该方法内部 将要被删除的信息封装成 FrozenBufferedUpdates 并生成 SegmentFlushTicket 后加入 flushQueue
            ticket = ticketQueue.addFlushTicket(flushingDWPT);
            // 该 DWPT 中需要被 flush 的文档数量
            final int flushingDocsInRam = flushingDWPT.getNumDocsInRAM();
            boolean dwptSuccess = false;
            try {
              // 一个 flush 线程生成一个 segment 同步生成 segment 异步清除 in-memory buffer
              final FlushedSegment newSegment = flushingDWPT.flush();
              ticketQueue.addSegment(ticket, newSegment);
              dwptSuccess = true;
            } finally {
              // 循环等待 buffer 中的文档刷完
              // while (numDocsInRAM.compareAndSet(oldValue, oldValue - numFlushed) == false)
              subtractFlushedNumDocs(flushingDocsInRam);
              if (flushingDWPT.pendingFilesToDelete().isEmpty() == false) {
                // 如果使用复合索引文件存储索引信息(useCompoundFile=true)
                // 在生成段后 需要删除原先的非复合索引文件
                putEvent(new DeleteNewFilesEvent(flushingDWPT.pendingFilesToDelete()));
                hasEvents = true;
              }
              if (dwptSuccess == false) {
                // flush 失败
                // 虽然没有最终生成 segment 对象
                // 但此时有可能已经生成索引文件 因此需要尝试删除索引文件
                putEvent(new FlushFailedEvent(flushingDWPT.getSegmentInfo()));
                hasEvents = true;
              }
            }
            // 至此 flush 成功
            success = true;
          } finally {
            if (!success && ticket != null) {
              // 标记 flush 失败并置空 segment
              ticketQueue.markTicketFailed(ticket);
            }
          }
          // 如果 flush 队列中的 DWPT 数超过 DWPTP 中 ThreadState 的个数,则 flush 队列出现堆积 
          // 这是因为一个 TreadState 只能持有一个 DWPT 引用,并且在开始 flush 时失去 DWPT 的引用并回到 DWPTP 中
          // 当下一次写入请求进来 TreadState 重新获取新的 DWPT,而被失去引用的 DWPT 仍在 doflush 中,未被回收
          // 此时 DWPT 数量超过 ThreadState,可认为出现堆积
          if (ticketQueue.getTicketCount() >= perThreadPool.getActiveThreadStateCount()) {
            // 强制加锁发布生成的段
            putEvent(ForcedPurgeEvent.INSTANCE);
            break;
          }
        } finally {
          Long bytes = flushingWriters.remove(dwpt);
          flushBytes -= bytes.longValue();
          perThreadPool.recycle(dwpt);
          // 到此 flush 主流程已结束 
          // 该方法做了以下三件事:
          // DWPT 从 flush 缓存中移除
          // 重新计算已被 flush 的文档大小
          // 回收 DWPT(默认不回收)
          flushControl.doAfterFlush(flushingDWPT);
        }
       
        flushingDWPT = flushControl.nextPendingFlush();
      }
  
      if (hasEvents) {
        // 尝试加锁发布生成的段
        writer.doAfterSegmentFlushed(false, false);
      }

      final double ramBufferSizeMB = config.getRAMBufferSizeMB();
      // 当 activeBytes + deleteBytesUsed >= ramBufferSizeMB 时触发自动 flush
      // 因此 内存中的删除信息越多 触发 flush 会越频繁,生成越多的小段
      // 否则会导致频繁的 flush 小段
      if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
          flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
        hasEvents = true;
        if (applyAllDeletes(deleteQueue) == false) {
          if (infoStream.isEnabled("DW")) {
            infoStream.message("DW", String.format(Locale.ROOT, "force apply deletes after flush bytesUsed=%.1f MB vs ramBuffer=%.1f MB",flushControl.getDeleteBytesUsed()/(1024.*1024.), ramBufferSizeMB));
          }
          // 设置删除事件 再下一次 flush 或者 merge 时,会优先处理被删除的数据
          putEvent(ApplyDeletesEvent.INSTANCE);
        }
      }
      return hasEvents;
    }

总结

到此,Luncen 的文档写入的大致流程已经梳理完了。可以看到为了提高写入速率,Lucene 做了许多优化。

为了提高资源利用率,Lucene 的写入采用并发模型,整个写入流程都是线程安全的。其中最消耗资源的地方莫过于 dwpt(分词、相关性计算、索引构建等),而每个 DWPT 都拥有自己独立的内存空间来处理数据。因此 dwpt 是天然线程安全不需要加锁来损失性能。

为了提高资源利用率,Lucene 设计了一个 DWPTP 来复用 ThreadState 和 DWPT 对象,引入 freeList 优化了这些对象的回收和复用逻辑。

同时,为了避免频繁被动触发 flush 生成长尾小段影响检索性能,Lucene 对内存中被删除的文档大小做了严格的限制。

参考文献

lucene源码

Chris 的小屋

推荐阅读

ElasticSearch 文档分值 score 计算&聚合搜索案例分析

Label Studio+Yolov5 实现目标检测预标注(二)

Label Studio+Yolov5 实现目标检测预标注(一)

业务交互网关洪峰应对之道

招贤纳士

政采云技术团队(Zero),一个富有激情、创造力和执行力的团队,Base 在风景如画的杭州。团队现有300多名研发小伙伴,既有来自阿里、华为、网易的“老”兵,也有来自浙大、中科大、杭电等校的新人。团队在日常业务开发之外,还分别在云原生、区块链、人工智能、低代码平台、中间件、大数据、物料体系、工程平台、性能体验、可视化等领域进行技术探索和实践,推动并落地了一系列的内部技术产品,持续探索技术的新边界。此外,团队还纷纷投身社区建设,目前已经是 google flutter、scikit-learn、Apache Dubbo、Apache Rocketmq、Apache Pulsar、CNCF Dapr、Apache DolphinScheduler、alibaba Seata 等众多优秀开源社区的贡献者。如果你想改变一直被事折腾,希望开始折腾事;如果你想改变一直被告诫需要多些想法,却无从破局;如果你想改变你有能力去做成那个结果,却不需要你;如果你想改变你想做成的事需要一个团队去支撑,但没你带人的位置;如果你想改变本来悟性不错,但总是有那一层窗户纸的模糊……如果你相信相信的力量,相信平凡人能成就非凡事,相信能遇到更好的自己。如果你希望参与到随着业务腾飞的过程,亲手推动一个有着深入的业务理解、完善的技术体系、技术创造价值、影响力外溢的技术团队的成长过程,我觉得我们该聊聊。任何时间,等着你写点什么,发给 zcy-tc@cai-inc.com

微信公众号

文章同步发布,政采云技术团队公众号,欢迎关注

政采云技术团队.png