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 2015/09/02 21:59:32 UTC

svn commit: r1700883 [2/5] - in /lucene/dev/trunk/lucene: ./ sandbox/src/java/org/apache/lucene/bkdtree/ sandbox/src/java/org/apache/lucene/rangetree/ spatial/src/java/org/apache/lucene/spatial/spatial4j/ spatial/src/test/org/apache/lucene/spatial/spat...

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesFormat.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,168 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.codecs.DocValuesConsumer;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.lucene50.Lucene50DocValuesFormat;
+import org.apache.lucene.geo3d.PlanetModel;
+import org.apache.lucene.geo3d.Vector;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+import java.io.IOException;
+
+/**
+ * A {@link DocValuesFormat} to efficiently index geo-spatial 3D x,y,z points
+ * from {@link Geo3DPointField} for fast shape intersection queries using
+ * ({@link PointInGeo3DShapeQuery})
+ *
+ * <p>This wraps {@link Lucene50DocValuesFormat}, but saves its own BKD tree
+ * structures to disk for fast query-time intersection. See <a
+ * href="https://www.cs.duke.edu/~pankaj/publications/papers/bkd-sstd.pdf">this paper</a>
+ * for details.
+ *
+ * <p>The BKD tree slices up 3D x,y,z space into smaller and
+ * smaller 3D rectangles, until the smallest rectangles have approximately
+ * between X/2 and X (X default is 1024) points in them, at which point
+ * such leaf cells are written as a block to disk, while the index tree
+ * structure records how space was sub-divided is loaded into HEAP
+ * at search time.  At search time, the tree is recursed based on whether
+ * each of left or right child overlap with the query shape, and once
+ * a leaf block is reached, all documents in that leaf block are collected
+ * if the cell is fully enclosed by the query shape, or filtered and then
+ * collected, if not.
+ *
+ * <p>The index is also quite compact, because docs only appear once in
+ * the tree (no "prefix terms").
+ *
+ * <p>In addition to the files written by {@link Lucene50DocValuesFormat}, this format writes:
+ * <ol>
+ *   <li><tt>.kd3d</tt>: BKD leaf data and index</li>
+ *   <li><tt>.kd3m</tt>: BKD metadata</li>
+ * </ol>
+ *
+ * <p>The disk format is experimental and free to change suddenly, and this code
+ * likely has new and exciting bugs!
+ *
+ * @lucene.experimental */
+
+public class Geo3DDocValuesFormat extends DocValuesFormat {
+
+  static final String DATA_CODEC_NAME = "Geo3DData";
+  static final int DATA_VERSION_START = 0;
+  static final int DATA_VERSION_CURRENT = DATA_VERSION_START;
+  static final String DATA_EXTENSION = "g3dd";
+
+  static final String META_CODEC_NAME = "Geo3DMeta";
+  static final int META_VERSION_START = 0;
+  static final int META_VERSION_CURRENT = META_VERSION_START;
+  static final String META_EXTENSION = "g3dm";
+
+  private final int maxPointsInLeafNode;
+  private final int maxPointsSortInHeap;
+  
+  private final DocValuesFormat delegate = new Lucene50DocValuesFormat();
+
+  private final PlanetModel planetModel;
+
+  /** Default constructor */
+  public Geo3DDocValuesFormat() {
+    this(PlanetModel.WGS84, BKD3DTreeWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE, BKD3DTreeWriter.DEFAULT_MAX_POINTS_SORT_IN_HEAP);
+  }
+
+  /** Creates this with custom configuration.
+   *
+   * @param planetModel the {@link PlanetModel} to use; this is only used when writing
+   * @param maxPointsInLeafNode Maximum number of points in each leaf cell.  Smaller values create a deeper tree with larger in-heap index and possibly
+   *    faster searching.  The default is 1024.
+   * @param maxPointsSortInHeap Maximum number of points where in-heap sort can be used.  When the number of points exceeds this, a (slower)
+   *    offline sort is used.  The default is 128 * 1024.
+   *
+   * @lucene.experimental */
+  public Geo3DDocValuesFormat(PlanetModel planetModel, int maxPointsInLeafNode, int maxPointsSortInHeap) {
+    super("BKD3DTree");
+    BKD3DTreeWriter.verifyParams(maxPointsInLeafNode, maxPointsSortInHeap);
+    this.maxPointsInLeafNode = maxPointsInLeafNode;
+    this.maxPointsSortInHeap = maxPointsSortInHeap;
+    this.planetModel = planetModel;
+  }
+
+  @Override
+  public DocValuesConsumer fieldsConsumer(final SegmentWriteState state) throws IOException {
+    return new Geo3DDocValuesConsumer(planetModel, delegate.fieldsConsumer(state), state, maxPointsInLeafNode, maxPointsSortInHeap);
+  }
+
+  @Override
+  public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new Geo3DDocValuesProducer(delegate.fieldsProducer(state), state);
+  }
+
+  /** Clips the incoming value to the allowed min/max range before encoding, instead of throwing an exception. */
+  static int encodeValueLenient(double planetMax, double x) {
+    if (x > planetMax) {
+      x = planetMax;
+    } else if (x < -planetMax) {
+      x = -planetMax;
+    }
+    return encodeValue(planetMax, x);
+  }
+    
+  static int encodeValue(double planetMax, double x) {
+    if (x > planetMax) {
+      throw new IllegalArgumentException("value=" + x + " is out-of-bounds (greater than planetMax=" + planetMax + ")");
+    }
+    if (x < -planetMax) {
+      throw new IllegalArgumentException("value=" + x + " is out-of-bounds (less than than -planetMax=" + -planetMax + ")");
+    }
+    long y = Math.round (x * (Integer.MAX_VALUE / planetMax));
+    assert y >= Integer.MIN_VALUE;
+    assert y <= Integer.MAX_VALUE;
+
+    return (int) y;
+  }
+
+  /** Center decode */
+  static double decodeValueCenter(double planetMax, int x) {
+    return x * (planetMax / Integer.MAX_VALUE);
+  }
+
+  /** More negative decode, at bottom of cell */
+  static double decodeValueMin(double planetMax, int x) {
+    return (((double)x) - 0.5) * (planetMax / Integer.MAX_VALUE);
+  }
+  
+  /** More positive decode, at top of cell  */
+  static double decodeValueMax(double planetMax, int x) {
+    return (((double)x) + 0.5) * (planetMax / Integer.MAX_VALUE);
+  }
+  
+
+  static int readInt(byte[] bytes, int offset) {
+    return ((bytes[offset] & 0xFF) << 24) | ((bytes[offset+1] & 0xFF) << 16)
+         | ((bytes[offset+2] & 0xFF) <<  8) |  (bytes[offset+3] & 0xFF);
+  }
+
+  static void writeInt(int value, byte[] bytes, int offset) {
+    bytes[offset] = (byte) ((value >> 24) & 0xff);
+    bytes[offset+1] = (byte) ((value >> 16) & 0xff);
+    bytes[offset+2] = (byte) ((value >> 8) & 0xff);
+    bytes[offset+3] = (byte) (value & 0xff);
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesProducer.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesProducer.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DDocValuesProducer.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,177 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.index.SortedSetDocValues;
+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.Bits;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+class Geo3DDocValuesProducer extends DocValuesProducer {
+
+  private final Map<String,BKD3DTreeReader> treeReaders = new HashMap<>();
+  private final Map<Integer,Long> fieldToIndexFPs = new HashMap<>();
+
+  private final IndexInput datIn;
+  private final AtomicLong ramBytesUsed;
+  private final int maxDoc;
+  private final DocValuesProducer delegate;
+  private final boolean merging;
+  private final double planetMax;
+
+  public Geo3DDocValuesProducer(DocValuesProducer delegate, SegmentReadState state) throws IOException {
+    String metaFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Geo3DDocValuesFormat.META_EXTENSION);
+    ChecksumIndexInput metaIn = state.directory.openChecksumInput(metaFileName, state.context);
+    CodecUtil.checkIndexHeader(metaIn, Geo3DDocValuesFormat.META_CODEC_NAME, Geo3DDocValuesFormat.META_VERSION_START, Geo3DDocValuesFormat.META_VERSION_CURRENT,
+                               state.segmentInfo.getId(), state.segmentSuffix);
+    int fieldCount = metaIn.readVInt();
+    for(int i=0;i<fieldCount;i++) {
+      int fieldNumber = metaIn.readVInt();
+      long indexFP = metaIn.readVLong();
+      fieldToIndexFPs.put(fieldNumber, indexFP);
+    }
+    CodecUtil.checkFooter(metaIn);
+    metaIn.close();
+
+    String datFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Geo3DDocValuesFormat.DATA_EXTENSION);
+    datIn = state.directory.openInput(datFileName, state.context);
+    CodecUtil.checkIndexHeader(datIn, Geo3DDocValuesFormat.DATA_CODEC_NAME, Geo3DDocValuesFormat.DATA_VERSION_START, Geo3DDocValuesFormat.DATA_VERSION_CURRENT,
+                               state.segmentInfo.getId(), state.segmentSuffix);
+    planetMax = Double.longBitsToDouble(datIn.readLong());
+    ramBytesUsed = new AtomicLong(RamUsageEstimator.shallowSizeOfInstance(getClass()));
+    maxDoc = state.segmentInfo.maxDoc();
+    this.delegate = delegate;
+    merging = false;
+  }
+
+  // clone for merge: we don't hang onto the Geo3Ds we load
+  Geo3DDocValuesProducer(Geo3DDocValuesProducer orig) throws IOException {
+    assert Thread.holdsLock(orig);
+    datIn = orig.datIn.clone();
+    ramBytesUsed = new AtomicLong(orig.ramBytesUsed.get());
+    delegate = orig.delegate.getMergeInstance();
+    fieldToIndexFPs.putAll(orig.fieldToIndexFPs);
+    treeReaders.putAll(orig.treeReaders);
+    merging = true;
+    maxDoc = orig.maxDoc;
+    planetMax = orig.planetMax;
+  }
+
+  @Override
+  public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(datIn, delegate);
+  }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(datIn);
+  }
+
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BKD3DTreeReader treeReader = treeReaders.get(field.name);
+    if (treeReader == null) {
+      // Lazy load
+      Long fp = fieldToIndexFPs.get(field.number);
+      if (fp == null) {
+        throw new IllegalArgumentException("this field was not indexed as a BKDPointField");
+      }
+
+      // LUCENE-6697: never do real IOPs with the original IndexInput because search
+      // threads can be concurrently cloning it:
+      IndexInput clone = datIn.clone();
+      clone.seek(fp);
+      treeReader = new BKD3DTreeReader(clone, maxDoc);
+
+      // Only hang onto the reader when we are not merging:
+      if (merging == false) {
+        treeReaders.put(field.name, treeReader);
+        ramBytesUsed.addAndGet(treeReader.ramBytesUsed());
+      }
+    }
+
+    return new Geo3DBinaryDocValues(treeReader, delegate.getBinary(field), planetMax);
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public SortedSetDocValues getSortedSet(FieldInfo field) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    return delegate.getDocsWithField(field);
+  }
+
+  @Override
+  public synchronized Collection<Accountable> getChildResources() {
+    List<Accountable> resources = new ArrayList<>();
+    for(Map.Entry<String,BKD3DTreeReader> ent : treeReaders.entrySet()) {
+      resources.add(Accountables.namedAccountable("field " + ent.getKey(), ent.getValue()));
+    }
+    resources.add(Accountables.namedAccountable("delegate", delegate));
+
+    return resources;
+  }
+
+  @Override
+  public synchronized DocValuesProducer getMergeInstance() throws IOException {
+    return new Geo3DDocValuesProducer(this);
+  }
+
+  @Override
+  public long ramBytesUsed() {
+    return ramBytesUsed.get() + delegate.ramBytesUsed();
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Geo3DPointField.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,70 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.geo3d.PlanetModel;
+import org.apache.lucene.geo3d.GeoPoint;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.util.BytesRef;
+
+// TODO: allow multi-valued, packing all points into a single BytesRef
+
+/** Add this to a document to index lat/lon point, but be sure to use {@link Geo3DDocValuesFormat} for the field.
+
+ *  @lucene.experimental */
+public final class Geo3DPointField extends Field {
+
+  /** Indexing {@link FieldType}. */
+  public static final FieldType TYPE = new FieldType();
+  static {
+    TYPE.setDocValuesType(DocValuesType.BINARY);
+    TYPE.freeze();
+  }
+
+  /** 
+   * Creates a new Geo3DPointField field with the specified lat, lon (in radians), given a planet model.
+   *
+   * @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
+   */
+  public Geo3DPointField(String name, PlanetModel planetModel, double lat, double lon) {
+    super(name, TYPE);
+    // Translate lat/lon to x,y,z:
+    final GeoPoint point = new GeoPoint(planetModel, lat, lon);
+    fillFieldsData(planetModel.getMaximumMagnitude(), point.x, point.y, point.z);
+  }
+
+  /** 
+   * Creates a new Geo3DPointField field with the specified x,y,z.
+   *
+   * @throws IllegalArgumentException if the field name is null or lat or lon are out of bounds
+   */
+  public Geo3DPointField(String name, PlanetModel planetModel, double x, double y, double z) {
+    super(name, TYPE);
+    fillFieldsData(planetModel.getMaximumMagnitude(), x, y, z);
+  }
+
+  private void fillFieldsData(double planetMax, double x, double y, double z) {
+    byte[] bytes = new byte[12];
+    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, x), bytes, 0);
+    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, y), bytes, 4);
+    Geo3DDocValuesFormat.writeInt(Geo3DDocValuesFormat.encodeValue(planetMax, z), bytes, 8);
+    fieldsData = new BytesRef(bytes);
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/GrowingHeapWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/GrowingHeapWriter.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/GrowingHeapWriter.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/GrowingHeapWriter.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,92 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.RamUsageEstimator;
+
+final class GrowingHeapWriter implements Writer {
+  int[] xs;
+  int[] ys;
+  int[] zs;
+  int[] docIDs;
+  long[] ords;
+  private int nextWrite;
+  final int maxSize;
+
+  public GrowingHeapWriter(int maxSize) {
+    xs = new int[16];
+    ys = new int[16];
+    zs = new int[16];
+    docIDs = new int[16];
+    ords = new long[16];
+    this.maxSize = maxSize;
+  }
+
+  private int[] growExact(int[] arr, int size) {
+    assert size > arr.length;
+    int[] newArr = new int[size];
+    System.arraycopy(arr, 0, newArr, 0, arr.length);
+    return newArr;
+  }
+
+  private long[] growExact(long[] arr, int size) {
+    assert size > arr.length;
+    long[] newArr = new long[size];
+    System.arraycopy(arr, 0, newArr, 0, arr.length);
+    return newArr;
+  }
+
+  @Override
+  public void append(int x, int y, int z, long ord, int docID) {
+    assert ord == nextWrite;
+    if (xs.length == nextWrite) {
+      int nextSize = Math.min(maxSize, ArrayUtil.oversize(nextWrite+1, RamUsageEstimator.NUM_BYTES_INT));
+      assert nextSize > nextWrite: "nextSize=" + nextSize + " vs nextWrite=" + nextWrite;
+      xs = growExact(xs, nextSize);
+      ys = growExact(ys, nextSize);
+      zs = growExact(zs, nextSize);
+      ords = growExact(ords, nextSize);
+      docIDs = growExact(docIDs, nextSize);
+    }
+    xs[nextWrite] = x;
+    ys[nextWrite] = y;
+    zs[nextWrite] = z;
+    ords[nextWrite] = ord;
+    docIDs[nextWrite] = docID;
+    nextWrite++;
+  }
+
+  @Override
+  public Reader getReader(long start) {
+    return new HeapReader(xs, ys, zs, ords, docIDs, (int) start, nextWrite);
+  }
+
+  @Override
+  public void close() {
+  }
+
+  @Override
+  public void destroy() {
+  }
+
+  @Override
+  public String toString() {
+    return "GrowingHeapWriter(count=" + nextWrite + " alloc=" + xs.length + ")";
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapReader.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapReader.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapReader.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,73 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+final class HeapReader implements Reader {
+  private int curRead;
+  final int[] xs;
+  final int[] ys;
+  final int[] zs;
+  final long[] ords;
+  final int[] docIDs;
+  final int end;
+
+  HeapReader(int[] xs, int[] ys, int[] zs, long[] ords, int[] docIDs, int start, int end) {
+    this.xs = xs;
+    this.ys = ys;
+    this.zs = zs;
+    this.ords = ords;
+    this.docIDs = docIDs;
+    curRead = start-1;
+    this.end = end;
+  }
+
+  @Override
+  public boolean next() {
+    curRead++;
+    return curRead < end;
+  }
+
+  @Override
+  public int x() {
+    return xs[curRead];
+  }
+
+  @Override
+  public int y() {
+    return ys[curRead];
+  }
+
+  @Override
+  public int z() {
+    return zs[curRead];
+  }
+
+  @Override
+  public int docID() {
+    return docIDs[curRead];
+  }
+
+  @Override
+  public long ord() {
+    return ords[curRead];
+  }
+
+  @Override
+  public void close() {
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/HeapWriter.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,69 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+final class HeapWriter implements Writer {
+  final int[] xs;
+  final int[] ys;
+  final int[] zs;
+  final int[] docIDs;
+  final long[] ords;
+  private int nextWrite;
+  private boolean closed;
+
+  public HeapWriter(int count) {
+    xs = new int[count];
+    ys = new int[count];
+    zs = new int[count];
+    docIDs = new int[count];
+    ords = new long[count];
+  }
+
+  @Override
+  public void append(int x, int y, int z, long ord, int docID) {
+    xs[nextWrite] = x;
+    ys[nextWrite] = y;
+    zs[nextWrite] = z;
+    ords[nextWrite] = ord;
+    docIDs[nextWrite] = docID;
+    nextWrite++;
+  }
+
+  @Override
+  public Reader getReader(long start) {
+    assert closed;
+    return new HeapReader(xs, ys, zs, ords, docIDs, (int) start, xs.length);
+  }
+
+  @Override
+  public void close() {
+    closed = true;
+    if (nextWrite != xs.length) {
+      throw new IllegalStateException("only wrote " + nextWrite + " values, but expected " + xs.length);
+    }
+  }
+
+  @Override
+  public void destroy() {
+  }
+
+  @Override
+  public String toString() {
+    return "HeapWriter(count=" + xs.length + ")";
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineReader.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,95 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.lucene.store.InputStreamDataInput;
+
+final class OfflineReader implements Reader {
+  final InputStreamDataInput in;
+  long countLeft;
+  private int x;
+  private int y;
+  private int z;
+  private long ord;
+  private int docID;
+
+  OfflineReader(Path tempFile, long start, long count) throws IOException {
+    InputStream fis = Files.newInputStream(tempFile);
+    long seekFP = start * BKD3DTreeWriter.BYTES_PER_DOC;
+    long skipped = 0;
+    while (skipped < seekFP) {
+      long inc = fis.skip(seekFP - skipped);
+      skipped += inc;
+      if (inc == 0) {
+        throw new RuntimeException("skip returned 0");
+      }
+    }
+    in = new InputStreamDataInput(new BufferedInputStream(fis));
+    this.countLeft = count;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (countLeft == 0) {
+      return false;
+    }
+    countLeft--;
+    x = in.readInt();
+    y = in.readInt();
+    z = in.readInt();
+    ord = in.readLong();
+    docID = in.readInt();
+    return true;
+  }
+
+  @Override
+  public int x() {
+    return x;
+  }
+
+  @Override
+  public int y() {
+    return y;
+  }
+
+  @Override
+  public int z() {
+    return z;
+  }
+
+  @Override
+  public long ord() {
+    return ord;
+  }
+
+  @Override
+  public int docID() {
+    return docID;
+  }
+
+  @Override
+  public void close() throws IOException {
+    in.close();
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/OfflineWriter.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,79 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.IOUtils;
+
+final class OfflineWriter implements Writer {
+
+  final Path tempFile;
+  final byte[] scratchBytes = new byte[BKD3DTreeWriter.BYTES_PER_DOC];
+  final ByteArrayDataOutput scratchBytesOutput = new ByteArrayDataOutput(scratchBytes);      
+  final OutputStreamDataOutput out;
+  final long count;
+  private long countWritten;
+  private boolean closed;
+
+  public OfflineWriter(Path tempDir, long count) throws IOException {
+    tempFile = Files.createTempFile(tempDir, "size" + count + ".", "");
+    out = new OutputStreamDataOutput(new BufferedOutputStream(Files.newOutputStream(tempFile)));
+    this.count = count;
+  }
+    
+  @Override
+  public void append(int x, int y, int z, long ord, int docID) throws IOException {
+    out.writeInt(x);
+    out.writeInt(y);
+    out.writeInt(z);
+    out.writeLong(ord);
+    out.writeInt(docID);
+    countWritten++;
+  }
+
+  @Override
+  public Reader getReader(long start) throws IOException {
+    assert closed;
+    return new OfflineReader(tempFile, start, count-start);
+  }
+
+  @Override
+  public void close() throws IOException {
+    closed = true;
+    out.close();
+    if (count != countWritten) {
+      throw new IllegalStateException("wrote " + countWritten + " values, but expected " + count);
+    }
+  }
+
+  @Override
+  public void destroy() throws IOException {
+    IOUtils.rm(tempFile);
+  }
+
+  @Override
+  public String toString() {
+    return "OfflineWriter(count=" + count + " tempFile=" + tempFile + ")";
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/PointInGeo3DShapeQuery.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,224 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import org.apache.lucene.geo3d.GeoArea;
+import org.apache.lucene.geo3d.GeoAreaFactory;
+import org.apache.lucene.geo3d.GeoShape;
+import org.apache.lucene.geo3d.PlanetModel;
+import org.apache.lucene.geo3d.XYZBounds;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
+
+import java.io.IOException;
+
+/** Finds all previously indexed points that fall within the specified polygon.
+ *
+ *  <p>The field must be indexed with {@link Geo3DDocValuesFormat}, and {@link Geo3DPointField} added per document.
+ *
+ *  <p>Because this implementation cannot intersect each cell with the polygon, it will be costly especially for large polygons, as every
+ *   possible point must be checked.
+ *
+ *  <p><b>NOTE</b>: for fastest performance, this allocates FixedBitSet(maxDoc) for each segment.  The score of each hit is the query boost.
+ *
+ * @lucene.experimental */
+
+public class PointInGeo3DShapeQuery extends Query {
+  final String field;
+  final PlanetModel planetModel;
+  final GeoShape shape;
+
+  /** The lats/lons must be clockwise or counter-clockwise. */
+  public PointInGeo3DShapeQuery(PlanetModel planetModel, String field, GeoShape shape) {
+    this.field = field;
+    this.planetModel = planetModel;
+    this.shape = shape;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+
+    // I don't use RandomAccessWeight here: it's no good to approximate with "match all docs"; this is an inverted structure and should be
+    // used in the first pass:
+
+    return new ConstantScoreWeight(this) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        LeafReader reader = context.reader();
+        BinaryDocValues bdv = reader.getBinaryDocValues(field);
+        if (bdv == null) {
+          // No docs in this segment had this field
+          return null;
+        }
+
+        if (bdv instanceof Geo3DBinaryDocValues == false) {
+          throw new IllegalStateException("field \"" + field + "\" was not indexed with Geo3DBinaryDocValuesFormat: got: " + bdv);
+        }
+        final Geo3DBinaryDocValues treeDV = (Geo3DBinaryDocValues) bdv;
+        BKD3DTreeReader tree = treeDV.getBKD3DTreeReader();
+
+        XYZBounds bounds = new XYZBounds();
+        shape.getBounds(bounds);
+
+        final double planetMax = planetModel.getMaximumMagnitude();
+        if (planetMax != treeDV.planetMax) {
+          throw new IllegalStateException(planetModel + " is not the same one used during indexing: planetMax=" + planetMax + " vs indexing planetMax=" + treeDV.planetMax);
+        }
+
+        /*
+        GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel,
+                                                      bounds.getMinimumX(),
+                                                      bounds.getMaximumX(),
+                                                      bounds.getMinimumY(),
+                                                      bounds.getMaximumY(),
+                                                      bounds.getMinimumZ(),
+                                                      bounds.getMaximumZ());
+
+        assert xyzSolid.getRelationship(shape) == GeoArea.WITHIN || xyzSolid.getRelationship(shape) == GeoArea.OVERLAPS: "expected WITHIN (1) or OVERLAPS (2) but got " + xyzSolid.getRelationship(shape) + "; shape="+shape+"; XYZSolid="+xyzSolid;
+        */
+
+        DocIdSet result = tree.intersect(Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumX()),
+                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumX()),
+                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumY()),
+                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumY()),
+                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMinimumZ()),
+                                         Geo3DDocValuesFormat.encodeValueLenient(planetMax, bounds.getMaximumZ()),
+                                         new BKD3DTreeReader.ValueFilter() {
+                                           @Override
+                                           public boolean accept(int docID) {
+                                             //System.out.println("  accept? docID=" + docID);
+                                             BytesRef bytes = treeDV.get(docID);
+                                             if (bytes == null) {
+                                               //System.out.println("    false (null)");
+                                               return false;
+                                             }
+
+                                             assert bytes.length == 12;
+                                             double x = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset));
+                                             double y = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+4));
+                                             double z = Geo3DDocValuesFormat.decodeValueCenter(treeDV.planetMax, Geo3DDocValuesFormat.readInt(bytes.bytes, bytes.offset+8));
+                                             // System.out.println("  accept docID=" + docID + " point: x=" + x + " y=" + y + " z=" + z);
+
+                                             // True if x,y,z is within shape
+                                             //System.out.println("    x=" + x + " y=" + y + " z=" + z);
+                                             //System.out.println("    ret: " + shape.isWithin(x, y, z));
+
+                                             return shape.isWithin(x, y, z);
+                                           }
+
+                                           @Override
+                                           public BKD3DTreeReader.Relation compare(int cellXMinEnc, int cellXMaxEnc, int cellYMinEnc, int cellYMaxEnc, int cellZMinEnc, int cellZMaxEnc) {
+                                             assert cellXMinEnc <= cellXMaxEnc;
+                                             assert cellYMinEnc <= cellYMaxEnc;
+                                             assert cellZMinEnc <= cellZMaxEnc;
+
+                                             // Because the BKD tree operates in quantized (64 bit -> 32 bit) space, and the cell bounds
+                                             // here are inclusive, we need to extend the bounds to the largest un-quantized values that
+                                             // could quantize into these bounds.  The encoding (Geo3DDocValuesFormat.encodeValue) does
+                                             // a Math.round from double to long, so e.g. 1.4 -> 1, and -1.4 -> -1:
+                                             double cellXMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellXMinEnc);
+                                             double cellXMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellXMaxEnc);
+                                             double cellYMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellYMinEnc);
+                                             double cellYMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellYMaxEnc);
+                                             double cellZMin = Geo3DDocValuesFormat.decodeValueMin(treeDV.planetMax, cellZMinEnc);
+                                             double cellZMax = Geo3DDocValuesFormat.decodeValueMax(treeDV.planetMax, cellZMaxEnc);
+                                             //System.out.println("  compare: x=" + cellXMin + "-" + cellXMax + " y=" + cellYMin + "-" + cellYMax + " z=" + cellZMin + "-" + cellZMax);
+
+                                             GeoArea xyzSolid = GeoAreaFactory.makeGeoArea(planetModel, cellXMin, cellXMax, cellYMin, cellYMax, cellZMin, cellZMax);
+
+                                             switch(xyzSolid.getRelationship(shape)) {
+                                             case GeoArea.CONTAINS:
+                                               // Shape fully contains the cell
+                                               //System.out.println("    inside");
+                                               return BKD3DTreeReader.Relation.CELL_INSIDE_SHAPE;
+                                             case GeoArea.OVERLAPS:
+                                               // They do overlap but neither contains the other:
+                                               //System.out.println("    crosses1");
+                                               return BKD3DTreeReader.Relation.SHAPE_CROSSES_CELL;
+                                             case GeoArea.WITHIN:
+                                               // Cell fully contains the shape:
+                                               //System.out.println("    crosses2");
+                                               return BKD3DTreeReader.Relation.SHAPE_INSIDE_CELL;
+                                             case GeoArea.DISJOINT:
+                                               // They do not overlap at all
+                                               //System.out.println("    outside");
+                                               return BKD3DTreeReader.Relation.SHAPE_OUTSIDE_CELL;
+                                             default:
+                                               assert false;
+                                               return BKD3DTreeReader.Relation.SHAPE_CROSSES_CELL;
+                                             }
+                                           }
+                                         });
+
+        final DocIdSetIterator disi = result.iterator();
+
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+    };
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked","rawtypes"})
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    PointInGeo3DShapeQuery that = (PointInGeo3DShapeQuery) o;
+
+    return planetModel.equals(that.planetModel) && shape.equals(that.shape);
+  }
+
+  @Override
+  public final int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + planetModel.hashCode();
+    result = 31 * result + shape.hashCode();
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(getClass().getSimpleName());
+    sb.append(':');
+    if (this.field.equals(field) == false) {
+      sb.append(" field=");
+      sb.append(this.field);
+      sb.append(':');
+    }
+    sb.append("PlanetModel: ");
+    sb.append(planetModel);
+    sb.append(" Shape: ");
+    sb.append(shape);
+    sb.append(ToStringUtils.boost(getBoost()));
+    return sb.toString();
+  }
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Reader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Reader.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Reader.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Reader.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,31 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/** Abstracts away whether OfflineSorter or simple arrays in heap are used. */
+interface Reader extends Closeable {
+  boolean next() throws IOException;
+  int x();
+  int y();
+  int z();
+  long ord();
+  int docID();
+}

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Writer.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Writer.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/Writer.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,29 @@
+package org.apache.lucene.bkdtree3d;
+
+/*
+ * 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.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/** Abstracts away whether OfflineSorter or simple arrays in heap are used. */
+interface Writer extends Closeable {
+  void append(int x, int y, int z, long ord, int docID) throws IOException;
+  Reader getReader(long start) throws IOException;
+  void destroy() throws IOException;
+}
+

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/package-info.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/package-info.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/bkdtree3d/package-info.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * Fast "indexed point inside geo3d shape" query implementation.
+ */
+package org.apache.lucene.bkdtree3d;

