You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by GitBox <gi...@apache.org> on 2019/08/13 13:34:06 UTC

[GitHub] [lucene-solr] bruno-roustant commented on a change in pull request #633: LUCENE-8753 UniformSplit PostingsFormat

bruno-roustant commented on a change in pull request #633: LUCENE-8753 UniformSplit PostingsFormat
URL: https://github.com/apache/lucene-solr/pull/633#discussion_r313398353
 
 

 ##########
 File path: lucene/codecs/src/java/org/apache/lucene/codecs/uniformsplit/BlockReader.java
 ##########
 @@ -0,0 +1,535 @@
+/*
+ * 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.lucene.codecs.uniformsplit;
+
+import java.io.IOException;
+import java.util.function.Supplier;
+
+import org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.lucene50.DeltaBaseTermStateSerializer;
+import org.apache.lucene.index.BaseTermsEnum;
+import org.apache.lucene.index.ImpactsEnum;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.RamUsageEstimator;
+
+/**
+ * Seeks the block corresponding to a given term, read the block bytes, and
+ * scans the block terms.
+ * <p>
+ * Reads fully the block in {@link #blockReadBuffer}. Then scans the block
+ * terms in memory. The details region is lazily decoded with {@link #termStatesReadBuffer}
+ * which shares the same byte array with {@link #blockReadBuffer}.
+ * See {@link BlockWriter} and {@link BlockLine} for the block format.
+ */
+public class BlockReader extends BaseTermsEnum implements Accountable {
+
+  private static final long BASE_RAM_USAGE = RamUsageEstimator.shallowSizeOfInstance(BlockReader.class)
+      + RamUsageEstimator.shallowSizeOfInstance(IndexInput .class)
+      +RamUsageEstimator.shallowSizeOfInstance(ByteArrayDataInput .class)*2;
+
+  /**
+   * {@link IndexInput} on the {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}.
+   */
+  protected IndexInput blockInput;
+  protected final PostingsReaderBase postingsReader;
+  protected final FieldMetadata fieldMetadata;
+  protected final BlockDecoder blockDecoder;
+
+  protected BlockLine.Serializer blockLineReader;
+  /**
+   * In-memory read buffer for the current block.
+   */
+  protected ByteArrayDataInput blockReadBuffer;
+  /**
+   * In-memory read buffer for the details region of the current block.
+   * It shares the same byte array as {@link #blockReadBuffer}, with a
+   * different position.
+   */
+  protected ByteArrayDataInput termStatesReadBuffer;
+  protected DeltaBaseTermStateSerializer termStateSerializer;
+
+  /**
+   * {@link IndexDictionary.Browser} supplier for lazy loading.
+   */
+  protected final Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier;
+  /**
+   * Holds the {@link IndexDictionary.Browser} once loaded.
+   */
+  protected IndexDictionary.Browser dictionaryBrowser;
+
+  /**
+   * Current block start file pointer, absolute in the
+   * {@link UniformSplitPostingsFormat#TERMS_BLOCKS_EXTENSION block file}.
+   */
+  protected long blockStartFP;
+  /**
+   * Current block header.
+   */
+  protected BlockHeader blockHeader;
+  /**
+   * Current block line.
+   */
+  protected BlockLine blockLine;
+  /**
+   * Current block line details.
+   */
+  protected BlockTermState termState;
+  /**
+   * Offset of the start of the first line of the current block (just after the header), relative to the block start.
+   */
+  protected int blockFirstLineStart;
+  /**
+   * Current line index in the block.
+   */
+  protected int lineIndexInBlock;
+  /**
+   * Whether the current {@link TermState} has been forced with a call to
+   * {@link #seekExact(BytesRef, TermState)}.
+   *
+   * @see #forcedTerm
+   */
+  protected boolean termStateForced;
+  /**
+   * Set when {@link #seekExact(BytesRef, TermState)} is called.
+   * <p>
+   * This optimizes the use-case when the caller calls first {@link #seekExact(BytesRef, TermState)}
+   * and then {@link #postings(PostingsEnum, int)}. In this case we don't access
+   * the terms block file (we don't seek) but directly the postings file because
+   * we already have the {@link TermState} with the file pointers to the postings
+   * file.
+   */
+  protected BytesRefBuilder forcedTerm;
+
+  // Scratch objects to avoid object reallocation.
+  protected BytesRef scratchBlockBytes;
+  protected final BlockTermState scratchTermState;
+
+  /**
+   * @param dictionaryBrowserSupplier to load the {@link IndexDictionary.Browser}
+   *                                  lazily in {@link #seekCeil(BytesRef)}.
+   * @param blockDecoder              Optional block decoder, may be null if none.
+   *                                  It can be used for decompression or decryption.
+   */
+  protected BlockReader(Supplier<IndexDictionary.Browser> dictionaryBrowserSupplier, IndexInput blockInput,
+                        PostingsReaderBase postingsReader, FieldMetadata fieldMetadata,
+                        BlockDecoder blockDecoder) throws IOException {
+    this.dictionaryBrowserSupplier = dictionaryBrowserSupplier;
+    this.blockInput = blockInput;
+    this.postingsReader = postingsReader;
+    this.fieldMetadata = fieldMetadata;
+    this.blockDecoder = blockDecoder;
+    this.blockStartFP = -1;
+    scratchTermState = postingsReader.newTermState();
+  }
+
+  @Override
+  public SeekStatus seekCeil(BytesRef searchedTerm) throws IOException {
+    if (isCurrentTerm(searchedTerm)) {
+      return SeekStatus.FOUND;
+    }
+    clearTermState();
+
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(searchedTerm);
+    blockStartFP = Math.max(blockStartFP, fieldMetadata.getFirstBlockStartFP());
+    if (isBeyondLastTerm(searchedTerm, blockStartFP)) {
+      return SeekStatus.END;
+    }
+    SeekStatus seekStatus = seekInBlock(searchedTerm, blockStartFP);
+    if (seekStatus != SeekStatus.END) {
+      return seekStatus;
+    }
+    // Go to next block.
+    return nextTerm() == null ? SeekStatus.END : SeekStatus.NOT_FOUND;
+  }
+
+  @Override
+  public boolean seekExact(BytesRef searchedTerm) throws IOException {
+    if (isCurrentTerm(searchedTerm)) {
+      return true;
+    }
+    clearTermState();
+
+    long blockStartFP = getOrCreateDictionaryBrowser().seekBlock(searchedTerm);
+    if (blockStartFP < fieldMetadata.getFirstBlockStartFP() || isBeyondLastTerm(searchedTerm, blockStartFP)) {
+      return false;
+    }
+    return seekInBlock(searchedTerm, blockStartFP) == SeekStatus.FOUND;
+  }
+
+  protected boolean isCurrentTerm(BytesRef searchedTerm) {
+    // Optimization and also required to not search with the same BytesRef
+    // instance as the BytesRef used to read the block line (BlockLine.Serializer).
+    // Indeed getCurrentTerm() is allowed to return the same BytesRef instance.
+    return searchedTerm.equals(term());
+  }
+
+  /**
+   * Indicates whether the searched term is beyond the last term of the field.
+   *
+   * @param blockStartFP The current block start file pointer.
+   */
+  protected boolean isBeyondLastTerm(BytesRef searchedTerm, long blockStartFP) {
+    return blockStartFP == fieldMetadata.getLastBlockStartFP()
+        && searchedTerm.compareTo(fieldMetadata.getLastTerm()) > 0;
+  }
+
+  /**
+   * Seeks to the provided term in the block starting at the provided file pointer.
+   * Does not exceed the block.
+   */
+  protected SeekStatus seekInBlock(BytesRef searchedTerm, long blockStartFP) throws IOException {
+    initializeHeader(searchedTerm, blockStartFP);
+    assert blockHeader != null;
+    return seekInBlock(searchedTerm);
+  }
+
+  /**
+   * Seeks to the provided term in this block.
+   * <p>
+   * Does not exceed this block; {@link TermsEnum.SeekStatus#END} is returned if it follows the block.
+   * <p>
+   * Compares the line terms with the <code>searchedTerm</code>, taking
+   * advantage of the incremental encoding properties.
+   * <p>
+   * Scans linearly the terms. Updates the current block line with the current
+   * term.
+   */
+  protected SeekStatus seekInBlock(BytesRef searchedTerm) throws IOException {
+    if (compareToMiddleAndJump(searchedTerm) == 0) {
+      return SeekStatus.FOUND;
+    }
+    int comparisonOffset = 0;
+    while (true) {
+      if (readLineInBlock() == null) {
+        // No more terms for the block.
+        return SeekStatus.END;
+      }
+      TermBytes lineTermBytes = blockLine.getTermBytes();
+      BytesRef lineTerm = lineTermBytes.getTerm();
+      assert lineTerm.offset == 0;
+
+      // Equivalent to comparing with BytesRef.compareTo(),
+      // but faster since we start comparing from min(comparisonOffset, suffixOffset).
 
 Review comment:
   There is an optimization with comparisonOffset in addition to the regular byte comparison. I have to benchmark to determine if Arrays.compareUnsigned is faster anyway.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org