You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/03/30 08:49:01 UTC

[GitHub] [lucene] jpountz commented on a change in pull request #7: LUCENE-9820: Separate logic for reading the BKD index from logic to intersecting it

jpountz commented on a change in pull request #7:
URL: https://github.com/apache/lucene/pull/7#discussion_r603887793



##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDIndexInput.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.lucene.index.PointValues.IntersectVisitor;
+
+/**
+ * Abstraction of a block KD-tree that contains multi-dimensional points in byte[] space.
+ *
+ * @lucene.internal
+ */
+public interface BKDIndexInput {
+
+  /** BKD tree parameters */
+  BKDConfig getConfig();
+
+  /** min packed value */
+  byte[] getMinPackedValue();
+
+  /** max packed value */
+  byte[] getMaxPackedValue();
+
+  /** Total number of points */
+  long getPointCount();

Review comment:
       Do we actually need the three above methods? This info is already available on the root of the IndexTree?

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDIndexInput.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.lucene.index.PointValues.IntersectVisitor;
+
+/**
+ * Abstraction of a block KD-tree that contains multi-dimensional points in byte[] space.
+ *
+ * @lucene.internal
+ */
+public interface BKDIndexInput {
+
+  /** BKD tree parameters */
+  BKDConfig getConfig();
+
+  /** min packed value */
+  byte[] getMinPackedValue();
+
+  /** max packed value */
+  byte[] getMaxPackedValue();
+
+  /** Total number of points */
+  long getPointCount();
+
+  /** Total number of documents */
+  int getDocCount();
+
+  /** Create a new {@link IndexTree} to navigate the index */
+  IndexTree getIndexTree();
+
+  /** Create a new {@link LeafIterator} to read all leaf nodes */
+  LeafIterator getLeafTreeIterator() throws IOException;

Review comment:
       Do we actually need it? Leaves can already be visited using the IndexTree by visiting the tree in depth-first order? I understand that this has slightly more overhead due to the need to browse the inner nodes in parallel to leaf nodes, but the bottleneck should really be leaf nodes, and removing this method would help us have a more minimal API?

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDIndexInput.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.lucene.index.PointValues.IntersectVisitor;
+
+/**
+ * Abstraction of a block KD-tree that contains multi-dimensional points in byte[] space.
+ *
+ * @lucene.internal
+ */
+public interface BKDIndexInput {

Review comment:
       The name is a bit confusing IMO given that it doesn't extend `IndexInput`. What about renaming it to `BKDReader`? (see my other comment on `BKDReader` too)

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDReader.java
##########
@@ -17,1048 +17,146 @@
 package org.apache.lucene.util.bkd;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.Arrays;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.PointValues;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.MathUtil;
 
 /**
- * Handles intersection of an multi-dimensional shape in byte[] space with a block KD-tree
- * previously written with {@link BKDWriter}.
+ * A {@link PointValues} wrapper for {@link BKDIndexInput} to handle intersections.
  *
  * @lucene.experimental
  */
 public final class BKDReader extends PointValues {

Review comment:
       Since the action to read data is now effectively the responsibility of `BKDIndexInput`, maybe this class should be renamed to `BKDPointValues`?

##########
File path: lucene/core/src/java/org/apache/lucene/util/bkd/BKDIndexInput.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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 org.apache.lucene.index.PointValues.IntersectVisitor;
+
+/**
+ * Abstraction of a block KD-tree that contains multi-dimensional points in byte[] space.
+ *
+ * @lucene.internal
+ */
+public interface BKDIndexInput {
+
+  /** BKD tree parameters */
+  BKDConfig getConfig();
+
+  /** min packed value */
+  byte[] getMinPackedValue();
+
+  /** max packed value */
+  byte[] getMaxPackedValue();
+
+  /** Total number of points */
+  long getPointCount();
+
+  /** Total number of documents */
+  int getDocCount();
+
+  /** Create a new {@link IndexTree} to navigate the index */
+  IndexTree getIndexTree();
+
+  /** Create a new {@link LeafIterator} to read all leaf nodes */
+  LeafIterator getLeafTreeIterator() throws IOException;
+
+  /** Basic operations to read the BKD tree. */
+  interface IndexTree extends Cloneable {
+
+    /** Clone, but you are not allowed to pop up past the point where the clone happened. */

Review comment:
       Can you be more explicit: is the behavior undefined if trying to pop too much, or should implementations throw an exception?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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