Added: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/BaseXYZSolid.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/BaseXYZSolid.java?rev=1700883&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/BaseXYZSolid.java (added)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/BaseXYZSolid.java Wed Sep  2 19:59:31 2015
@@ -0,0 +1,168 @@
+package org.apache.lucene.geo3d;
+
+/*
+ * 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.
+ */
+
+/**
+ * Base class of a family of 3D rectangles, bounded on six sides by X,Y,Z limits
+ *
+ * @lucene.internal
+ */
+public abstract class BaseXYZSolid extends BasePlanetObject implements GeoArea {
+
+  /** Unit vector in x */
+  protected static final Vector xUnitVector = new Vector(1.0, 0.0, 0.0);
+  /** Unit vector in y */
+  protected static final Vector yUnitVector = new Vector(0.0, 1.0, 0.0);
+  /** Unit vector in z */
+  protected static final Vector zUnitVector = new Vector(0.0, 0.0, 1.0);
+  
+  /** Vertical plane normal to x unit vector passing through origin */
+  protected static final Plane xVerticalPlane = new Plane(0.0, 1.0, 0.0, 0.0);
+  /** Vertical plane normal to y unit vector passing through origin */
+  protected static final Plane yVerticalPlane = new Plane(1.0, 0.0, 0.0, 0.0);
+
+  /** Empty point vector */
+  protected static final GeoPoint[] EMPTY_POINTS = new GeoPoint[0];
+  
+  /**
+   * Base solid constructor.
+   *@param planetModel is the planet model.
+   */
+  public BaseXYZSolid(final PlanetModel planetModel) {
+    super(planetModel);
+  }
+  
+  /** Construct a single array from a number of individual arrays.
+   * @param pointArrays is the array of point arrays.
+   * @return the single unified array.
+   */
+  protected static GeoPoint[] glueTogether(final GeoPoint[]... pointArrays) {
+    int count = 0;
+    for (final GeoPoint[] pointArray : pointArrays) {
+      count += pointArray.length;
+    }
+    final GeoPoint[] rval = new GeoPoint[count];
+    count = 0;
+    for (final GeoPoint[] pointArray : pointArrays) {
+      for (final GeoPoint point : pointArray) {
+        rval[count++] = point;
+      }
+    }
+    return rval;
+  }
+  
+  @Override
+  public boolean isWithin(final Vector point) {
+    return isWithin(point.x, point.y, point.z);
+  }
+  
+  @Override
+  public abstract boolean isWithin(final double x, final double y, final double z);
+  
+  // Signals for relationship of edge points to shape
+  
+  /** All edgepoints inside shape */
+  protected final static int ALL_INSIDE = 0;
+  /** Some edgepoints inside shape */
+  protected final static int SOME_INSIDE = 1;
+  /** No edgepoints inside shape */
+  protected final static int NONE_INSIDE = 2;
+  /** No edgepoints at all (means a shape that is the whole world) */
+  protected final static int NO_EDGEPOINTS = 3;
+
+  /** Determine the relationship between this area and the provided
+   * shape's edgepoints.
+   *@param path is the shape.
+   *@return the relationship.
+   */
+  protected int isShapeInsideArea(final GeoShape path) {
+    final GeoPoint[] pathPoints = path.getEdgePoints();
+    if (pathPoints.length == 0)
+      return NO_EDGEPOINTS;
+    boolean foundOutside = false;
+    boolean foundInside = false;
+    for (final GeoPoint p : pathPoints) {
+      if (isWithin(p)) {
+        foundInside = true;
+      } else {
+        foundOutside = true;
+      }
+      if (foundInside && foundOutside) {
+        return SOME_INSIDE;
+      }
+    }
+    if (!foundInside && !foundOutside)
+      return NONE_INSIDE;
+    if (foundInside && !foundOutside)
+      return ALL_INSIDE;
+    if (foundOutside && !foundInside)
+      return NONE_INSIDE;
+    return SOME_INSIDE;
+  }
+
+  /** Determine the relationship between a shape and this area's
+   * edgepoints.
+   *@param path is the shape.
+   *@return the relationship.
+   */
+  protected int isAreaInsideShape(final GeoShape path) {
+    final GeoPoint[] edgePoints = getEdgePoints();
+    if (edgePoints.length == 0) {
+      return NO_EDGEPOINTS;
+    }
+    boolean foundOutside = false;
+    boolean foundInside = false;
+    for (final GeoPoint p : edgePoints) {
+      if (path.isWithin(p)) {
+        foundInside = true;
+      } else {
+        foundOutside = true;
+      }
+      if (foundInside && foundOutside) {
+        return SOME_INSIDE;
+      }
+    }
+    if (!foundInside && !foundOutside)
+      return NONE_INSIDE;
+    if (foundInside && !foundOutside)
+      return ALL_INSIDE;
+    if (foundOutside && !foundInside)
+      return NONE_INSIDE;
+    return SOME_INSIDE;
+  }
+
+  /** Get the edge points for this shape.
+   *@return the edge points.
+   */
+  protected abstract GeoPoint[] getEdgePoints();
+  
+  @Override
+  public boolean equals(Object o) {
+    if (!(o instanceof BaseXYZSolid))
+      return false;
+    BaseXYZSolid other = (BaseXYZSolid) o;
+    return super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+}
+  

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Bounds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Bounds.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Bounds.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/Bounds.java Wed Sep  2 19:59:31 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.geo3d;
  */
 
 /**
- * An object for accumulating bounds information.
+ * An interface for accumulating bounds information.
  * The bounds object is initially empty.  Bounding points
  * are then applied by supplying (x,y,z) tuples.  It is also
  * possible to indicate the following edge cases:
@@ -30,356 +30,85 @@ package org.apache.lucene.geo3d;
  *
  * @lucene.experimental
  */
-public class Bounds {
+public interface Bounds {
 
-  /** Set to true if no longitude bounds can be stated */
-  protected boolean noLongitudeBound = false;
-  /** Set to true if no top latitude bound can be stated */
-  protected boolean noTopLatitudeBound = false;
-  /** Set to true if no bottom latitude bound can be stated */
-  protected boolean noBottomLatitudeBound = false;
-
-  /** If non-null, the minimum latitude bound */
-  protected Double minLatitude = null;
-  /** If non-null, the maximum latitude bound */
-  protected Double maxLatitude = null;
-
-  // For longitude bounds, this class needs to worry about keeping track of the distinction
-  // between left-side bounds and right-side bounds.  Points are always submitted in pairs
-  // which have a maximum longitude separation of Math.PI.  It's therefore always possible
-  // to determine which point represents a left bound, and which point represents a right
-  // bound.
-  //
-  // The next problem is how to compare two of the same kind of bound, e.g. two left bounds.
-  // We need to keep track of the leftmost longitude of the shape, but since this is a circle,
-  // this is arbitrary.  What we could try to do instead would be to find a pair of (left,right) bounds such
-  // that:
-  // (1) all other bounds are within, and
-  // (2) the left minus right distance is minimized
-  // Unfortunately, there are still shapes that cannot be summarized in this way correctly.
-  // For example. consider a spiral that entirely circles the globe; we might arbitrarily choose
-  // lat/lon bounds that do not in fact circle the globe.
-  //
-  // One way to handle the longitude issue correctly is therefore to stipulate that we
-  // walk the bounds of the shape in some kind of connected order.  Each point or circle is therefore
-  // added in a sequence.  We also need an interior point to make sure we have the right
-  // choice of longitude bounds.  But even with this, we still can't always choose whether the actual shape
-  // goes right or left.
-  //
-  // We can make the specification truly general by submitting the following in order:
-  // addSide(PlaneSide side, Membership... constraints)
-  // ...
-  // This is unambiguous, but I still can't see yet how this would help compute the bounds.  The plane
-  // solution would in general seem to boil down to the same logic that relies on points along the path
-  // to define the shape boundaries.  I guess the one thing that you do know for a bounded edge is that
-  // the endpoints are actually connected.  But it is not clear whether relationship helps in any way.
-  //
-  // In any case, if we specify shapes by a sequence of planes, we should stipulate that multiple sequences
-  // are allowed, provided they progressively tile an area of the sphere that is connected and sequential.
-  // For example, paths do alternating rectangles and circles, in sequence.  Each sequence member is
-  // described by a sequence of planes.  I think it would also be reasonable to insist that the first segment
-  // of a shape overlap or adjoin the previous shape.
-  //
-  // Here's a way to think about it that might help: Traversing every edge should grow the longitude bounds
-  // in the direction of the traversal.  So if the traversal is always known to be less than PI in total longitude
-  // angle, then it is possible to use the endpoints to determine the unambiguous extension of the envelope.
-  // For example, say you are currently at longitude -0.5.  The next point is at longitude PI-0.1.  You could say
-  // that the difference in longitude going one way around would be beter than the distance the other way
-  // around, and therefore the longitude envelope should be extended accordingly.  But in practice, when an
-  // edge goes near a pole and may be inclined as well, the longer longitude change might be the right path, even
-  // if the arc length is short.  So this too doesn't work.
-  //
-  // Given we have a hard time making an exact match, here's the current proposal.  The proposal is a
-  // heuristic, based on the idea that most areas are small compared to the circumference of the globe.
-  // We keep track of the last point we saw, and take each point as it arrives, and compute its longitude.
-  // Then, we have a choice as to which way to expand the envelope: we can expand by going to the left or
-  // to the right.  We choose the direction with the least longitude difference.  (If we aren't sure,
-  // and can recognize that, we can set "unconstrained in longitude".)
-
-  /** If non-null, the left longitude bound */
-  protected Double leftLongitude = null;
-  /** If non-null, the right longitude bound */
-  protected Double rightLongitude = null;
-
-  /** Construct an empty bounds object */
-  public Bounds() {
-  }
-
-  /** Get maximum latitude, if any.
-   *@return maximum latitude or null.
-   */
-  public Double getMaxLatitude() {
-    return maxLatitude;
-  }
-
-  /** Get minimum latitude, if any.
-   *@return minimum latitude or null.
-   */
-  public Double getMinLatitude() {
-    return minLatitude;
-  }
-
-  /** Get left longitude, if any.
-   *@return left longitude, or null.
-   */
-  public Double getLeftLongitude() {
-    return leftLongitude;
-  }
-
-  /** Get right longitude, if any.
-   *@return right longitude, or null.
-   */
-  public Double getRightLongitude() {
-    return rightLongitude;
-  }
-
-  /** Check if there's no longitude bound.
-   *@return true if no longitude bound.
-   */
-  public boolean checkNoLongitudeBound() {
-    return noLongitudeBound;
-  }
-
-  /** Check if there's no top latitude bound.
-   *@return true if no top latitude bound.
-   */
-  public boolean checkNoTopLatitudeBound() {
-    return noTopLatitudeBound;
-  }
-
-  /** Check if there's no bottom latitude bound.
-   *@return true if no bottom latitude bound.
-   */
-  public boolean checkNoBottomLatitudeBound() {
-    return noBottomLatitudeBound;
-  }
-
-  /** Add a constraint representing a horizontal circle with a
-   * specified z value.
-   *@param z is the z value.
-   *@return the updated Bounds object.
-   */
-  public Bounds addHorizontalCircle(double z) {
-    if (!noTopLatitudeBound || !noBottomLatitudeBound) {
-      // Compute a latitude value
-      double latitude = Math.asin(z);
-      addLatitudeBound(latitude);
-    }
-    return this;
-  }
-
-  /** Add a constraint representing a horizontal circle at
-   * a specific latitude.
+  /** Add a general plane to the bounds description.
+   *@param planetModel is the planet model.
+   *@param plane is the plane.
+   *@param bounds are the membership bounds for points along the arc.
+   */
+  public Bounds addPlane(final PlanetModel planetModel, final Plane plane, final Membership... bounds);
+  
+  /** Add a horizontal plane to the bounds description.
+   * This method should EITHER use the supplied latitude, OR use the supplied
+   * plane, depending on what is most efficient.
+   *@param planetModel is the planet model.
    *@param latitude is the latitude.
-   *@return the updated Bounds object.
-   */
-  public Bounds addLatitudeZone(double latitude) {
-    if (!noTopLatitudeBound || !noBottomLatitudeBound) {
-      addLatitudeBound(latitude);
-    }
-    return this;
-  }
+   *@param horizontalPlane is the plane.
+   *@param bounds are the constraints on the plane.
+   *@return updated Bounds object.
+   */
+  public Bounds addHorizontalPlane(final PlanetModel planetModel,
+    final double latitude,
+    final Plane horizontalPlane,
+    final Membership... bounds);
+    
+  /** Add a vertical plane to the bounds description.
+   * This method should EITHER use the supplied longitude, OR use the supplied
+   * plane, depending on what is most efficient.
+   *@param planetModel is the planet model.
+   *@param longitude is the longitude.
+   *@param verticalPlane is the plane.
+   *@param bounds are the constraints on the plane.
+   *@return updated Bounds object.
+   */
+  public Bounds addVerticalPlane(final PlanetModel planetModel,
+    final double longitude,
+    final Plane verticalPlane,
+    final Membership... bounds);
 
-  /** Add a constraint representing a longitude slice.
-   *@param newLeftLongitude is the left longitude value.
-   *@param newRightLongitude is the right longitude value.
+  /** Add a single point.
+   *@param point is the point.
    *@return the updated Bounds object.
    */
-  public Bounds addLongitudeSlice(double newLeftLongitude, double newRightLongitude) {
-    if (!noLongitudeBound) {
-      addLongitudeBound(newLeftLongitude, newRightLongitude);
-    }
-    return this;
-  }
-
-  /** Update latitude bound.
-   *@param latitude is the latitude.
-   */
-  protected void addLatitudeBound(double latitude) {
-    if (!noTopLatitudeBound && (maxLatitude == null || latitude > maxLatitude))
-      maxLatitude = latitude;
-    if (!noBottomLatitudeBound && (minLatitude == null || latitude < minLatitude))
-      minLatitude = latitude;
-  }
+  public Bounds addPoint(final GeoPoint point);
 
-  /** Update longitude bound.
-   *@param newLeftLongitude is the left longitude.
-   *@param newRightLongitude is the right longitude.
-   */
-  protected void addLongitudeBound(double newLeftLongitude, double newRightLongitude) {
-    if (leftLongitude == null && rightLongitude == null) {
-      leftLongitude = newLeftLongitude;
-      rightLongitude = newRightLongitude;
-    } else {
-      // Map the current range to something monotonically increasing
-      double currentLeftLongitude = leftLongitude;
-      double currentRightLongitude = rightLongitude;
-      if (currentRightLongitude < currentLeftLongitude)
-        currentRightLongitude += 2.0 * Math.PI;
-      double adjustedLeftLongitude = newLeftLongitude;
-      double adjustedRightLongitude = newRightLongitude;
-      if (adjustedRightLongitude < adjustedLeftLongitude)
-        adjustedRightLongitude += 2.0 * Math.PI;
-      // Compare to see what the relationship is
-      if (currentLeftLongitude <= adjustedLeftLongitude && currentRightLongitude >= adjustedRightLongitude) {
-        // No adjustment needed.
-      } else if (currentLeftLongitude >= adjustedLeftLongitude && currentRightLongitude <= adjustedRightLongitude) {
-        // New longitude entirely contains old one
-        leftLongitude = newLeftLongitude;
-        rightLongitude = newRightLongitude;
-      } else {
-        if (currentLeftLongitude > adjustedLeftLongitude) {
-          // New left longitude needed
-          leftLongitude = newLeftLongitude;
-        }
-        if (currentRightLongitude < adjustedRightLongitude) {
-          // New right longitude needed
-          rightLongitude = newRightLongitude;
-        }
-      }
-    }
-    double testRightLongitude = rightLongitude;
-    if (testRightLongitude < leftLongitude)
-      testRightLongitude += Math.PI * 2.0;
-    // If the bound exceeds 180 degrees, we know we could have screwed up.
-    if (testRightLongitude - leftLongitude >= Math.PI) {
-      noLongitudeBound = true;
-      leftLongitude = null;
-      rightLongitude = null;
-    }
-  }
-
-  /** Update longitude bound.
-   *@param longitude is the new longitude value.
+  /** Add an X value.
+   *@param point is the point to take the x value from.
+   *@return the updated object.
    */
-  protected void addLongitudeBound(double longitude) {
-    // If this point is within the current bounds, we're done; otherwise
-    // expand one side or the other.
-    if (leftLongitude == null && rightLongitude == null) {
-      leftLongitude = longitude;
-      rightLongitude = longitude;
-    } else {
-      // Compute whether we're to the right of the left value.  But the left value may be greater than
-      // the right value.
-      double currentLeftLongitude = leftLongitude;
-      double currentRightLongitude = rightLongitude;
-      if (currentRightLongitude < currentLeftLongitude)
-        currentRightLongitude += 2.0 * Math.PI;
-      // We have a range to look at that's going in the right way.
-      // Now, do the same trick with the computed longitude.
-      if (longitude < currentLeftLongitude)
-        longitude += 2.0 * Math.PI;
-
-      if (longitude < currentLeftLongitude || longitude > currentRightLongitude) {
-        // Outside of current bounds.  Consider carefully how we'll expand.
-        double leftExtensionAmt;
-        double rightExtensionAmt;
-        if (longitude < currentLeftLongitude) {
-          leftExtensionAmt = currentLeftLongitude - longitude;
-        } else {
-          leftExtensionAmt = currentLeftLongitude + 2.0 * Math.PI - longitude;
-        }
-        if (longitude > currentRightLongitude) {
-          rightExtensionAmt = longitude - currentRightLongitude;
-        } else {
-          rightExtensionAmt = longitude + 2.0 * Math.PI - currentRightLongitude;
-        }
-        if (leftExtensionAmt < rightExtensionAmt) {
-          currentLeftLongitude = leftLongitude - leftExtensionAmt;
-          while (currentLeftLongitude <= -Math.PI) {
-            currentLeftLongitude += 2.0 * Math.PI;
-          }
-          leftLongitude = currentLeftLongitude;
-        } else {
-          currentRightLongitude = rightLongitude + rightExtensionAmt;
-          while (currentRightLongitude > Math.PI) {
-            currentRightLongitude -= 2.0 * Math.PI;
-          }
-          rightLongitude = currentRightLongitude;
-        }
-      }
-    }
-    double testRightLongitude = rightLongitude;
-    if (testRightLongitude < leftLongitude)
-      testRightLongitude += Math.PI * 2.0;
-    if (testRightLongitude - leftLongitude >= Math.PI) {
-      noLongitudeBound = true;
-      leftLongitude = null;
-      rightLongitude = null;
-    }
-  }
+  public Bounds addXValue(final GeoPoint point);
 
-  /** Add a single point.
-   *@param v is the point vector.
-   *@return the updated Bounds object.
+  /** Add a Y value.
+   *@param point is the point to take the y value from.
+   *@return the updated object.
    */
-  public Bounds addPoint(final Vector v) {
-    return addPoint(v.x, v.y, v.z);
-  }
+  public Bounds addYValue(final GeoPoint point);
 
-  /** Add a single point.
-   *@param x is the point x.
-   *@param y is the point y.
-   *@param z is the point z.
-   *@return the updated Bounds object.
+  /** Add a Z value.
+   *@param point is the point to take the z value from.
+   *@return the updated object.
    */
-  public Bounds addPoint(final double x, final double y, final double z) {
-    if (!noLongitudeBound) {
-      // Get a longitude value
-      double longitude = Math.atan2(y, x);
-      //System.err.println(" add longitude bound at "+longitude * 180.0/Math.PI);
-      addLongitudeBound(longitude);
-    }
-    if (!noTopLatitudeBound || !noBottomLatitudeBound) {
-      // Compute a latitude value
-      double latitude = Math.asin(z/Math.sqrt(z * z + x * x + y * y));
-      addLatitudeBound(latitude);
-    }
-    return this;
-  }
-
-  /** Add a single point.
-   *@param latitude is the point's latitude.
-   *@param longitude is the point's longitude.
+  public Bounds addZValue(final GeoPoint point);
+  
+  /** Signal that the shape exceeds Math.PI in longitude.
    *@return the updated Bounds object.
    */
-  public Bounds addPoint(double latitude, double longitude) {
-    if (!noLongitudeBound) {
-      // Get a longitude value
-      addLongitudeBound(longitude);
-    }
-    if (!noTopLatitudeBound || !noBottomLatitudeBound) {
-      // Compute a latitude value
-      addLatitudeBound(latitude);
-    }
-    return this;
-  }
-
+  public Bounds isWide();
+  
   /** Signal that there is no longitude bound.
    *@return the updated Bounds object.
    */
-  public Bounds noLongitudeBound() {
-    noLongitudeBound = true;
-    leftLongitude = null;
-    rightLongitude = null;
-    return this;
-  }
+  public Bounds noLongitudeBound();
 
   /** Signal that there is no top latitude bound.
    *@return the updated Bounds object.
    */
-  public Bounds noTopLatitudeBound() {
-    noTopLatitudeBound = true;
-    maxLatitude = null;
-    return this;
-  }
+  public Bounds noTopLatitudeBound();
 
   /** Signal that there is no bottom latitude bound.
    *@return the updated Bounds object.
    */
-  public Bounds noBottomLatitudeBound() {
-    noBottomLatitudeBound = true;
-    minLatitude = null;
-    return this;
-  }
+  public Bounds noBottomLatitudeBound();
+  
 }

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoArea.java Wed Sep  2 19:59:31 2015
@@ -33,13 +33,13 @@ public interface GeoArea extends Members
 
   // Relationship values for "getRelationship()"
   
-  /** The referenced shape CONTAINS this shape */
+  /** The referenced shape CONTAINS this area */
   public static final int CONTAINS = 0;
-  /** The referenced shape IS WITHIN this shape */
+  /** The referenced shape IS WITHIN this area */
   public static final int WITHIN = 1;
-  /** The referenced shape OVERLAPS this shape */
+  /** The referenced shape OVERLAPS this area */
   public static final int OVERLAPS = 2;
-  /** The referenced shape has no relation to this shape */
+  /** The referenced shape has no relation to this area */
   public static final int DISJOINT = 3;
 
   /**
@@ -48,10 +48,17 @@ public interface GeoArea extends Members
    * other way around. For example, if this GeoArea is entirely within the
    * shape, then CONTAINS should be returned.  If the shape is entirely enclosed
    * by this GeoArea, then WITHIN should be returned.
-   * Note well: When a shape consists of multiple independent overlapping subshapes,
-   * it is sometimes impossible to determine the distinction between
-   * OVERLAPS and CONTAINS.  In that case, OVERLAPS may be returned even
-   * though the proper result would in fact be CONTAINS.  Code accordingly.
+   *
+   * It is permissible to return OVERLAPS instead of WITHIN if the shape
+   * intersects with the area at even a single point.  So, a circle inscribed in
+   * a rectangle could return either OVERLAPS or WITHIN, depending on
+   * implementation.  It is not permissible to return CONTAINS or DISJOINT
+   * in this circumstance, however.
+   *
+   * Similarly, it is permissible to return OVERLAPS instead of CONTAINS
+   * under conditions where the shape consists of multiple independent overlapping
+   * subshapes, and the area overlaps one of the subshapes.  It is not permissible
+   * to return WITHIN or DISJOINT in this circumstance, however.
    *
    * @param shape is the shape to consider.
    * @return the relationship, from the perspective of the shape.

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoAreaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoAreaFactory.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoAreaFactory.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoAreaFactory.java Wed Sep  2 19:59:31 2015
@@ -28,7 +28,7 @@ public class GeoAreaFactory {
 
   /**
    * Create a GeoArea of the right kind given the specified bounds.
-   *
+   * @param planetModel is the planet model
    * @param topLat    is the top latitude
    * @param bottomLat is the bottom latitude
    * @param leftLon   is the left longitude
@@ -39,4 +39,43 @@ public class GeoAreaFactory {
     return GeoBBoxFactory.makeGeoBBox(planetModel, topLat, bottomLat, leftLon, rightLon);
   }
 
+  /**
+   * Create a GeoArea of the right kind given (x,y,z) bounds.
+   * @param planetModel is the planet model
+   * @param minX is the min X boundary
+   * @param maxX is the max X boundary
+   * @param minY is the min Y boundary
+   * @param maxY is the max Y boundary
+   * @param minZ is the min Z boundary
+   * @param maxZ is the max Z boundary
+   */
+  public static GeoArea makeGeoArea(final PlanetModel planetModel, final double minX, final double maxX, final double minY, final double maxY, final double minZ, final double maxZ) {
+    if (Math.abs(maxX - minX) < Vector.MINIMUM_RESOLUTION) {
+      if (Math.abs(maxY - minY) < Vector.MINIMUM_RESOLUTION) {
+        if (Math.abs(maxZ - minZ) < Vector.MINIMUM_RESOLUTION) {
+          return new dXdYdZSolid(planetModel, (minX+maxX) * 0.5, (minY+maxY) * 0.5, minZ);
+        } else {
+          return new dXdYZSolid(planetModel, (minX+maxX) * 0.5, (minY+maxY) * 0.5, minZ, maxZ);
+        }
+      } else {
+        if (Math.abs(maxZ - minZ) < Vector.MINIMUM_RESOLUTION) {
+          return new dXYdZSolid(planetModel, (minX+maxX) * 0.5, minY, maxY, (minZ+maxZ) * 0.5);
+        } else {
+          return new dXYZSolid(planetModel, (minX+maxX) * 0.5, minY, maxY, minZ, maxZ);
+        }
+      }
+    }
+    if (Math.abs(maxY - minY) < Vector.MINIMUM_RESOLUTION) {
+      if (Math.abs(maxZ - minZ) < Vector.MINIMUM_RESOLUTION) {
+        return new XdYdZSolid(planetModel, minX, maxX, (minY+maxY) * 0.5, (minZ+maxZ) * 0.5);
+      } else {
+        return new XdYZSolid(planetModel, minX, maxX, (minY+maxY) * 0.5, minZ, maxZ);
+      }
+    }
+    if (Math.abs(maxZ - minZ) < Vector.MINIMUM_RESOLUTION) {
+      return new XYdZSolid(planetModel, minX, maxX, minY, maxY, (minZ+maxZ) * 0.5);
+    }
+    return new XYZSolid(planetModel, minX, maxX, minY, maxY, minZ, maxZ);
+  }
+  
 }

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoBaseShape.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoBaseShape.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoBaseShape.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoBaseShape.java Wed Sep  2 19:59:31 2015
@@ -32,16 +32,27 @@ public abstract class GeoBaseShape exten
   }
 
   @Override
