You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/03/06 14:26:55 UTC

[3/4] lucene-solr git commit: LUCENE-7070: insert missing 's' in PointFormat/Reader/Writer

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
new file mode 100644
index 0000000..ab21431
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsReader.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.Accountable;
+
+/** Abstract API to visit point values.
+ *
+ * @lucene.experimental
+ */
+public abstract class PointsReader extends PointValues implements Closeable, Accountable {
+
+  /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
+  protected PointsReader() {}
+
+  /** 
+   * Checks consistency of this reader.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g. 
+   * may involve computing a checksum value against large data files.
+   * @lucene.internal
+   */
+  public abstract void checkIntegrity() throws IOException;
+
+  /** 
+   * Returns an instance optimized for merging.
+   * <p>
+   * The default implementation returns {@code this} */
+  public PointsReader getMergeInstance() throws IOException {
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
new file mode 100644
index 0000000..56689ec
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/PointsWriter.java
@@ -0,0 +1,151 @@
+/*
+ * 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;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.MergeState;
+
+/** Abstract API to write points
+ *
+ * @lucene.experimental
+ */
+
+public abstract class PointsWriter implements Closeable {
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected PointsWriter() {
+  }
+
+  /** Write all values contained in the provided reader */
+  public abstract void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException;
+
+  /** Default naive merge implementation for one field: it just re-indexes all the values
+   *  from the incoming segment.  The default codec overrides this for 1D fields and uses
+   *  a faster but more complex implementation. */
+  protected void mergeOneField(MergeState mergeState, FieldInfo fieldInfo) throws IOException {
+    writeField(fieldInfo,
+               new PointsReader() {
+                 @Override
+                 public void intersect(String fieldName, IntersectVisitor mergedVisitor) throws IOException {
+                   if (fieldName.equals(fieldInfo.name) == false) {
+                     throw new IllegalArgumentException("field name must match the field being merged");
+                   }
+                   for (int i=0;i<mergeState.pointsReaders.length;i++) {
+                     PointsReader pointsReader = mergeState.pointsReaders[i];
+                     if (pointsReader == null) {
+                       // This segment has no points
+                       continue;
+                     }
+                     MergeState.DocMap docMap = mergeState.docMaps[i];
+                     int docBase = mergeState.docBase[i];
+                     pointsReader.intersect(fieldInfo.name,
+                                            new IntersectVisitor() {
+                                              @Override
+                                              public void visit(int docID) {
+                                                // Should never be called because our compare method never returns Relation.CELL_INSIDE_QUERY
+                                                throw new IllegalStateException();
+                                              }
+
+                                              @Override
+                                              public void visit(int docID, byte[] packedValue) throws IOException {
+                                                int newDocID = docMap.get(docID);
+                                                if (newDocID != -1) {
+                                                  // Not deleted:
+                                                  mergedVisitor.visit(docBase + newDocID, packedValue);
+                                                }
+                                              }
+
+                                              @Override
+                                              public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+                                                // Forces this segment's PointsReader to always visit all docs + values:
+                                                return Relation.CELL_CROSSES_QUERY;
+                                              }
+                                            });
+                   }
+                 }
+
+                 @Override
+                 public void checkIntegrity() {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public long ramBytesUsed() {
+                   return 0L;
+                 }
+
+                 @Override
+                 public void close() {
+                 }
+
+                 @Override
+                 public byte[] getMinPackedValue(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public byte[] getMaxPackedValue(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public int getNumDimensions(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public int getBytesPerDimension(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public long size(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+
+                 @Override
+                 public int getDocCount(String fieldName) {
+                   throw new UnsupportedOperationException();
+                 }
+               });
+  }
+
+  /** Default merge implementation to merge incoming points readers by visiting all their points and
+   *  adding to this writer */
+  public void merge(MergeState mergeState) throws IOException {
+    // check each incoming reader
+    for (PointsReader reader : mergeState.pointsReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+    // merge field at a time
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.getPointDimensionCount() != 0) {
+        mergeOneField(mergeState, fieldInfo);
+      }
+    }
+    finish();
+  }
+
+  /** Called once at the end before close */
+  public abstract void finish() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
index e9ef9a8..9f0d546 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60Codec.java
@@ -21,12 +21,12 @@ import java.util.Objects;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CompoundFormat;
-import org.apache.lucene.codecs.PointFormat;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PointsFormat;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.SegmentInfoFormat;
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -129,8 +129,8 @@ public class Lucene60Codec extends Codec {
   }
 
   @Override
-  public final PointFormat pointFormat() {
-    return new Lucene60PointFormat();
+  public final PointsFormat pointsFormat() {
+    return new Lucene60PointsFormat();
   }
 
   /** Returns the postings format that should be used for writing 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointFormat.java
deleted file mode 100644
index 8f43187..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointFormat.java
+++ /dev/null
@@ -1,106 +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.codecs.lucene60;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PointFormat;
-import org.apache.lucene.codecs.PointReader;
-import org.apache.lucene.codecs.PointWriter;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-
-/**
- * Lucene 6.0 point format, which encodes dimensional values in a block KD-tree structure
- * for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
- *
- * <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
- * binary tree of split values referencing those blocks at the leaves.
- *
- * <p>The <code>.dim</code> file has both blocks and the index split
- * values, for each field.  The file starts with {@link CodecUtil#writeIndexHeader}.
- *
- * <p>The blocks are written like this:
- *
- * <ul>
- *  <li> count (vInt)
- *  <li> delta-docID (vInt) <sup>count</sup> (delta coded docIDs, in sorted order)
- *  <li> packedValue<sup>count</sup> (the <code>byte[]</code> value of each dimension packed into a single <code>byte[]</code>)
- * </ul>
- *
- * <p>After all blocks for a field are written, then the index is written:
- * <ul>
- *  <li> numDims (vInt)
- *  <li> maxPointsInLeafNode (vInt)
- *  <li> bytesPerDim (vInt)
- *  <li> count (vInt)
- *  <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
- *  <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
- * </ul>
- *
- * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
- *
- * <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's
- * index data was written.  It starts with {@link CodecUtil#writeIndexHeader}, then has:
- *
- * <ul>
- *   <li> fieldCount (vInt)
- *   <li> (fieldNumber (vInt), fieldFilePointer (vLong))<sup>fieldCount</sup>
- * </ul>
- *
- * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
- *
- * @lucene.experimental
- */
-
-public final class Lucene60PointFormat extends PointFormat {
-
-  static final String DATA_CODEC_NAME = "Lucene60PointFormatData";
-  static final String META_CODEC_NAME = "Lucene60PointFormatMeta";
-
-  /**
-   * Filename extension for the leaf blocks
-   */
-  public static final String DATA_EXTENSION = "dim";
-
-  /**
-   * Filename extension for the index per field
-   */
-  public static final String INDEX_EXTENSION = "dii";
-
-  static final int DATA_VERSION_START = 0;
-  static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
-
-  static final int INDEX_VERSION_START = 0;
-  static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
-
-  /** Sole constructor */
-  public Lucene60PointFormat() {
-  }
-
-  @Override
-  public PointWriter fieldsWriter(SegmentWriteState state) throws IOException {
-    return new Lucene60PointWriter(state);
-  }
-
-  @Override
-  public PointReader fieldsReader(SegmentReadState state) throws IOException {
-    return new Lucene60PointReader(state);
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointReader.java
deleted file mode 100644
index 91a1e6c..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointReader.java
+++ /dev/null
@@ -1,241 +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.codecs.lucene60;
-
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PointReader;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.Accountable;
-import org.apache.lucene.util.Accountables;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.bkd.BKDReader;
-
-/** Reads point values previously written with {@link Lucene60PointWriter} */
-public class Lucene60PointReader extends PointReader implements Closeable {
-  final IndexInput dataIn;
-  final SegmentReadState readState;
-  final Map<Integer,BKDReader> readers = new HashMap<>();
-
-  /** Sole constructor */
-  public Lucene60PointReader(SegmentReadState readState) throws IOException {
-    this.readState = readState;
-
-
-    String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
-                                                          readState.segmentSuffix,
-                                                          Lucene60PointFormat.INDEX_EXTENSION);
-
-    Map<Integer,Long> fieldToFileOffset = new HashMap<>();
-
-    // Read index file
-    try (ChecksumIndexInput indexIn = readState.directory.openChecksumInput(indexFileName, readState.context)) {
-      Throwable priorE = null;
-      try {
-        CodecUtil.checkIndexHeader(indexIn,
-                                   Lucene60PointFormat.META_CODEC_NAME,
-                                   Lucene60PointFormat.INDEX_VERSION_START,
-                                   Lucene60PointFormat.INDEX_VERSION_START,
-                                   readState.segmentInfo.getId(),
-                                   readState.segmentSuffix);
-        int count = indexIn.readVInt();
-        for(int i=0;i<count;i++) {
-          int fieldNumber = indexIn.readVInt();
-          long fp = indexIn.readVLong();
-          fieldToFileOffset.put(fieldNumber, fp);
-        }
-      } catch (Throwable t) {
-        priorE = t;
-      } finally {
-        CodecUtil.checkFooter(indexIn, priorE);
-      }
-    }
-
-    String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
-                                                         readState.segmentSuffix,
-                                                         Lucene60PointFormat.DATA_EXTENSION);
-    boolean success = false;
-    dataIn = readState.directory.openInput(dataFileName, readState.context);
-    try {
-
-      CodecUtil.checkIndexHeader(dataIn,
-                                 Lucene60PointFormat.DATA_CODEC_NAME,
-                                 Lucene60PointFormat.DATA_VERSION_START,
-                                 Lucene60PointFormat.DATA_VERSION_START,
-                                 readState.segmentInfo.getId(),
-                                 readState.segmentSuffix);
-
-      // NOTE: data file is too costly to verify checksum against all the bytes on open,
-      // but for now we at least verify proper structure of the checksum footer: which looks
-      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
-      // such as file truncation.
-      CodecUtil.retrieveChecksum(dataIn);
-
-      for(Map.Entry<Integer,Long> ent : fieldToFileOffset.entrySet()) {
-        int fieldNumber = ent.getKey();
-        long fp = ent.getValue();
-        dataIn.seek(fp);
-        BKDReader reader = new BKDReader(dataIn);
-        readers.put(fieldNumber, reader);
-      }
-
-      success = true;
-    } finally {
-      if (success == false) {
-        IOUtils.closeWhileHandlingException(this);
-      }
-    }
-  }
-
-  private BKDReader getBKDReader(String fieldName) {
-    FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(fieldName);
-    if (fieldInfo == null) {
-      throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
-    }
-    if (fieldInfo.getPointDimensionCount() == 0) {
-      throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
-    }
-
-    return readers.get(fieldInfo.number);
-  }
-
-  @Override
-  public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
-    BKDReader bkdReader = getBKDReader(fieldName);
-
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return;
-    }
-
-    bkdReader.intersect(visitor);
-  }
-
-  @Override
-  public long ramBytesUsed() {
-    long sizeInBytes = 0;
-    for(BKDReader reader : readers.values()) {
-      sizeInBytes += reader.ramBytesUsed();
-    }
-    return sizeInBytes;
-  }
-
-  @Override
-  public Collection<Accountable> getChildResources() {
-    List<Accountable> resources = new ArrayList<>();
-    for(Map.Entry<Integer,BKDReader> ent : readers.entrySet()) {
-      resources.add(Accountables.namedAccountable(readState.fieldInfos.fieldInfo(ent.getKey()).name,
-                                                  ent.getValue()));
-    }
-    return Collections.unmodifiableList(resources);
-  }
-
-  @Override
-  public void checkIntegrity() throws IOException {
-    CodecUtil.checksumEntireFile(dataIn);
-  }
-
-  @Override
-  public void close() throws IOException {
-    dataIn.close();
-    // Free up heap:
-    readers.clear();
-  }
-
-  @Override
-  public byte[] getMinPackedValue(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return null;
-    }
-
-    return bkdReader.getMinPackedValue();
-  }
-
-  @Override
-  public byte[] getMaxPackedValue(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return null;
-    }
-
-    return bkdReader.getMaxPackedValue();
-  }
-
-  @Override
-  public int getNumDimensions(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return 0;
-    }
-    return bkdReader.getNumDimensions();
-  }
-
-  @Override
-  public int getBytesPerDimension(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return 0;
-    }
-    return bkdReader.getBytesPerDimension();
-  }
-
-  @Override
-  public long size(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return 0;
-    }
-    return bkdReader.getPointCount();
-  }
-
-  @Override
-  public int getDocCount(String fieldName) {
-    BKDReader bkdReader = getBKDReader(fieldName);
-    if (bkdReader == null) {
-      // Schema ghost corner case!  This field did index points in the past, but
-      // now all docs having this point field were deleted in this segment:
-      return 0;
-    }
-    return bkdReader.getDocCount();
-  }
-}
-  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointWriter.java
deleted file mode 100644
index 8a00d4c..0000000
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointWriter.java
+++ /dev/null
@@ -1,225 +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.codecs.lucene60;
-
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PointReader;
-import org.apache.lucene.codecs.PointWriter;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.PointValues.IntersectVisitor;
-import org.apache.lucene.index.PointValues.Relation;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.bkd.BKDReader;
-import org.apache.lucene.util.bkd.BKDWriter;
-
-/** Writes dimensional values */
-public class Lucene60PointWriter extends PointWriter implements Closeable {
-  
-  final IndexOutput dataOut;
-  final Map<String,Long> indexFPs = new HashMap<>();
-  final SegmentWriteState writeState;
-  final int maxPointsInLeafNode;
-  final double maxMBSortInHeap;
-  private boolean finished;
-
-  /** Full constructor */
-  public Lucene60PointWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
-    assert writeState.fieldInfos.hasPointValues();
-    this.writeState = writeState;
-    this.maxPointsInLeafNode = maxPointsInLeafNode;
-    this.maxMBSortInHeap = maxMBSortInHeap;
-    String dataFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
-                                                         writeState.segmentSuffix,
-                                                         Lucene60PointFormat.DATA_EXTENSION);
-    dataOut = writeState.directory.createOutput(dataFileName, writeState.context);
-    boolean success = false;
-    try {
-      CodecUtil.writeIndexHeader(dataOut,
-                                 Lucene60PointFormat.DATA_CODEC_NAME,
-                                 Lucene60PointFormat.DATA_VERSION_CURRENT,
-                                 writeState.segmentInfo.getId(),
-                                 writeState.segmentSuffix);
-      success = true;
-    } finally {
-      if (success == false) {
-        IOUtils.closeWhileHandlingException(dataOut);
-      }
-    }
-  }
-
-  /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
-  public Lucene60PointWriter(SegmentWriteState writeState) throws IOException {
-    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
-  }
-
-  @Override
-  public void writeField(FieldInfo fieldInfo, PointReader values) throws IOException {
-
-    try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
-                                          writeState.directory,
-                                          writeState.segmentInfo.name,
-                                          fieldInfo.getPointDimensionCount(),
-                                          fieldInfo.getPointNumBytes(),
-                                          maxPointsInLeafNode,
-                                          maxMBSortInHeap)) {
-
-      values.intersect(fieldInfo.name, new IntersectVisitor() {
-          @Override
-          public void visit(int docID) {
-            throw new IllegalStateException();
-          }
-
-          public void visit(int docID, byte[] packedValue) throws IOException {
-            writer.add(packedValue, docID);
-          }
-
-          @Override
-          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
-            return Relation.CELL_CROSSES_QUERY;
-          }
-        });
-
-      // We could have 0 points on merge since all docs with dimensional fields may be deleted:
-      if (writer.getPointCount() > 0) {
-        indexFPs.put(fieldInfo.name, writer.finish(dataOut));
-      }
-    }
-  }
-
-  @Override
-  public void merge(MergeState mergeState) throws IOException {
-    for(PointReader reader : mergeState.pointReaders) {
-      if (reader instanceof Lucene60PointReader == false) {
-        // We can only bulk merge when all to-be-merged segments use our format:
-        super.merge(mergeState);
-        return;
-      }
-    }
-    for (PointReader reader : mergeState.pointReaders) {
-      if (reader != null) {
-        reader.checkIntegrity();
-      }
-    }
-
-    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
-      if (fieldInfo.getPointDimensionCount() != 0) {
-        if (fieldInfo.getPointDimensionCount() == 1) {
-          //System.out.println("MERGE: field=" + fieldInfo.name);
-          // Optimize the 1D case to use BKDWriter.merge, which does a single merge sort of the
-          // already sorted incoming segments, instead of trying to sort all points again as if
-          // we were simply reindexing them:
-          try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
-                                                writeState.directory,
-                                                writeState.segmentInfo.name,
-                                                fieldInfo.getPointDimensionCount(),
-                                                fieldInfo.getPointNumBytes(),
-                                                maxPointsInLeafNode,
-                                                maxMBSortInHeap)) {
-            List<BKDReader> bkdReaders = new ArrayList<>();
-            List<MergeState.DocMap> docMaps = new ArrayList<>();
-            List<Integer> docIDBases = new ArrayList<>();
-            for(int i=0;i<mergeState.pointReaders.length;i++) {
-              PointReader reader = mergeState.pointReaders[i];
-
-              if (reader != null) {
-
-                // we confirmed this up above
-                assert reader instanceof Lucene60PointReader;
-                Lucene60PointReader reader60 = (Lucene60PointReader) reader;
-
-                // NOTE: we cannot just use the merged fieldInfo.number (instead of resolving to this
-                // reader's FieldInfo as we do below) because field numbers can easily be different
-                // when addIndexes(Directory...) copies over segments from another index:
-
-
-                FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
-                FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
-                if (readerFieldInfo != null) {
-                  BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
-                  if (bkdReader != null) {
-                    docIDBases.add(mergeState.docBase[i]);
-                    bkdReaders.add(bkdReader);
-                    docMaps.add(mergeState.docMaps[i]);
-                  }
-                }
-              }
-            }
-
-            long fp = writer.merge(dataOut, docMaps, bkdReaders, docIDBases);
-            if (fp != -1) {
-              indexFPs.put(fieldInfo.name, fp);
-            }
-          }
-        } else {
-          mergeOneField(mergeState, fieldInfo);
-        }
-      }
-    }
-
-    finish();
-  }
-
-  @Override
-  public void finish() throws IOException {
-    if (finished) {
-      throw new IllegalStateException("already finished");
-    }
-    finished = true;
-    CodecUtil.writeFooter(dataOut);
-
-    String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
-                                                          writeState.segmentSuffix,
-                                                          Lucene60PointFormat.INDEX_EXTENSION);
-    // Write index file
-    try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
-      CodecUtil.writeIndexHeader(indexOut,
-                                 Lucene60PointFormat.META_CODEC_NAME,
-                                 Lucene60PointFormat.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);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    dataOut.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
new file mode 100644
index 0000000..e558d0d
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsFormat.java
@@ -0,0 +1,106 @@
+/*
+ * 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.lucene60;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+/**
+ * Lucene 6.0 point format, which encodes dimensional values in a block KD-tree structure
+ * for fast shape intersection filtering. See <a href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a> for details.
+ *
+ * <p>This data structure is written as a series of blocks on disk, with an in-memory perfectly balanced
+ * binary tree of split values referencing those blocks at the leaves.
+ *
+ * <p>The <code>.dim</code> file has both blocks and the index split
+ * values, for each field.  The file starts with {@link CodecUtil#writeIndexHeader}.
+ *
+ * <p>The blocks are written like this:
+ *
+ * <ul>
+ *  <li> count (vInt)
+ *  <li> delta-docID (vInt) <sup>count</sup> (delta coded docIDs, in sorted order)
+ *  <li> packedValue<sup>count</sup> (the <code>byte[]</code> value of each dimension packed into a single <code>byte[]</code>)
+ * </ul>
+ *
+ * <p>After all blocks for a field are written, then the index is written:
+ * <ul>
+ *  <li> numDims (vInt)
+ *  <li> maxPointsInLeafNode (vInt)
+ *  <li> bytesPerDim (vInt)
+ *  <li> count (vInt)
+ *  <li> byte[bytesPerDim]<sup>count</sup> (packed <code>byte[]</code> all split values)
+ *  <li> delta-blockFP (vLong)<sup>count</sup> (delta-coded file pointers to the on-disk leaf blocks))
+ * </ul>
+ *
+ * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
+ *
+ * <p>The <code>.dii</code> file records the file pointer in the <code>.dim</code> file where each field's
+ * index data was written.  It starts with {@link CodecUtil#writeIndexHeader}, then has:
+ *
+ * <ul>
+ *   <li> fieldCount (vInt)
+ *   <li> (fieldNumber (vInt), fieldFilePointer (vLong))<sup>fieldCount</sup>
+ * </ul>
+ *
+ * <p>After all fields blocks + index data are written, {@link CodecUtil#writeFooter} writes the checksum.
+ *
+ * @lucene.experimental
+ */
+
+public final class Lucene60PointsFormat extends PointsFormat {
+
+  static final String DATA_CODEC_NAME = "Lucene60PointsFormatData";
+  static final String META_CODEC_NAME = "Lucene60PointsFormatMeta";
+
+  /**
+   * Filename extension for the leaf blocks
+   */
+  public static final String DATA_EXTENSION = "dim";
+
+  /**
+   * Filename extension for the index per field
+   */
+  public static final String INDEX_EXTENSION = "dii";
+
+  static final int DATA_VERSION_START = 0;
+  static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
+
+  static final int INDEX_VERSION_START = 0;
+  static final int INDEX_VERSION_CURRENT = INDEX_VERSION_START;
+
+  /** Sole constructor */
+  public Lucene60PointsFormat() {
+  }
+
+  @Override
+  public PointsWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new Lucene60PointsWriter(state);
+  }
+
+  @Override
+  public PointsReader fieldsReader(SegmentReadState state) throws IOException {
+    return new Lucene60PointsReader(state);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
new file mode 100644
index 0000000..e7a612c
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsReader.java
@@ -0,0 +1,241 @@
+/*
+ * 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.lucene60;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.bkd.BKDReader;
+
+/** Reads point values previously written with {@link Lucene60PointsWriter} */
+public class Lucene60PointsReader extends PointsReader implements Closeable {
+  final IndexInput dataIn;
+  final SegmentReadState readState;
+  final Map<Integer,BKDReader> readers = new HashMap<>();
+
+  /** Sole constructor */
+  public Lucene60PointsReader(SegmentReadState readState) throws IOException {
+    this.readState = readState;
+
+
+    String indexFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
+                                                          readState.segmentSuffix,
+                                                          Lucene60PointsFormat.INDEX_EXTENSION);
+
+    Map<Integer,Long> fieldToFileOffset = new HashMap<>();
+
+    // Read index file
+    try (ChecksumIndexInput indexIn = readState.directory.openChecksumInput(indexFileName, readState.context)) {
+      Throwable priorE = null;
+      try {
+        CodecUtil.checkIndexHeader(indexIn,
+                                   Lucene60PointsFormat.META_CODEC_NAME,
+                                   Lucene60PointsFormat.INDEX_VERSION_START,
+                                   Lucene60PointsFormat.INDEX_VERSION_START,
+                                   readState.segmentInfo.getId(),
+                                   readState.segmentSuffix);
+        int count = indexIn.readVInt();
+        for(int i=0;i<count;i++) {
+          int fieldNumber = indexIn.readVInt();
+          long fp = indexIn.readVLong();
+          fieldToFileOffset.put(fieldNumber, fp);
+        }
+      } catch (Throwable t) {
+        priorE = t;
+      } finally {
+        CodecUtil.checkFooter(indexIn, priorE);
+      }
+    }
+
+    String dataFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name,
+                                                         readState.segmentSuffix,
+                                                         Lucene60PointsFormat.DATA_EXTENSION);
+    boolean success = false;
+    dataIn = readState.directory.openInput(dataFileName, readState.context);
+    try {
+
+      CodecUtil.checkIndexHeader(dataIn,
+                                 Lucene60PointsFormat.DATA_CODEC_NAME,
+                                 Lucene60PointsFormat.DATA_VERSION_START,
+                                 Lucene60PointsFormat.DATA_VERSION_START,
+                                 readState.segmentInfo.getId(),
+                                 readState.segmentSuffix);
+
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(dataIn);
+
+      for(Map.Entry<Integer,Long> ent : fieldToFileOffset.entrySet()) {
+        int fieldNumber = ent.getKey();
+        long fp = ent.getValue();
+        dataIn.seek(fp);
+        BKDReader reader = new BKDReader(dataIn);
+        readers.put(fieldNumber, reader);
+      }
+
+      success = true;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private BKDReader getBKDReader(String fieldName) {
+    FieldInfo fieldInfo = readState.fieldInfos.fieldInfo(fieldName);
+    if (fieldInfo == null) {
+      throw new IllegalArgumentException("field=\"" + fieldName + "\" is unrecognized");
+    }
+    if (fieldInfo.getPointDimensionCount() == 0) {
+      throw new IllegalArgumentException("field=\"" + fieldName + "\" did not index point values");
+    }
+
+    return readers.get(fieldInfo.number);
+  }
+
+  @Override
+  public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
+    BKDReader bkdReader = getBKDReader(fieldName);
+
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return;
+    }
+
+    bkdReader.intersect(visitor);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    long sizeInBytes = 0;
+    for(BKDReader reader : readers.values()) {
+      sizeInBytes += reader.ramBytesUsed();
+    }
+    return sizeInBytes;
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    for(Map.Entry<Integer,BKDReader> ent : readers.entrySet()) {
+      resources.add(Accountables.namedAccountable(readState.fieldInfos.fieldInfo(ent.getKey()).name,
+                                                  ent.getValue()));
+    }
+    return Collections.unmodifiableList(resources);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(dataIn);
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataIn.close();
+    // Free up heap:
+    readers.clear();
+  }
+
+  @Override
+  public byte[] getMinPackedValue(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return null;
+    }
+
+    return bkdReader.getMinPackedValue();
+  }
+
+  @Override
+  public byte[] getMaxPackedValue(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return null;
+    }
+
+    return bkdReader.getMaxPackedValue();
+  }
+
+  @Override
+  public int getNumDimensions(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return 0;
+    }
+    return bkdReader.getNumDimensions();
+  }
+
+  @Override
+  public int getBytesPerDimension(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return 0;
+    }
+    return bkdReader.getBytesPerDimension();
+  }
+
+  @Override
+  public long size(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return 0;
+    }
+    return bkdReader.getPointCount();
+  }
+
+  @Override
+  public int getDocCount(String fieldName) {
+    BKDReader bkdReader = getBKDReader(fieldName);
+    if (bkdReader == null) {
+      // Schema ghost corner case!  This field did index points in the past, but
+      // now all docs having this point field were deleted in this segment:
+      return 0;
+    }
+    return bkdReader.getDocCount();
+  }
+}
+  

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
new file mode 100644
index 0000000..3d09c45
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointsWriter.java
@@ -0,0 +1,225 @@
+/*
+ * 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.lucene60;
+
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.PointValues.IntersectVisitor;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.bkd.BKDReader;
+import org.apache.lucene.util.bkd.BKDWriter;
+
+/** Writes dimensional values */
+public class Lucene60PointsWriter extends PointsWriter implements Closeable {
+  
+  final IndexOutput dataOut;
+  final Map<String,Long> indexFPs = new HashMap<>();
+  final SegmentWriteState writeState;
+  final int maxPointsInLeafNode;
+  final double maxMBSortInHeap;
+  private boolean finished;
+
+  /** Full constructor */
+  public Lucene60PointsWriter(SegmentWriteState writeState, int maxPointsInLeafNode, double maxMBSortInHeap) throws IOException {
+    assert writeState.fieldInfos.hasPointValues();
+    this.writeState = writeState;
+    this.maxPointsInLeafNode = maxPointsInLeafNode;
+    this.maxMBSortInHeap = maxMBSortInHeap;
+    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;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(dataOut);
+      }
+    }
+  }
+
+  /** Uses the defaults values for {@code maxPointsInLeafNode} (1024) and {@code maxMBSortInHeap} (16.0) */
+  public Lucene60PointsWriter(SegmentWriteState writeState) throws IOException {
+    this(writeState, BKDWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKDWriter.DEFAULT_MAX_MB_SORT_IN_HEAP);
+  }
+
+  @Override
+  public void writeField(FieldInfo fieldInfo, PointsReader values) throws IOException {
+
+    try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
+                                          writeState.directory,
+                                          writeState.segmentInfo.name,
+                                          fieldInfo.getPointDimensionCount(),
+                                          fieldInfo.getPointNumBytes(),
+                                          maxPointsInLeafNode,
+                                          maxMBSortInHeap)) {
+
+      values.intersect(fieldInfo.name, new IntersectVisitor() {
+          @Override
+          public void visit(int docID) {
+            throw new IllegalStateException();
+          }
+
+          public void visit(int docID, byte[] packedValue) throws IOException {
+            writer.add(packedValue, docID);
+          }
+
+          @Override
+          public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
+            return Relation.CELL_CROSSES_QUERY;
+          }
+        });
+
+      // We could have 0 points on merge since all docs with dimensional fields may be deleted:
+      if (writer.getPointCount() > 0) {
+        indexFPs.put(fieldInfo.name, writer.finish(dataOut));
+      }
+    }
+  }
+
+  @Override
+  public void merge(MergeState mergeState) throws IOException {
+    for(PointsReader reader : mergeState.pointsReaders) {
+      if (reader instanceof Lucene60PointsReader == false) {
+        // We can only bulk merge when all to-be-merged segments use our format:
+        super.merge(mergeState);
+        return;
+      }
+    }
+    for (PointsReader reader : mergeState.pointsReaders) {
+      if (reader != null) {
+        reader.checkIntegrity();
+      }
+    }
+
+    for (FieldInfo fieldInfo : mergeState.mergeFieldInfos) {
+      if (fieldInfo.getPointDimensionCount() != 0) {
+        if (fieldInfo.getPointDimensionCount() == 1) {
+          //System.out.println("MERGE: field=" + fieldInfo.name);
+          // Optimize the 1D case to use BKDWriter.merge, which does a single merge sort of the
+          // already sorted incoming segments, instead of trying to sort all points again as if
+          // we were simply reindexing them:
+          try (BKDWriter writer = new BKDWriter(writeState.segmentInfo.maxDoc(),
+                                                writeState.directory,
+                                                writeState.segmentInfo.name,
+                                                fieldInfo.getPointDimensionCount(),
+                                                fieldInfo.getPointNumBytes(),
+                                                maxPointsInLeafNode,
+                                                maxMBSortInHeap)) {
+            List<BKDReader> bkdReaders = new ArrayList<>();
+            List<MergeState.DocMap> docMaps = new ArrayList<>();
+            List<Integer> docIDBases = new ArrayList<>();
+            for(int i=0;i<mergeState.pointsReaders.length;i++) {
+              PointsReader reader = mergeState.pointsReaders[i];
+
+              if (reader != null) {
+
+                // we confirmed this up above
+                assert reader instanceof Lucene60PointsReader;
+                Lucene60PointsReader reader60 = (Lucene60PointsReader) reader;
+
+                // NOTE: we cannot just use the merged fieldInfo.number (instead of resolving to this
+                // reader's FieldInfo as we do below) because field numbers can easily be different
+                // when addIndexes(Directory...) copies over segments from another index:
+
+
+                FieldInfos readerFieldInfos = mergeState.fieldInfos[i];
+                FieldInfo readerFieldInfo = readerFieldInfos.fieldInfo(fieldInfo.name);
+                if (readerFieldInfo != null) {
+                  BKDReader bkdReader = reader60.readers.get(readerFieldInfo.number);
+                  if (bkdReader != null) {
+                    docIDBases.add(mergeState.docBase[i]);
+                    bkdReaders.add(bkdReader);
+                    docMaps.add(mergeState.docMaps[i]);
+                  }
+                }
+              }
+            }
+
+            long fp = writer.merge(dataOut, docMaps, bkdReaders, docIDBases);
+            if (fp != -1) {
+              indexFPs.put(fieldInfo.name, fp);
+            }
+          }
+        } else {
+          mergeOneField(mergeState, fieldInfo);
+        }
+      }
+    }
+
+    finish();
+  }
+
+  @Override
+  public void finish() throws IOException {
+    if (finished) {
+      throw new IllegalStateException("already finished");
+    }
+    finished = true;
+    CodecUtil.writeFooter(dataOut);
+
+    String indexFileName = IndexFileNames.segmentFileName(writeState.segmentInfo.name,
+                                                          writeState.segmentSuffix,
+                                                          Lucene60PointsFormat.INDEX_EXTENSION);
+    // Write index file
+    try (IndexOutput indexOut = writeState.directory.createOutput(indexFileName, writeState.context)) {
+      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);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    dataOut.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
index a52d6f6..64531f5 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene60/package-info.java
@@ -193,7 +193,7 @@
  * An optional file indicating which documents are live.
  * </li>
  * <li>
- * {@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point values}.
+ * {@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}.
  * Optional pair of files, recording dimesionally indexed fields, to enable fast
  * numeric range filtering and large numeric values like BigInteger and BigDecimal (1D)
  * and geo shape intersection (2D, 3D).
@@ -322,7 +322,7 @@
  * <td>Info about what files are live</td>
  * </tr>
  * <tr>
- * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointFormat Point values}</td>
+ * <td>{@link org.apache.lucene.codecs.lucene60.Lucene60PointsFormat Point values}</td>
  * <td>.dii, .dim</td>
  * <td>Holds indexed points, if any</td>
  * </tr>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index 0bfa350..3c437c1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -33,9 +33,9 @@ import java.util.Locale;
 import java.util.Map;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -1687,9 +1687,9 @@ public final class CheckIndex implements Closeable {
     Status.PointsStatus status = new Status.PointsStatus();
     try {
       if (fieldInfos.hasPointValues()) {
-        PointReader values = reader.getPointReader();
+        PointsReader values = reader.getPointsReader();
         if (values == null) {
-          throw new RuntimeException("there are fields with points, but reader.getPointReader() is null");
+          throw new RuntimeException("there are fields with points, but reader.getPointsReader() is null");
         }
         for (FieldInfo fieldInfo : fieldInfos) {
           if (fieldInfo.getPointDimensionCount() > 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
index eb53648..194acd8e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CodecReader.java
@@ -25,10 +25,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.util.Accountable;
@@ -77,10 +77,10 @@ public abstract class CodecReader extends LeafReader implements Accountable {
   public abstract FieldsProducer getPostingsReader();
 
   /**
-   * Expert: retrieve underlying PointReader
+   * Expert: retrieve underlying PointsReader
    * @lucene.internal
    */
-  public abstract PointReader getPointReader();
+  public abstract PointsReader getPointsReader();
   
   @Override
   public final void document(int docID, StoredFieldVisitor visitor) throws IOException {
@@ -323,8 +323,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
     }
 
     // points
-    if (getPointReader() != null) {
-      ramBytesUsed += getPointReader().ramBytesUsed();
+    if (getPointsReader() != null) {
+      ramBytesUsed += getPointsReader().ramBytesUsed();
     }
     
     return ramBytesUsed;
@@ -359,8 +359,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
     }
 
     // points
-    if (getPointReader() != null) {
-      resources.add(Accountables.namedAccountable("points", getPointReader()));
+    if (getPointsReader() != null) {
+      resources.add(Accountables.namedAccountable("points", getPointsReader()));
     }
     
     return Collections.unmodifiableList(resources);
@@ -394,8 +394,8 @@ public abstract class CodecReader extends LeafReader implements Accountable {
     }
 
     // points
-    if (getPointReader() != null) {
-      getPointReader().checkIntegrity();
+    if (getPointsReader() != null) {
+      getPointsReader().checkIntegrity();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index d1a68da..408f0bb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -23,12 +23,12 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.codecs.PointFormat;
-import org.apache.lucene.codecs.PointWriter;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.NormsConsumer;
 import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.search.similarities.Similarity;
@@ -149,7 +149,7 @@ final class DefaultIndexingChain extends DocConsumer {
 
   /** Writes all buffered points. */
   private void writePoints(SegmentWriteState state) throws IOException {
-    PointWriter pointWriter = null;
+    PointsWriter pointsWriter = null;
     boolean success = false;
     try {
       for (int i=0;i<fieldHash.length;i++) {
@@ -160,16 +160,16 @@ final class DefaultIndexingChain extends DocConsumer {
               // BUG
               throw new AssertionError("segment=" + state.segmentInfo + ": field=\"" + perField.fieldInfo.name + "\" has no points but wrote them");
             }
-            if (pointWriter == null) {
+            if (pointsWriter == null) {
               // lazy init
-              PointFormat fmt = state.segmentInfo.getCodec().pointFormat();
+              PointsFormat fmt = state.segmentInfo.getCodec().pointsFormat();
               if (fmt == null) {
                 throw new IllegalStateException("field=\"" + perField.fieldInfo.name + "\" was indexed as points but codec does not support points");
               }
-              pointWriter = fmt.fieldsWriter(state);
+              pointsWriter = fmt.fieldsWriter(state);
             }
 
-            perField.pointValuesWriter.flush(state, pointWriter);
+            perField.pointValuesWriter.flush(state, pointsWriter);
             perField.pointValuesWriter = null;
           } else if (perField.fieldInfo.getPointDimensionCount() != 0) {
             // BUG
@@ -178,15 +178,15 @@ final class DefaultIndexingChain extends DocConsumer {
           perField = perField.next;
         }
       }
-      if (pointWriter != null) {
-        pointWriter.finish();
+      if (pointsWriter != null) {
+        pointsWriter.finish();
       }
       success = true;
     } finally {
       if (success) {
-        IOUtils.close(pointWriter);
+        IOUtils.close(pointsWriter);
       } else {
-        IOUtils.closeWhileHandlingException(pointWriter);
+        IOUtils.closeWhileHandlingException(pointsWriter);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
index 2574bce..0dccf31 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
@@ -49,7 +49,7 @@ public final class FieldInfo {
   private long dvGen;
 
   /** If both of these are positive it means this field indexed points
-   *  (see {@link org.apache.lucene.codecs.PointFormat}). */
+   *  (see {@link org.apache.lucene.codecs.PointsFormat}). */
   private int pointDimensionCount;
   private int pointNumBytes;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
index 8b2a55f..41f0984 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java
@@ -19,10 +19,10 @@ package org.apache.lucene.index;
 
 import java.util.Objects;
 
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.util.Bits;
@@ -82,8 +82,8 @@ public class FilterCodecReader extends CodecReader {
   }
 
   @Override
-  public PointReader getPointReader() {
-    return in.getPointReader();
+  public PointsReader getPointsReader() {
+    return in.getPointsReader();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
index 76db600..9622d4e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
@@ -300,7 +300,7 @@ public abstract class LeafReader extends IndexReader {
    */
   public abstract Bits getLiveDocs();
 
-  /** Returns the {@link org.apache.lucene.codecs.PointReader} used for numeric or
+  /** Returns the {@link PointValues} used for numeric or
    *  spatial searches, or null if there are no point fields. */
   public abstract PointValues getPointValues();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/MergeState.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
index 531c799..7242785 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java
@@ -20,10 +20,10 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.util.Bits;
@@ -67,7 +67,7 @@ public class MergeState {
   public final FieldsProducer[] fieldsProducers;
 
   /** Point readers to merge */
-  public final PointReader[] pointReaders;
+  public final PointsReader[] pointsReaders;
 
   /** New docID base per reader. */
   public final int[] docBase;
@@ -90,7 +90,7 @@ public class MergeState {
     storedFieldsReaders = new StoredFieldsReader[numReaders];
     termVectorsReaders = new TermVectorsReader[numReaders];
     docValuesProducers = new DocValuesProducer[numReaders];
-    pointReaders = new PointReader[numReaders];
+    pointsReaders = new PointsReader[numReaders];
     fieldInfos = new FieldInfos[numReaders];
     liveDocs = new Bits[numReaders];
 
@@ -122,9 +122,9 @@ public class MergeState {
       }
       
       fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
-      pointReaders[i] = reader.getPointReader();
-      if (pointReaders[i] != null) {
-        pointReaders[i] = pointReaders[i].getMergeInstance();
+      pointsReaders[i] = reader.getPointsReader();
+      if (pointsReaders[i] != null) {
+        pointsReaders[i] = pointsReaders[i].getMergeInstance();
       }
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index 546bf71..35b9a90 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -18,8 +18,8 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.codecs.PointReader;
-import org.apache.lucene.codecs.PointWriter;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -60,10 +60,10 @@ class PointValuesWriter {
     numDocs++;
   }
 
-  public void flush(SegmentWriteState state, PointWriter writer) throws IOException {
+  public void flush(SegmentWriteState state, PointsWriter writer) throws IOException {
 
     writer.writeField(fieldInfo,
-                      new PointReader() {
+                      new PointsReader() {
                         @Override
                         public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
                           if (fieldName.equals(fieldInfo.name) == false) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
index 296fbbc..e99c1ad 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
@@ -24,9 +24,9 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -54,7 +54,7 @@ final class SegmentCoreReaders {
 
   final StoredFieldsReader fieldsReaderOrig;
   final TermVectorsReader termVectorsReaderOrig;
-  final PointReader pointReader;
+  final PointsReader pointsReader;
   final Directory cfsReader;
   /** 
    * fieldinfos for this core: means gen=-1.
@@ -125,9 +125,9 @@ final class SegmentCoreReaders {
       }
 
       if (coreFieldInfos.hasPointValues()) {
-        pointReader = codec.pointFormat().fieldsReader(segmentReadState);
+        pointsReader = codec.pointsFormat().fieldsReader(segmentReadState);
       } else {
-        pointReader = null;
+        pointsReader = null;
       }
       success = true;
     } finally {
@@ -157,7 +157,7 @@ final class SegmentCoreReaders {
       Throwable th = null;
       try {
         IOUtils.close(termVectorsLocal, fieldsReaderLocal, fields, termVectorsReaderOrig, fieldsReaderOrig,
-                      cfsReader, normsProducer, pointReader);
+                      cfsReader, normsProducer, pointsReader);
       } catch (Throwable throwable) {
         th = throwable;
       } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
index bb8e256..b0d9bcf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentMerger.java
@@ -21,10 +21,10 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PointWriter;
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsConsumer;
+import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.store.Directory;
@@ -164,7 +164,7 @@ final class SegmentMerger {
   }
 
   private void mergePoints(SegmentWriteState segmentWriteState) throws IOException {
-    try (PointWriter writer = codec.pointFormat().fieldsWriter(segmentWriteState)) {
+    try (PointsWriter writer = codec.pointsFormat().fieldsWriter(segmentWriteState)) {
       writer.merge(mergeState);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
index 205017b..8ed93e3 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
@@ -21,11 +21,11 @@ import java.io.IOException;
 import java.util.Collections;
 
 import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldInfosFormat;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.store.Directory;
@@ -220,7 +220,7 @@ public final class SegmentReader extends CodecReader {
   @Override
   public PointValues getPointValues() {
     ensureOpen();
-    return core.pointReader;
+    return core.pointsReader;
   }
 
   @Override
@@ -242,9 +242,9 @@ public final class SegmentReader extends CodecReader {
   }
 
   @Override
-  public PointReader getPointReader() {
+  public PointsReader getPointsReader() {
     ensureOpen();
-    return core.pointReader;
+    return core.pointsReader;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
index a741111..3a73701 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SlowCodecReaderWrapper.java
@@ -20,10 +20,10 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.lucene.codecs.PointReader;
 import org.apache.lucene.codecs.DocValuesProducer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
+import org.apache.lucene.codecs.PointsReader;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.util.Bits;
@@ -97,7 +97,7 @@ public final class SlowCodecReaderWrapper {
         }
 
         @Override
-        public PointReader getPointReader() {
+        public PointsReader getPointsReader() {
           return pointValuesToReader(reader.getPointValues());
         }
 
@@ -129,11 +129,11 @@ public final class SlowCodecReaderWrapper {
     }
   }
 
-  private static PointReader pointValuesToReader(PointValues values) {
+  private static PointsReader pointValuesToReader(PointValues values) {
     if (values == null) {
       return null;
     }
-    return new PointReader() {
+    return new PointsReader() {
       @Override
       public void intersect(String fieldName, IntersectVisitor visitor) throws IOException {
         values.intersect(fieldName, visitor);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointFormat.java
deleted file mode 100644
index 272fc6b..0000000
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointFormat.java
+++ /dev/null
@@ -1,83 +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.codecs.lucene60;
-
-
-import java.io.IOException;
-
-import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.FilterCodec;
-import org.apache.lucene.codecs.PointFormat;
-import org.apache.lucene.codecs.PointReader;
-import org.apache.lucene.codecs.PointWriter;
-import org.apache.lucene.index.BasePointFormatTestCase;
-import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.TestUtil;
-
-/**
- * Tests Lucene60PointFormat
- */
-public class TestLucene60PointFormat extends BasePointFormatTestCase {
-  private final Codec codec;
-  
-  public TestLucene60PointFormat() {
-    // standard issue
-    Codec defaultCodec = TestUtil.getDefaultCodec();
-    if (random().nextBoolean()) {
-      // randomize parameters
-      int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
-      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
-      if (VERBOSE) {
-        System.out.println("TEST: using Lucene60PointFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
-      }
-
-      // sneaky impersonation!
-      codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
-        @Override
-        public PointFormat pointFormat() {
-          return new PointFormat() {
-            @Override
-            public PointWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
-              return new Lucene60PointWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
-            }
-
-            @Override
-            public PointReader fieldsReader(SegmentReadState readState) throws IOException {
-              return new Lucene60PointReader(readState);
-            }
-          };
-        }
-      };
-    } else {
-      // standard issue
-      codec = defaultCodec;
-    }
-  }
-
-  @Override
-  protected Codec getCodec() {
-    return codec;
-  }
-
-  @Override
-  public void testMergeStability() throws Exception {
-    assumeFalse("TODO: mess with the parameters and test gets angry!", codec instanceof FilterCodec);
-    super.testMergeStability();
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/74e7e150/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
new file mode 100644
index 0000000..afa8ec4
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene60/TestLucene60PointsFormat.java
@@ -0,0 +1,83 @@
+/*
+ * 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.lucene60;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.FilterCodec;
+import org.apache.lucene.codecs.PointsFormat;
+import org.apache.lucene.codecs.PointsReader;
+import org.apache.lucene.codecs.PointsWriter;
+import org.apache.lucene.index.BasePointsFormatTestCase;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Tests Lucene60PointsFormat
+ */
+public class TestLucene60PointsFormat extends BasePointsFormatTestCase {
+  private final Codec codec;
+  
+  public TestLucene60PointsFormat() {
+    // standard issue
+    Codec defaultCodec = TestUtil.getDefaultCodec();
+    if (random().nextBoolean()) {
+      // randomize parameters
+      int maxPointsInLeafNode = TestUtil.nextInt(random(), 50, 500);
+      double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
+      if (VERBOSE) {
+        System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
+      }
+
+      // sneaky impersonation!
+      codec = new FilterCodec(defaultCodec.getName(), defaultCodec) {
+        @Override
+        public PointsFormat pointsFormat() {
+          return new PointsFormat() {
+            @Override
+            public PointsWriter fieldsWriter(SegmentWriteState writeState) throws IOException {
+              return new Lucene60PointsWriter(writeState, maxPointsInLeafNode, maxMBSortInHeap);
+            }
+
+            @Override
+            public PointsReader fieldsReader(SegmentReadState readState) throws IOException {
+              return new Lucene60PointsReader(readState);
+            }
+          };
+        }
+      };
+    } else {
+      // standard issue
+      codec = defaultCodec;
+    }
+  }
+
+  @Override
+  protected Codec getCodec() {
+    return codec;
+  }
+
+  @Override
+  public void testMergeStability() throws Exception {
+    assumeFalse("TODO: mess with the parameters and test gets angry!", codec instanceof FilterCodec);
+    super.testMergeStability();
+  }
+  
+}