[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254361915
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.RadixSelector;
+
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  //size of the online buffer: 8 KB
+  private static final int MAX_SIZE_OFFLINE_BUFFER = 1024 * 8;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes per dimension
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortInHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortInHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortInHeap = 2 * maxPointsSortInHeap;
+int numberOfPointsOffline  = MAX_SIZE_OFFLINE_BUFFER / (packedBytesLength 
+ Integer.BYTES);
+this.offlineBuffer = new byte[numberOfPointsOffline * (packedBytesLength + 
Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   *
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens. The method destroys the original writer.
+   *
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
 
 Review comment:
   Right, let's play with this idea on a follow-up PR. +1 to keep from and to 
for now then.


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254270203
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.RadixSelector;
+
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  //size of the online buffer: 8 KB
+  private static final int MAX_SIZE_OFFLINE_BUFFER = 1024 * 8;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes per dimension
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortInHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortInHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortInHeap = 2 * maxPointsSortInHeap;
+int numberOfPointsOffline  = MAX_SIZE_OFFLINE_BUFFER / (packedBytesLength 
+ Integer.BYTES);
+this.offlineBuffer = new byte[numberOfPointsOffline * (packedBytesLength + 
Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   *
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens. The method destroys the original writer.
+   *
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
 
 Review comment:
   do we ever call this with from != 0 and to != points.count()?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254264333
 
 

 ##
 File path: 
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
 ##
 @@ -1239,90 +1017,68 @@ assert valuesInOrderAndBounds(count, sortedDim, 
minPackedValue, maxPackedValue,
 
   /** The array (sized numDims) of PathSlice describe the cell we have 
currently recursed to. */
   private void build(int nodeID, int leafNodeOffset,
- PathSlice[] slices,
- LongBitSet ordBitSet,
+ PointWriter data,
  IndexOutput out,
+ BKDRadixSelector radixSelector,
  byte[] minPackedValue, byte[] maxPackedValue,
  byte[] splitPackedValues,
- long[] leafBlockFPs,
- List toCloseHeroically) throws IOException {
-
-for(PathSlice slice : slices) {
-  assert slice.count == slices[0].count;
-}
-
-if (numDataDims == 1 && slices[0].writer instanceof OfflinePointWriter && 
slices[0].count <= maxPointsSortInHeap) {
-  // Special case for 1D, to cutover to heap once we recurse deeply enough:
-  slices[0] = switchToHeap(slices[0], toCloseHeroically);
-}
+ long[] leafBlockFPs) throws IOException {
 
 if (nodeID >= leafNodeOffset) {
 
   // Leaf node: write block
   // We can write the block in any order so by default we write it sorted 
by the dimension that has the
   // least number of unique bytes at commonPrefixLengths[dim], which makes 
compression more efficient
-  int sortedDim = 0;
-  int sortedDimCardinality = Integer.MAX_VALUE;
-
-  for (int dim=0;dim= maxPointsInLeafNode, so we 
better be in heap at this point:
+  HeapPointWriter heapSource = (HeapPointWriter) data;
 
-// Find common prefix by comparing first and last values, already 
sorted in this dimension:
-heapSource.readPackedValue(Math.toIntExact(source.start), scratch1);
-heapSource.readPackedValue(Math.toIntExact(source.start + source.count 
- 1), scratch2);
+  //we store common prefix on scratch1
+  computeCommonPrefixLength(heapSource, scratch1);
 
-int offset = dim * bytesPerDim;
-commonPrefixLengths[dim] = bytesPerDim;
-for(int j=0;j= maxPointsInLeafNode, so we 
better be in heap at this point:
-  HeapPointWriter heapSource = (HeapPointWriter) source.writer;
+  sortHeapPointWriter(heapSource, sortedDim);
 
   // Save the block file pointer:
   leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
   //System.out.println("  write leaf block @ fp=" + out.getFilePointer());
 
   // Write docIDs first, as their own chunk, so that at intersect time we 
can add all docIDs w/o
   // loading the values:
-  int count = Math.toIntExact(source.count);
+  int count = Math.toIntExact(heapSource.count());
   assert count > 0: "nodeID=" + nodeID + " leafNodeOffset=" + 
leafNodeOffset;
-  writeLeafBlockDocs(out, heapSource.docIDs, 
Math.toIntExact(source.start), count);
+  writeLeafBlockDocs(out, heapSource.docIDs, Math.toIntExact(0), count);
 
 Review comment:
   s/Math.toIntExact(0)/0/


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254275260
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,311 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.RadixSelector;
+
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  //size of the online buffer: 8 KB
+  private static final int MAX_SIZE_OFFLINE_BUFFER = 1024 * 8;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes per dimension
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortInHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortInHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortInHeap = 2 * maxPointsSortInHeap;
+int numberOfPointsOffline  = MAX_SIZE_OFFLINE_BUFFER / (packedBytesLength 
+ Integer.BYTES);
+this.offlineBuffer = new byte[numberOfPointsOffline * (packedBytesLength + 
Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   *
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens. The method destroys the original writer.
+   *
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
+checkArgs(from, to, partitionPoint);
+
+//If we are on heap then we just select on heap
+if (points instanceof HeapPointWriter) {
+  return heapSelect((HeapPointWriter) points, left, right, dim, 
Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), 
0);
+}
+
+//reset histogram
+for (int i = 0; i < bytesSorted; i++) {
+  Arrays.fill(histogram[i], 0);
+}
+OfflinePointWriter offlinePointWriter = (OfflinePointWriter) points;
+
+//find common prefix, it does already set histogram values if needed
+int commonPrefix = findCommonPrefix(offlinePointWriter, from, to, dim);
+
+//if all equals we just partition the data
+if (commonPrefix ==  bytesSorted) {
+  partition(offlinePointWriter, left,  right, null, from, to, dim, 
commonPrefix - 1, partitionPoint);
+  return partitionPointFromCommonPrefix();
+}
+//let's rock'n'roll
+return buildHistogramAndPartition(offlinePointWriter, left, right, from, 
to, partitionPoint, 0, commonPrefix, dim);
+  }
+
+  void checkArgs(long from, long to, long partitionPoint) {
+if (partitionPoint < from) {
+  throw new IllegalArgumentException("partitionPoint must be >= from");
+}
+if (partitionPoint >= 

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254263885
 
 

 ##
 File path: 
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
 ##
 @@ -1239,90 +1017,68 @@ assert valuesInOrderAndBounds(count, sortedDim, 
minPackedValue, maxPackedValue,
 
   /** The array (sized numDims) of PathSlice describe the cell we have 
currently recursed to. */
   private void build(int nodeID, int leafNodeOffset,
- PathSlice[] slices,
- LongBitSet ordBitSet,
+ PointWriter data,
  IndexOutput out,
+ BKDRadixSelector radixSelector,
  byte[] minPackedValue, byte[] maxPackedValue,
  byte[] splitPackedValues,
- long[] leafBlockFPs,
- List toCloseHeroically) throws IOException {
-
-for(PathSlice slice : slices) {
-  assert slice.count == slices[0].count;
-}
-
-if (numDataDims == 1 && slices[0].writer instanceof OfflinePointWriter && 
slices[0].count <= maxPointsSortInHeap) {
-  // Special case for 1D, to cutover to heap once we recurse deeply enough:
-  slices[0] = switchToHeap(slices[0], toCloseHeroically);
-}
+ long[] leafBlockFPs) throws IOException {
 
 if (nodeID >= leafNodeOffset) {
 
   // Leaf node: write block
   // We can write the block in any order so by default we write it sorted 
by the dimension that has the
   // least number of unique bytes at commonPrefixLengths[dim], which makes 
compression more efficient
-  int sortedDim = 0;
-  int sortedDimCardinality = Integer.MAX_VALUE;
-
-  for (int dim=0;dim= maxPointsInLeafNode, so we 
better be in heap at this point:
+  HeapPointWriter heapSource = (HeapPointWriter) data;
 
-// Find common prefix by comparing first and last values, already 
sorted in this dimension:
-heapSource.readPackedValue(Math.toIntExact(source.start), scratch1);
-heapSource.readPackedValue(Math.toIntExact(source.start + source.count 
- 1), scratch2);
+  //we store common prefix on scratch1
+  computeCommonPrefixLength(heapSource, scratch1);
 
-int offset = dim * bytesPerDim;
-commonPrefixLengths[dim] = bytesPerDim;
-for(int j=0;j

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254265065
 
 

 ##
 File path: 
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
 ##
 @@ -1337,12 +1093,12 @@ private void build(int nodeID, int leafNodeOffset,
 
 @Override
 public BytesRef apply(int i) {
-  heapSource.getPackedValueSlice(Math.toIntExact(source.start + i), 
scratch);
+  heapSource.getPackedValueSlice(Math.toIntExact(i), scratch);
   return scratch;
 }
   };
   assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, 
maxPackedValue, packedValues,
-  heapSource.docIDs, Math.toIntExact(source.start));
+  heapSource.docIDs, Math.toIntExact(0));
 
 Review comment:
   no need for toIntExact


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254265749
 
 

 ##
 File path: 
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
 ##
 @@ -1355,91 +,67 @@ assert valuesInOrderAndBounds(count, sortedDim, 
minPackedValue, maxPackedValue,
 splitDim = 0;
   }
 
-  PathSlice source = slices[splitDim];
 
-  assert nodeID < splitPackedValues.length: "nodeID=" + nodeID + " 
splitValues.length=" + splitPackedValues.length;
+  assert nodeID < splitPackedValues.length : "nodeID=" + nodeID + " 
splitValues.length=" + splitPackedValues.length;
 
   // How many points will be in the left tree:
-  long rightCount = source.count / 2;
-  long leftCount = source.count - rightCount;
+  long rightCount = data.count() / 2;
+  long leftCount = data.count() - rightCount;
+
+  PointWriter leftPointWriter;
+  PointWriter rightPointWriter;
+  byte[] splitValue;
+
+  try (PointWriter leftPointWriter2 = getPointWriter(leftCount, "left" + 
splitDim);
+   PointWriter rightPointWriter2 = getPointWriter(rightCount, "right" 
+ splitDim)) {
+splitValue = radixSelector.select(data, leftPointWriter2, 
rightPointWriter2, 0, Math.toIntExact(data.count()),  
Math.toIntExact(leftCount), splitDim);
 
 Review comment:
   I think we should not call toIntExact here, the count can be > 
Integer.MAX_VALUE sometimes, and it's fine?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254284219
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
 ##
 @@ -194,40 +169,28 @@ protected BKDWriter(int maxDoc, Directory tempDir, 
String tempFileNamePrefix, in
 minPackedValue = new byte[packedIndexBytesLength];
 maxPackedValue = new byte[packedIndexBytesLength];
 
-// If we may have more than 1+Integer.MAX_VALUE values, then we must 
encode ords with long (8 bytes), else we can use int (4 bytes).
-this.longOrds = longOrds;
-
-this.singleValuePerDoc = singleValuePerDoc;
+// dimensional values (numDims * bytesPerDim) + docID (int)
+bytesPerDoc = packedBytesLength + Integer.BYTES;
 
-// dimensional values (numDims * bytesPerDim) + ord (int or long) + docID 
(int)
-if (singleValuePerDoc) {
-  // Lucene only supports up to 2.1 docs, so we better not need longOrds 
in this case:
-  assert longOrds == false;
-  bytesPerDoc = packedBytesLength + Integer.BYTES;
-} else if (longOrds) {
-  bytesPerDoc = packedBytesLength + Long.BYTES + Integer.BYTES;
-} else {
-  bytesPerDoc = packedBytesLength + Integer.BYTES + Integer.BYTES;
-}
 
 // As we recurse, we compute temporary partitions of the data, halving the
 // number of points at each recursion.  Once there are few enough points,
 // we can switch to sorting in heap instead of offline (on disk).  At any
 // time in the recursion, we hold the number of points at that level, plus
 // all recursive halves (i.e. 16 + 8 + 4 + 2) so the memory usage is 2X
 // what that level would consume, so we multiply by 0.5 to convert from
-// bytes to points here.  Each dimension has its own sorted partition, so
-// we must divide by numDims as wel.
+// bytes to points here.  In addition the radix partitioning may sort on 
memory
+// double of this size so we multiply by another 0.5.
 
-maxPointsSortInHeap = (int) (0.5 * (maxMBSortInHeap * 1024 * 1024) / 
(bytesPerDoc * numDataDims));
+maxPointsSortInHeap = (int) (0.25 * (maxMBSortInHeap * 1024 * 1024) / 
(bytesPerDoc));
 
 Review comment:
   I understand the first 0.5 factor due to how recursion levels add up, but 
not the second one.


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-06 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r254264432
 
 

 ##
 File path: 
lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
 ##
 @@ -1337,12 +1093,12 @@ private void build(int nodeID, int leafNodeOffset,
 
 @Override
 public BytesRef apply(int i) {
-  heapSource.getPackedValueSlice(Math.toIntExact(source.start + i), 
scratch);
+  heapSource.getPackedValueSlice(Math.toIntExact(i), scratch);
 
 Review comment:
   no need for toIntExact?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253378113
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  //holder for partition bytes
+  private final byte[] partitionBytes;
+  //re-usable on-heap selector
+  private final HeapSelector heapSelector;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortedOffHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortedOffHeap = maxPointsSortedOffHeap;
+this.offlineBuffer = new byte[maxPointsSortedOffHeap * (packedBytesLength 
+ Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.partitionBytes =  new byte[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.heapSelector = new HeapSelector(numDim, bytesPerDim);
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens.
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
+checkArgs(from, to, partitionPoint);
+
+//If we are on heap then we just select on heap
+if (points instanceof HeapPointWriter) {
+  return heapSelect((HeapPointWriter) points, left, right, dim, 
Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), 
0);
+}
+
+//reset histogram
+for (int i = 0; i < bytesSorted; i++) {
+  Arrays.fill(histogram[i], 0);
+}
+OfflinePointWriter offlinePointWriter = (OfflinePointWriter) points;
+
+//find common prefix, it does already set histogram values if needed
+int commonPrefix = findCommonPrefix(offlinePointWriter, from, to, dim);
+
+//if all equals we just partition the data
+if (commonPrefix ==  bytesSorted) {
+  return partition(offlinePointWriter, left, right, from, to, 
partitionPoint, dim, null, commonPrefix - 1, partitionPoint);
+}
+//let's rock'n'roll
+return buildHistogramAndPartition(offlinePointWriter, null, left, right, 
from, to, partitionPoint, 0, commonPrefix, dim,0, 0);
+  }
+
+  void checkArgs(long from, long to, long middle) {
+if (middle < from) {
+  throw new IllegalArgumentException("middle must be >= from");
+}
+if (middle >= to) {
+  throw new I

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253378270
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  //holder for partition bytes
+  private final byte[] partitionBytes;
+  //re-usable on-heap selector
+  private final HeapSelector heapSelector;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortedOffHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortedOffHeap = maxPointsSortedOffHeap;
+this.offlineBuffer = new byte[maxPointsSortedOffHeap * (packedBytesLength 
+ Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.partitionBytes =  new byte[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.heapSelector = new HeapSelector(numDim, bytesPerDim);
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens.
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
+checkArgs(from, to, partitionPoint);
+
+//If we are on heap then we just select on heap
+if (points instanceof HeapPointWriter) {
+  return heapSelect((HeapPointWriter) points, left, right, dim, 
Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), 
0);
+}
+
+//reset histogram
+for (int i = 0; i < bytesSorted; i++) {
+  Arrays.fill(histogram[i], 0);
+}
+OfflinePointWriter offlinePointWriter = (OfflinePointWriter) points;
+
+//find common prefix, it does already set histogram values if needed
+int commonPrefix = findCommonPrefix(offlinePointWriter, from, to, dim);
+
+//if all equals we just partition the data
+if (commonPrefix ==  bytesSorted) {
+  return partition(offlinePointWriter, left, right, from, to, 
partitionPoint, dim, null, commonPrefix - 1, partitionPoint);
+}
+//let's rock'n'roll
+return buildHistogramAndPartition(offlinePointWriter, null, left, right, 
from, to, partitionPoint, 0, commonPrefix, dim,0, 0);
+  }
+
+  void checkArgs(long from, long to, long middle) {
+if (middle < from) {
+  throw new IllegalArgumentException("middle must be >= from");
+}
+if (middle >= to) {
+  throw new I

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253373809
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  //holder for partition bytes
+  private final byte[] partitionBytes;
+  //re-usable on-heap selector
+  private final HeapSelector heapSelector;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortedOffHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortedOffHeap = maxPointsSortedOffHeap;
+this.offlineBuffer = new byte[maxPointsSortedOffHeap * (packedBytesLength 
+ Integer.BYTES)];
 
 Review comment:
   My gut feeling is that we don't need to spend so much memory on this buffer 
for good performance and could instead make it around 8KB all the time 
(non-configurable) so that on-heap selection can use about 2x more memory.


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253381960
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
 
 Review comment:
   s/Off/On/ ?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253380758
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java
 ##
 @@ -19,24 +19,30 @@
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.List;
+
+import org.apache.lucene.util.BytesRef;
 
 /** Appends many points, and then at the end provides a {@link PointReader} to 
iterate
  *  those points.  This abstracts away whether we write to disk, or use simple 
arrays
  *  in heap.
  *
- *  @lucene.internal */
-public interface PointWriter extends Closeable {
-  /** Add a new point */
-  void append(byte[] packedValue, long ord, int docID) throws IOException;
+ *  @lucene.internal
+ *  */
+public interface PointWriter extends Closeable {
+  /** Add a new point from byte array*/
+  void append(byte[] packedValue, int docID) throws IOException;
 
 Review comment:
   do we still need this one, ie. could callers always call the method that 
takes a bytesref?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253376398
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  //holder for partition bytes
+  private final byte[] partitionBytes;
+  //re-usable on-heap selector
+  private final HeapSelector heapSelector;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortedOffHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortedOffHeap = maxPointsSortedOffHeap;
+this.offlineBuffer = new byte[maxPointsSortedOffHeap * (packedBytesLength 
+ Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.partitionBytes =  new byte[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.heapSelector = new HeapSelector(numDim, bytesPerDim);
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens.
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
+checkArgs(from, to, partitionPoint);
+
+//If we are on heap then we just select on heap
+if (points instanceof HeapPointWriter) {
+  return heapSelect((HeapPointWriter) points, left, right, dim, 
Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), 
0);
+}
+
+//reset histogram
+for (int i = 0; i < bytesSorted; i++) {
+  Arrays.fill(histogram[i], 0);
+}
+OfflinePointWriter offlinePointWriter = (OfflinePointWriter) points;
+
+//find common prefix, it does already set histogram values if needed
+int commonPrefix = findCommonPrefix(offlinePointWriter, from, to, dim);
+
+//if all equals we just partition the data
+if (commonPrefix ==  bytesSorted) {
+  return partition(offlinePointWriter, left, right, from, to, 
partitionPoint, dim, null, commonPrefix - 1, partitionPoint);
+}
+//let's rock'n'roll
+return buildHistogramAndPartition(offlinePointWriter, null, left, right, 
from, to, partitionPoint, 0, commonPrefix, dim,0, 0);
+  }
+
+  void checkArgs(long from, long to, long middle) {
+if (middle < from) {
+  throw new IllegalArgumentException("middle must be >= from");
+}
+if (middle >= to) {
+  throw new I

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253380207
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/util/bkd/PointWriter.java
 ##
 @@ -19,24 +19,30 @@
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.List;
+
+import org.apache.lucene.util.BytesRef;
 
 /** Appends many points, and then at the end provides a {@link PointReader} to 
iterate
  *  those points.  This abstracts away whether we write to disk, or use simple 
arrays
  *  in heap.
  *
- *  @lucene.internal */
-public interface PointWriter extends Closeable {
-  /** Add a new point */
-  void append(byte[] packedValue, long ord, int docID) throws IOException;
+ *  @lucene.internal
+ *  */
+public interface PointWriter extends Closeable {
 
 Review comment:
   I don't think we need generics here, making the getReader signature return 
an OfflinePointReader in OfflinePointWriter should be enough?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253373968
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
 ##
 @@ -74,55 +69,68 @@ public OfflinePointReader(Directory tempDir, String 
tempFileName, int packedByte
   // at another level of the BKDWriter recursion
   in = tempDir.openInput(tempFileName, IOContext.READONCE);
 }
+
 name = tempFileName;
 
 long seekFP = start * bytesPerDoc;
 in.seek(seekFP);
 countLeft = length;
-packedValue = new byte[packedBytesLength];
-this.longOrds = longOrds;
+if (reusableBuffer != null) {
 
 Review comment:
   then if we do that maybe we can remove the maxPointInHeap ctor argument and 
compute it from this buffer size?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253376318
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
+  private final int bytesPerDim;
+  // number of bytes to be sorted: bytesPerDim + Integer.BYTES
+  private final int bytesSorted;
+  //data dimensions size
+  private final int packedBytesLength;
+  //flag to when we are moving to sort on heap
+  private final int maxPointsSortedOffHeap;
+  //reusable buffer
+  private final byte[] offlineBuffer;
+  //holder for partition points
+  private final int[] partitionBucket;
+  //holder for partition bytes
+  private final byte[] partitionBytes;
+  //re-usable on-heap selector
+  private final HeapSelector heapSelector;
+  // scratch object to move bytes around
+  private final BytesRef bytesRef1 = new BytesRef();
+  // scratch object to move bytes around
+  private final BytesRef bytesRef2 = new BytesRef();
+  //Directory to create new Offline writer
+  private final Directory tempDir;
+  // prefix for temp files
+  private final String tempFileNamePrefix;
+
+
+
+  /**
+   * Sole constructor.
+   */
+  public BKDRadixSelector(int numDim, int bytesPerDim, int 
maxPointsSortedOffHeap, Directory tempDir, String tempFileNamePrefix) {
+this.bytesPerDim = bytesPerDim;
+this.packedBytesLength = numDim * bytesPerDim;
+this.bytesSorted = bytesPerDim + Integer.BYTES;
+this.maxPointsSortedOffHeap = maxPointsSortedOffHeap;
+this.offlineBuffer = new byte[maxPointsSortedOffHeap * (packedBytesLength 
+ Integer.BYTES)];
+this.partitionBucket = new int[bytesSorted];
+this.partitionBytes =  new byte[bytesSorted];
+this.histogram = new long[bytesSorted][HISTOGRAM_SIZE];
+this.bytesRef1.length = numDim * bytesPerDim;
+this.heapSelector = new HeapSelector(numDim, bytesPerDim);
+this.tempDir = tempDir;
+this.tempFileNamePrefix = tempFileNamePrefix;
+  }
+
+  /**
+   * Method to partition the input data. It returns the value of the dimension 
where
+   * the split happens.
+   */
+  public byte[] select(PointWriter points, PointWriter left, PointWriter 
right, long from, long to, long partitionPoint, int dim) throws IOException {
+checkArgs(from, to, partitionPoint);
+
+//If we are on heap then we just select on heap
+if (points instanceof HeapPointWriter) {
+  return heapSelect((HeapPointWriter) points, left, right, dim, 
Math.toIntExact(from), Math.toIntExact(to),  Math.toIntExact(partitionPoint), 
0);
+}
+
+//reset histogram
+for (int i = 0; i < bytesSorted; i++) {
+  Arrays.fill(histogram[i], 0);
+}
+OfflinePointWriter offlinePointWriter = (OfflinePointWriter) points;
+
+//find common prefix, it does already set histogram values if needed
+int commonPrefix = findCommonPrefix(offlinePointWriter, from, to, dim);
+
+//if all equals we just partition the data
+if (commonPrefix ==  bytesSorted) {
+  return partition(offlinePointWriter, left, right, from, to, 
partitionPoint, dim, null, commonPrefix - 1, partitionPoint);
+}
+//let's rock'n'roll
+return buildHistogramAndPartition(offlinePointWriter, null, left, right, 
from, to, partitionPoint, 0, commonPrefix, dim,0, 0);
+  }
+
+  void checkArgs(long from, long to, long middle) {
+if (middle < from) {
+  throw new IllegalArgumentException("middle must be >= from");
+}
+if (middle >= to) {
+  throw new I

[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253373228
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
 ##
 @@ -74,55 +69,68 @@ public OfflinePointReader(Directory tempDir, String 
tempFileName, int packedByte
   // at another level of the BKDWriter recursion
   in = tempDir.openInput(tempFileName, IOContext.READONCE);
 }
+
 name = tempFileName;
 
 long seekFP = start * bytesPerDoc;
 in.seek(seekFP);
 countLeft = length;
-packedValue = new byte[packedBytesLength];
-this.longOrds = longOrds;
+if (reusableBuffer != null) {
+  assert reusableBuffer.length >= this.maxPointOnHeap * bytesPerDoc;
+  this.onHeapBuffer = reusableBuffer;
+} else {
+  this.onHeapBuffer = new byte[this.maxPointOnHeap * bytesPerDoc];
+}
   }
 
   @Override
   public boolean next() throws IOException {
-if (countLeft >= 0) {
-  if (countLeft == 0) {
-return false;
+if (this.pointsInBuffer == 0) {
+  if (countLeft >= 0) {
+if (countLeft == 0) {
+  return false;
+}
   }
-  countLeft--;
-}
-try {
-  in.readBytes(packedValue, 0, packedValue.length);
-} catch (EOFException eofe) {
-  assert countLeft == -1;
-  return false;
-}
-docID = in.readInt();
-if (singleValuePerDoc == false) {
-  if (longOrds) {
-ord = in.readLong();
-  } else {
-ord = in.readInt();
+  try {
+if (countLeft > maxPointOnHeap) {
+  in.readBytes(onHeapBuffer, 0, maxPointOnHeap * bytesPerDoc);
+  pointsInBuffer = maxPointOnHeap - 1;
+  countLeft -= maxPointOnHeap;
+} else {
+  in.readBytes(onHeapBuffer, 0, (int) countLeft * bytesPerDoc);
+  pointsInBuffer = Math.toIntExact(countLeft - 1);
+  countLeft = 0;
+}
+this.offset = 0;
+  } catch (EOFException eofe) {
+assert countLeft == -1;
+return false;
   }
 } else {
-  ord = docID;
+  this.pointsInBuffer--;
+  this.offset += bytesPerDoc;
 }
 return true;
   }
 
   @Override
-  public byte[] packedValue() {
-return packedValue;
+  public void packedValue(BytesRef bytesRef) {
+bytesRef.bytes = onHeapBuffer;
+bytesRef.offset = offset;
+bytesRef.length = packedValueLength;
   }
 
-  @Override
-  public long ord() {
-return ord;
+  protected void docValue(BytesRef bytesRef) {
 
 Review comment:
   Based on the naming I thought it would only be the 4 bytes that represent 
the docID, maybe give it a more explicit name?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253372453
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointReader.java
 ##
 @@ -74,55 +69,68 @@ public OfflinePointReader(Directory tempDir, String 
tempFileName, int packedByte
   // at another level of the BKDWriter recursion
   in = tempDir.openInput(tempFileName, IOContext.READONCE);
 }
+
 name = tempFileName;
 
 long seekFP = start * bytesPerDoc;
 in.seek(seekFP);
 countLeft = length;
-packedValue = new byte[packedBytesLength];
-this.longOrds = longOrds;
+if (reusableBuffer != null) {
 
 Review comment:
   could we require a non-null buffer instead?


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


With regards,
Apache Git Services

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



[GitHub] jpountz commented on a change in pull request #556: LUCENE-8673: Use radix partitioning when merging dimensional points

2019-02-04 Thread GitBox
jpountz commented on a change in pull request #556: LUCENE-8673: Use radix 
partitioning when merging dimensional points
URL: https://github.com/apache/lucene-solr/pull/556#discussion_r253369643
 
 

 ##
 File path: 
lucene/core/src/java/org/apache/lucene/util/bkd/BKDRadixSelector.java
 ##
 @@ -0,0 +1,433 @@
+/*
+ * 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.util.bkd;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FutureArrays;
+import org.apache.lucene.util.IntroSelector;
+
+/**
+ *
+ * Offline Radix selector for BKD tree.
+ *
+ *  @lucene.internal
+ * */
+public final class BKDRadixSelector {
+  //size of the histogram
+  private static final int HISTOGRAM_SIZE = 256;
+  // we store one histogram per recursion level
+  private final long[][] histogram;
+  //bytes we are sorting
 
 Review comment:
   this description better applies to bytesSorted than to bytesPerDim?


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


With regards,
Apache Git Services

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