-  public Bounds getBounds(Bounds bounds) {
-    if (bounds == null)
-      bounds = new Bounds();
+  public void getBounds(Bounds bounds) {
     if (isWithin(planetModel.NORTH_POLE)) {
-      bounds.noTopLatitudeBound().noLongitudeBound();
+      bounds.noTopLatitudeBound().noLongitudeBound()
+        .addPoint(planetModel.NORTH_POLE);
     }
     if (isWithin(planetModel.SOUTH_POLE)) {
-      bounds.noBottomLatitudeBound().noLongitudeBound();
+      bounds.noBottomLatitudeBound().noLongitudeBound()
+        .addPoint(planetModel.SOUTH_POLE);
+    }
+    if (isWithin(planetModel.MIN_X_POLE)) {
+      bounds.addPoint(planetModel.MIN_X_POLE);
+    }
+    if (isWithin(planetModel.MAX_X_POLE)) {
+      bounds.addPoint(planetModel.MAX_X_POLE);
+    }
+    if (isWithin(planetModel.MIN_Y_POLE)) {
+      bounds.addPoint(planetModel.MIN_Y_POLE);
+    }
+    if (isWithin(planetModel.MAX_Y_POLE)) {
+      bounds.addPoint(planetModel.MAX_Y_POLE);
     }
-    return bounds;
   }
 
 }

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCircle.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCircle.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCircle.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCircle.java Wed Sep  2 19:59:31 2015
@@ -46,8 +46,10 @@ public class GeoCircle extends GeoBaseDi
       throw new IllegalArgumentException("Latitude out of bounds");
     if (lon < -Math.PI || lon > Math.PI)
       throw new IllegalArgumentException("Longitude out of bounds");
