You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/11/20 15:16:22 UTC

[12/31] lucene-solr:jira/http2: LUCENE-8470: Remove legacy doc values APIs from lucene/core.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3302ed2a/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
deleted file mode 100644
index a75274e..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/LegacySortedNumericDocValuesWrapper.java
+++ /dev/null
@@ -1,98 +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.lucene.index;
-
-import java.io.IOException;
-
-/**
- * Wraps a {@link LegacySortedNumericDocValues} into a {@link SortedNumericDocValues}.
- *
- * @deprecated Implement {@link SortedNumericDocValues} directly.
- */
-@Deprecated
-public final class LegacySortedNumericDocValuesWrapper extends SortedNumericDocValues {
-  private final LegacySortedNumericDocValues values;
-  private final int maxDoc;
-  private int docID = -1;
-  private int upto;
-  
-  public LegacySortedNumericDocValuesWrapper(LegacySortedNumericDocValues values, int maxDoc) {
-    this.values = values;
-    this.maxDoc = maxDoc;
-  }
-
-  @Override
-  public int docID() {
-    return docID;
-  }
-
-  @Override
-  public int nextDoc() {
-    assert docID != NO_MORE_DOCS;
-    while (true) {
-      docID++;
-      if (docID == maxDoc) {
-        docID = NO_MORE_DOCS;
-        break;
-      }
-      values.setDocument(docID);
-      if (values.count() != 0) {
-        break;
-      }
-    }
-    upto = 0;
-    return docID;
-  }
-
-  @Override
-  public int advance(int target) {
-    if (target < docID) {
-      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
-    }
-    if (target >= maxDoc) {
-      docID = NO_MORE_DOCS;
-    } else {
-      docID = target-1;
-      nextDoc();
-    }
-    return docID;
-  }
-
-  @Override
-  public boolean advanceExact(int target) throws IOException {
-    docID = target;
-    values.setDocument(docID);
-    upto = 0;
-    return values.count() != 0;
-  }
-
-  @Override
-  public long cost() {
-    return 0;
-  }
-
-  @Override
-  public long nextValue() {
-    return values.valueAt(upto++);
-  }
-
-  @Override
-  public int docValueCount() {
-    return values.count();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3302ed2a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
deleted file mode 100644
index 0c6c809..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValues.java
+++ /dev/null
@@ -1,111 +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.lucene.index;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.util.BytesRef;
-
-/**
- * A per-document set of presorted byte[] values.
- * <p>
- * Per-Document values in a SortedDocValues are deduplicated, dereferenced,
- * and sorted into a dictionary of unique values. A pointer to the
- * dictionary value (ordinal) can be retrieved for each document. Ordinals
- * are dense and in increasing sorted order.
- *
- * @deprecated Use {@link SortedSetDocValues} instead.
- */
-@Deprecated
-public abstract class LegacySortedSetDocValues {
-  
-  /** Sole constructor. (For invocation by subclass 
-   * constructors, typically implicit.) */
-  protected LegacySortedSetDocValues() {}
-
-  /** When returned by {@link #nextOrd()} it means there are no more 
-   *  ordinals for the document.
-   */
-  public static final long NO_MORE_ORDS = -1;
-
-  /** 
-   * Returns the next ordinal for the current document (previously
-   * set by {@link #setDocument(int)}.
-   * @return next ordinal for the document, or {@link #NO_MORE_ORDS}. 
-   *         ordinals are dense, start at 0, then increment by 1 for 
-   *         the next value in sorted order. 
-   */
-  public abstract long nextOrd();
-  
-  /** 
-   * Sets iteration to the specified docID 
-   * @param docID document ID 
-   */
-  public abstract void setDocument(int docID);
-
-  /** Retrieves the value for the specified ordinal. The returned
-   * {@link BytesRef} may be re-used across calls to lookupOrd so make sure to
-   * {@link BytesRef#deepCopyOf(BytesRef) copy it} if you want to keep it
-   * around.
-   * @param ord ordinal to lookup
-   * @see #nextOrd
-   */
-  public abstract BytesRef lookupOrd(long ord);
-
-  /**
-   * Returns the number of unique values.
-   * @return number of unique values in this SortedDocValues. This is
-   *         also equivalent to one plus the maximum ordinal.
-   */
-  public abstract long getValueCount();
-
-  /** If {@code key} exists, returns its ordinal, else
-   *  returns {@code -insertionPoint-1}, like {@code
-   *  Arrays.binarySearch}.
-   *
-   *  @param key Key to look up
-   **/
-  public long lookupTerm(BytesRef key) {
-    long low = 0;
-    long high = getValueCount()-1;
-
-    while (low <= high) {
-      long mid = (low + high) >>> 1;
-      final BytesRef term = lookupOrd(mid);
-      int cmp = term.compareTo(key);
-
-      if (cmp < 0) {
-        low = mid + 1;
-      } else if (cmp > 0) {
-        high = mid - 1;
-      } else {
-        return mid; // key found
-      }
-    }
-
-    return -(low + 1);  // key not found.
-  }
-  
-  /** 
-   * Returns a {@link TermsEnum} over the values.
-   * The enum supports {@link TermsEnum#ord()} and {@link TermsEnum#seekExact(long)}.
-   */
-  public TermsEnum termsEnum() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3302ed2a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java b/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
deleted file mode 100644
index 0e96e02..0000000
--- a/lucene/core/src/java/org/apache/lucene/index/LegacySortedSetDocValuesWrapper.java
+++ /dev/null
@@ -1,112 +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.lucene.index;
-
-import java.io.IOException;
-
-import org.apache.lucene.util.BytesRef;
-
-/**
- * Wraps a {@link LegacySortedSetDocValues} into a {@link SortedSetDocValues}.
- *
- * @deprecated Implement {@link SortedSetDocValues} directly.
- */
-@Deprecated
-public final class LegacySortedSetDocValuesWrapper extends SortedSetDocValues {
-  private final LegacySortedSetDocValues values;
-  private final int maxDoc;
-  private int docID = -1;
-  private long ord;
-  
-  public LegacySortedSetDocValuesWrapper(LegacySortedSetDocValues values, int maxDoc) {
-    this.values = values;
-    this.maxDoc = maxDoc;
-  }
-
-  @Override
-  public int docID() {
-    return docID;
-  }
-
-  @Override
-  public int nextDoc() {
-    assert docID != NO_MORE_DOCS;
-    docID++;
-    while (docID < maxDoc) {
-      values.setDocument(docID);
-      ord = values.nextOrd();
-      if (ord != NO_MORE_ORDS) {
-        return docID;
-      }
-      docID++;
-    }
-    docID = NO_MORE_DOCS;
-    return NO_MORE_DOCS;
-  }
-
-  @Override
-  public int advance(int target) {
-    if (target < docID) {
-      throw new IllegalArgumentException("cannot advance backwards: docID=" + docID + " target=" + target);
-    }
-    if (target >= maxDoc) {
-      this.docID = NO_MORE_DOCS;
-    } else {
-      this.docID = target-1;
-      nextDoc();
-    }
-    return docID;
-  }
-
-  @Override
-  public boolean advanceExact(int target) throws IOException {
-    docID = target;
-    values.setDocument(docID);
-    ord = values.nextOrd();
-    return ord != NO_MORE_ORDS;
-  }
-
-  @Override
-  public long cost() {
-    return 0;
-  }
-
-  @Override
-  public long nextOrd() {
-    long result = ord;
-    if (result != NO_MORE_ORDS) {
-      ord = values.nextOrd();
-    }
-    return result;
-  }
-
-  @Override
-  public BytesRef lookupOrd(long ord) {
-    return values.lookupOrd((int) ord);
-  }
-
-  @Override
-  public long getValueCount() {
-    return values.getValueCount();
-  }
-
-  @Override
-  public String toString() {
-    return "LegacySortedSetDocValuesWrapper(" + values + ")";
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3302ed2a/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java b/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
index 054f3a3..8446e0e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
+++ b/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.LegacyNumericDocValues;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
@@ -449,7 +448,10 @@ public class PackedInts {
    * A read-only random access array of positive integers.
    * @lucene.internal
    */
-  public static abstract class Reader extends LegacyNumericDocValues implements Accountable {
+  public static abstract class Reader implements Accountable {
+
+    /** Get the long at the given index. Behavior is undefined for out-of-range indices. */
+    public abstract long get(int index);
 
     /**
      * Bulk get: read at least one and at most <code>len</code> longs starting

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3302ed2a/solr/core/src/java/org/apache/solr/uninverting/DocTermOrds.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/uninverting/DocTermOrds.java b/solr/core/src/java/org/apache/solr/uninverting/DocTermOrds.java
index 580a971..736be37 100644
--- a/solr/core/src/java/org/apache/solr/uninverting/DocTermOrds.java
+++ b/solr/core/src/java/org/apache/solr/uninverting/DocTermOrds.java
@@ -28,8 +28,6 @@ import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.ImpactsEnum;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LegacySortedSetDocValues;
-import org.apache.lucene.index.LegacySortedSetDocValuesWrapper;
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Terms;
@@ -756,24 +754,27 @@ public class DocTermOrds implements Accountable {
     if (isEmpty()) {
       return DocValues.emptySortedSet();
     } else {
-      return new LegacySortedSetDocValuesWrapper(new Iterator(reader), reader.maxDoc());
+      return new Iterator(reader);
     }
   }
   
-  private class Iterator extends LegacySortedSetDocValues {
+  private class Iterator extends SortedSetDocValues {
     final LeafReader reader;
     final TermsEnum te;  // used internally for lookupOrd() and lookupTerm()
+    final int maxDoc;
     // currently we read 5 at a time (using the logic of the old iterator)
     final int buffer[] = new int[5];
     int bufferUpto;
     int bufferLength;
     
+    private int doc = -1;
     private int tnum;
     private int upto;
     private byte[] arr;
     
     Iterator(LeafReader reader) throws IOException {
       this.reader = reader;
+      this.maxDoc = reader.maxDoc();
       this.te = termsEnum();
     }
     
@@ -835,8 +836,8 @@ public class DocTermOrds implements Accountable {
       return bufferUpto;
     }
 
-    @Override
-    public void setDocument(int docID) {
+    private void setDocument(int docID) {
+      this.doc = docID;
       tnum = 0;
       final int code = index[docID];
       if ((code & 0x80000000) != 0) {
@@ -855,6 +856,37 @@ public class DocTermOrds implements Accountable {
     }
 
     @Override
+    public boolean advanceExact(int target) throws IOException {
+      setDocument(target);
+      return bufferLength > 0;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return advance(docID() + 1);
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      for (int d = target; d < maxDoc; ++d) {
+        if (advanceExact(d)) {
+          return d;
+        }
+      }
+      return doc = NO_MORE_DOCS;
+    }
+
+    @Override
+    public long cost() {
+      return maxDoc;
+    }
+
+    @Override
     public BytesRef lookupOrd(long ord) {
       try {
         return DocTermOrds.this.lookupTerm(te, (int) ord);