-    if (cutoffAngle <= 0.0 || cutoffAngle > Math.PI)
+    if (cutoffAngle < 0.0 || cutoffAngle > Math.PI)
       throw new IllegalArgumentException("Cutoff angle out of bounds");
+    if (cutoffAngle < Vector.MINIMUM_RESOLUTION)
+      throw new IllegalArgumentException("Cutoff angle cannot be effectively zero");
     this.center = new GeoPoint(planetModel, lat, lon);
     // In an ellipsoidal world, cutoff distances make no sense, unfortunately.  Only membership
     // can be used to make in/out determination.
@@ -78,12 +80,15 @@ public class GeoCircle extends GeoBaseDi
       this.edgePoints = new GeoPoint[0];
     } else {
       // Construct normal plane
-      final Plane normalPlane = Plane.constructNormalizedVerticalPlane(upperPoint, lowerPoint, center);
+      final Plane normalPlane = Plane.constructNormalizedZPlane(upperPoint, lowerPoint, center);
       // Construct a sided plane that goes through the two points and whose normal is in the normalPlane.
       this.circlePlane = SidedPlane.constructNormalizedPerpendicularSidedPlane(center, normalPlane, upperPoint, lowerPoint);
       if (circlePlane == null)
-        throw new RuntimeException("Couldn't construct circle plane.  Cutoff angle = "+cutoffAngle+"; upperPoint = "+upperPoint+"; lowerPoint = "+lowerPoint);
-      this.edgePoints = new GeoPoint[]{upperPoint};
+        throw new IllegalArgumentException("Couldn't construct circle plane, probably too small?  Cutoff angle = "+cutoffAngle+"; upperPoint = "+upperPoint+"; lowerPoint = "+lowerPoint);
+      final GeoPoint recomputedIntersectionPoint = circlePlane.getSampleIntersectionPoint(planetModel, normalPlane);
+      if (recomputedIntersectionPoint == null)
+        throw new IllegalArgumentException("Couldn't construct intersection point, probably circle too small?  Plane = "+circlePlane);
+      this.edgePoints = new GeoPoint[]{recomputedIntersectionPoint};
     }
   }
 
@@ -130,16 +135,14 @@ public class GeoCircle extends GeoBaseDi
   }
 
   @Override
-  public Bounds getBounds(Bounds bounds) {
-    bounds = super.getBounds(bounds);
+  public void getBounds(Bounds bounds) {
+    super.getBounds(bounds);
     if (circlePlane == null) {
-      // Entire world
-      bounds.noTopLatitudeBound().noBottomLatitudeBound().noLongitudeBound();
-      return bounds;
+      // Entire world; should already be covered
+      return;
     }
     bounds.addPoint(center);
-    circlePlane.recordBounds(planetModel, bounds);
-    return bounds;
+    bounds.addPlane(planetModel, circlePlane);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCompositeMembershipShape.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCompositeMembershipShape.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCompositeMembershipShape.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoCompositeMembershipShape.java Wed Sep  2 19:59:31 2015
@@ -71,13 +71,10 @@ public class GeoCompositeMembershipShape
   }
 
   @Override
-  public Bounds getBounds(Bounds bounds) {
-    if (bounds == null)
-      bounds = new Bounds();
+  public void getBounds(Bounds bounds) {
     for (GeoMembershipShape shape : shapes) {
-      bounds = shape.getBounds(bounds);
+      shape.getBounds(bounds);
     }
-    return bounds;
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoConvexPolygon.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoConvexPolygon.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoConvexPolygon.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoConvexPolygon.java Wed Sep  2 19:59:31 2015
@@ -213,8 +213,8 @@ public class GeoConvexPolygon extends Ge
   }
 
   @Override
-  public Bounds getBounds(Bounds bounds) {
-    bounds = super.getBounds(bounds);
+  public void getBounds(Bounds bounds) {
+    super.getBounds(bounds);
 
     // Add all the points
     for (final GeoPoint point : points) {
@@ -232,14 +232,8 @@ public class GeoConvexPolygon extends Ge
           membershipBounds[count++] = edges[otherIndex];
         }
       }
-      edge.recordBounds(planetModel, bounds, membershipBounds);
+      bounds.addPlane(planetModel, edge, membershipBounds);
     }
-
-    if (fullDistance >= Math.PI) {
-      // We can't reliably assume that bounds did its longitude calculation right, so we force it to be unbounded.
-      bounds.noLongitudeBound();
-    }
-    return bounds;
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoDegenerateHorizontalLine.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoDegenerateHorizontalLine.java?rev=1700883&r1=1700882&r2=1700883&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoDegenerateHorizontalLine.java (original)
+++ lucene/dev/trunk/lucene/spatial3d/src/java/org/apache/lucene/geo3d/GeoDegenerateHorizontalLine.java Wed Sep  2 19:59:31 2015
@@ -87,8 +87,8 @@ public class GeoDegenerateHorizontalLine
     final double cosRightLon = Math.cos(rightLon);
 
     // Now build the two points
-    this.LHC = new GeoPoint(planetModel, sinLatitude, sinLeftLon, cosLatitude, cosLeftLon);
-    this.RHC = new GeoPoint(planetModel, sinLatitude, sinRightLon, cosLatitude, cosRightLon);
+    this.LHC = new GeoPoint(planetModel, sinLatitude, sinLeftLon, cosLatitude, cosLeftLon, latitude, leftLon);
+    this.RHC = new GeoPoint(planetModel, sinLatitude, sinRightLon, cosLatitude, cosRightLon, latitude, rightLon);
 
     this.plane = new Plane(planetModel, sinLatitude);
 
@@ -156,11 +156,10 @@ public class GeoDegenerateHorizontalLine
   }
 
   @Override
-  public Bounds getBounds(Bounds bounds) {
-    if (bounds == null)
-      bounds = new Bounds();
-    bounds.addLatitudeZone(latitude).addLongitudeSlice(leftLon, rightLon);
-    return bounds;
+  public void getBounds(Bounds bounds) {
+    super.getBounds(bounds);
+    bounds.addHorizontalPlane(planetModel, latitude, plane, leftPlane, rightPlane)
+      .addPoint(LHC).addPoint(RHC);
   }
 
   @Override