You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2022/07/13 15:25:14 UTC

[GitHub] [solr] danrosher opened a new pull request, #940: SOLR-16292 : NVector for alternative great-circle distance calculations

danrosher opened a new pull request, #940:
URL: https://github.com/apache/solr/pull/940

   https://issues.apache.org/jira/browse/SOLR-16292
   
   # Description
   
   [N-Vector](https://en.wikipedia.org/wiki/N-vector#Example:_Great_circle_distance) is a three-parameter representation that can be used to calculate the great-circle distance (assuming a spherical Earth).
   
   For small distances it compares well with Math.acos, and is a faster way of calculating the great-circle distance than Haversine.
   
   # Solution
   
   Store N-Vectors in solr index via CoordinateFieldType with 3 values for the nvector into single value double subfields, use java Math class for indexing these
   Use an maclaurin approximation for acos for calculating great-circle distance at query-time via a function query
   
   # Tests
   
   Tests for the FastInvTrigTest impl to compare it's acos with Math.acos.
   Tests for NVectorUtil, NVectorDist , latLongToNVector and NVectorToLatLong
   Tests for indexing N-Vectors and calculating the great-circle distance via the function query.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [ ] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   - [ ] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1191737824

   I'm a little confused as to what the sloppyAsin results are...
   
   From your tests, we should still prefer NVector with MacLaurian expansion at 17 terms over using Sloppy Haversine, right? Is Sloppy Asin using NVector with no series expansion for the trig and instead the lookup tables from Sloppy Math? Which gives us the 10cm accuracy at _almost_ the original 10 term expansion performance level?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922326429


##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.solr.util;
+
+public class NVectorUtil {
+
+    public static final double EARTH_RADIUS = 6173.008;//km google:standard mean earth radius;

Review Comment:
   Oops! Yes using org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM now



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r925650225


##########
solr/core/src/java/org/apache/solr/schema/NVector.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class NVector extends CoordinateFieldType {
+
+    @Override
+    protected void init(IndexSchema schema, Map<String, String> args) {
+        super.init(schema, args);
+        dimension = 3;
+        createSuffixCache(3);
+    }
+
+    @Override
+    public List<IndexableField> createFields(SchemaField field, Object value) {
+        String externalVal = value.toString();
+        String[] point = parseCommaSeparatedList(externalVal, dimension);
+        String[] nvector = NVectorUtil.latLongToNVector(point);
+
+        List<IndexableField> f = new ArrayList<>((dimension * 2) + 1);
+
+        if (field.indexed()) {
+            for (int i = 0; i < dimension; i++) {
+                SchemaField sf = subField(field, i, schema);
+                f.addAll(sf.createFields(nvector[i]));
+            }
+        }
+
+        if (field.stored()) {
+            f.add(createField(field.getName(), externalVal, StoredField.TYPE));
+        }
+        return f;
+    }
+
+    @Override
+    public ValueSource getValueSource(SchemaField field, QParser parser) {
+        ArrayList<ValueSource> vs = new ArrayList<>(dimension);
+        for (int i = 0; i < dimension; i++) {
+            SchemaField sub = subField(field, i, schema);
+            vs.add(sub.getType()
+                .getValueSource(sub, parser));
+        }
+        return new NVectorValueSource(vs);
+    }
+
+
+    /**
+     * Given a string containing <i>dimension</i> values encoded in it, separated by commas,
+     * return a String array of length <i>dimension</i> containing the values.
+     *
+     * @param externalVal The value to parse
+     * @param dimension   The expected number of values for the point
+     * @return An array of the values that make up the point (aka vector)
+     * @throws SolrException if the dimension specified does not match the number found
+     */
+    public static String[] parseCommaSeparatedList(String externalVal, int dimension) throws SolrException {
+        //TODO: Should we support sparse vectors?
+        String[] out = new String[dimension];
+        int idx = externalVal.indexOf(',');
+        int end = idx;
+        int start = 0;
+        int i = 0;
+        if (idx == -1 && dimension == 1 && externalVal.length() > 0) {//we have a single point, dimension better be 1
+            out[0] = externalVal.trim();
+            i = 1;
+        } else if (idx > 0) {//if it is zero, that is an error
+            //Parse out a comma separated list of values, as in: 73.5,89.2,7773.4
+            for (; i < dimension; i++) {
+                while (start < end && externalVal.charAt(start) == ' ') start++;
+                while (end > start && externalVal.charAt(end - 1) == ' ') end--;
+                if (start == end) {
+                    break;
+                }
+                out[i] = externalVal.substring(start, end);
+                start = idx + 1;
+                end = externalVal.indexOf(',', start);

Review Comment:
   I've added a separator init param and parse doubles using NumberFormat with default locale



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1192648270

   Ok, I get it now. Yes, let's do the SloppyMath.asin approach, splitting out the sort key (I would call it dot product and add comments about why we can use it as sort key instead of calling the method `SortKey`)


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


Re: [PR] SOLR-16292 : NVector for alternative great-circle distance calculations [solr]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1951491368

   This PR had no visible activity in the past 60 days, labeling it as stale. Any new activity will remove the stale label. To attract more reviewers, please tag someone or notify the dev@solr.apache.org mailing list. Thank you for your contribution!


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922322314


##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {
+
+    private final MultiValueSource p1;
+    private final MultiValueSource p2;
+    private final double radius;
+
+    public NVector(MultiValueSource p1, MultiValueSource p2, double radius) {
+        this.p1 = p1;
+        this.p2 = p2;
+        if (p1.dimension() != 3 || p2.dimension() != 3) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Illegal dimension for value sources");
+        }
+        this.radius = radius;
+    }
+
+    @Override
+    public FunctionValues getValues(Map<Object,Object> context, LeafReaderContext readerContext) throws IOException {
+
+        final FunctionValues vals1 = p1.getValues(context, readerContext);
+        final FunctionValues vals2 = p2.getValues(context, readerContext);
+
+        return new DoubleDocValues(this) {
+
+            @Override
+            public double doubleVal(int doc) throws IOException {
+                double[] dv1 = new double[p1.dimension()];
+                double[] dv2 = new double[p2.dimension()];
+                vals1.doubleVal(doc, dv1);
+                vals2.doubleVal(doc, dv2);
+                return  NVectorDist(dv1, dv2, radius);
+            }
+
+            @Override
+            public String toString(int doc) throws IOException {
+                return name() +
+                    ',' +
+                    vals1.toString(doc) +
+                    ',' +
+                    vals2.toString(doc) +
+                    ')';
+            }
+        };
+    }
+
+    protected String name() {
+        return "nvector";
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this.getClass() != o.getClass()) return false;

Review Comment:
   caters for null now



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r940115461


##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {
+
+    String DEFAULT_SEPARATOR = ",";

Review Comment:
   updated



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922325966


##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {
+
+    private final MultiValueSource p1;
+    private final MultiValueSource p2;

Review Comment:
   They are nvectors, I've changed to something more meaningful, I was told about them recently from a prior employer https://en.wikipedia.org/wiki/N-vector, and thought if we can pre-calculate and store the vectors, and have a fast impl for acos then this can lead to a fast distance calculation.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922364025


##########
solr/core/src/test-files/solr/collection1/conf/schema-nvector.xml:
##########
@@ -0,0 +1,930 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- The Solr schema file. This file should be named "schema.xml" and
+     should be located where the classloader for the Solr webapp can find it.
+
+     This schema is used for testing, and as such has everything and the
+     kitchen sink thrown in. See example/solr/conf/schema.xml for a
+     more concise example.
+
+  -->
+
+<schema name="test" version="1.0">

Review Comment:
   reduced schema size



##########
solr/core/src/test/org/apache/solr/util/FastInvTrigTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.solr.util;
+
+import org.apache.solr.SolrTestCase;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static org.apache.solr.util.NVectorUtil.EARTH_RADIUS;
+
+public class FastInvTrigTest extends SolrTestCase {
+
+    final static int num_points = 100000;
+    final static double EPSILON = 0.0001;
+    static final Random r = new Random();

Review Comment:
   using random() now



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r927733523


##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {

Review Comment:
   should this extend from `PointType` instead? is there more that we get from it?



##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {
+
+    String DEFAULT_SEPARATOR = ",";
+    String separator = DEFAULT_SEPARATOR;
+
+    @Override
+    protected void init(IndexSchema schema, Map<String, String> args) {
+        super.init(schema, args);
+        separator = args.getOrDefault("separator", DEFAULT_SEPARATOR);
+        dimension = 3;
+        createSuffixCache(3);
+    }
+
+    @Override
+    public List<IndexableField> createFields(SchemaField field, Object value) {
+        String externalVal = value.toString();
+        String[] point = parseCommaSeparatedList(externalVal, dimension, separator);
+        String[] nvector;
+        try {
+            NumberFormat format = NumberFormat.getInstance(Locale.getDefault());
+            format.setParseIntegerOnly(false);
+            nvector = NVectorUtil.latLongToNVector(point, format);
+        } catch (ParseException e) {
+            throw new SolrException(
+                    SolrException.ErrorCode.BAD_REQUEST,
+                    "format exception parsing: "+externalVal);
+        }
+        List<IndexableField> f = new ArrayList<>((dimension * 2) + 1);
+
+        if (field.indexed()) {
+            for (int i = 0; i < dimension; i++) {
+                SchemaField sf = subField(field, i, schema);
+                f.addAll(sf.createFields(nvector[i]));
+            }
+        }
+
+        if (field.stored()) {
+            f.add(createField(field.getName(), externalVal, StoredField.TYPE));
+        }
+        return f;
+    }
+
+    @Override
+    public ValueSource getValueSource(SchemaField field, QParser parser) {
+        ArrayList<ValueSource> vs = new ArrayList<>(dimension);
+        for (int i = 0; i < dimension; i++) {
+            SchemaField sub = subField(field, i, schema);
+            vs.add(sub.getType().getValueSource(sub, parser));
+        }
+        return new NVectorValueSource(vs);
+    }
+
+    /**
+     * Given a string containing <i>dimension</i> values encoded in it, separated by commas, return a
+     * String array of length <i>dimension</i> containing the values.
+     *
+     * @param externalVal The value to parse
+     * @param dimension   The expected number of values for the point
+     * @param separator   The separator between values
+     * @return An array of the values that make up the point (aka vector)
+     * @throws SolrException if the dimension specified does not match the number found
+     */
+    public static String[] parseCommaSeparatedList(String externalVal, int dimension, String separator)

Review Comment:
   Can we have some unit tests for this? Parsing is one of those thing that seems easy but there are always edge cases and it's easy to introduce regressions.



##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {
+
+    String DEFAULT_SEPARATOR = ",";

Review Comment:
   private static final



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = lat * (Math.PI / 180);
+    double lonRad = lon * (Math.PI / 180);

Review Comment:
   `Math.toRadians`



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = lat * (Math.PI / 180);
+    double lonRad = lon * (Math.PI / 180);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {
+    return new double[] {
+      Math.asin(n[2]) * (180 / Math.PI), Math.atan(n[1] / n[0]) * (180 / Math.PI)
+    };
+  }
+
+  public static double[] NVectorToLatLong(String[] n) {
+    return NVectorToLatLong(
+        new double[] {
+          Double.parseDouble(n[0]), Double.parseDouble(n[1]), Double.parseDouble(n[2])
+        });
+  }
+
+  public static double NVectorDotProduct(double[] a, double[] b) {
+    return a[0] * b[0] + a[1] * b[1] + a[2] * b[2];
+  }
+
+  public static double NVectorDist(double[] a, double[] b) {
+    return NVectorDist(a, b, EARTH_MEAN_RADIUS_KM);
+  }
+
+  public static double NVectorDist(double[] a, double[] b, double radius) {
+    return radius * (pip2 - SloppyMath.asin(a[0] * b[0] + a[1] * b[1] + a[2] * b[2]));

Review Comment:
   ```suggestion
       return NVectorDist(NVectorDotProduct(a, b), radius);
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1207992275

   
   > I think the core of the idea is good here. Big thing missing is an update to the ref guide, probably function-queries.adoc or spatial-search.adoc?
   > 
   
   When i get a moment I'll ad something to spatial-search.adoc perhaps?
   
   > Another thought I had was how exactly do we expect users to use this. If they're still going to be providing indexable data in lat/long and also expecting lat/long for output information, then will this really be faster than using haversine? Or does it move the computation to the indexing side when we only have to do it once, so over multiple queries the total time taken gets reduced...
   
   This moves most of the calculation to the indexing side. We then only need to calculate an n-vector for the input lat/lon. The sorting can then be done on the dot-product alone. Users can optionally index spatial data in multiple formats (e.g. LanLonSpatialField and NVectorField) should they find that a performance boost. N-Vector provides faster comparison  (and great circle distance calculation) than haversine, additionally without caveats, or accuracy degradation, for calculations at poles/equator etc.
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r920385253


##########
solr/core/src/test/org/apache/solr/util/NVectorUtilTest.java:
##########
@@ -0,0 +1,65 @@
+
+/*
+ * 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.solr.util;
+
+import org.junit.Test;
+
+import java.text.DecimalFormat;
+
+import static org.junit.Assert.assertEquals;
+
+public class NVectorUtilTest {
+
+    DecimalFormat df = new DecimalFormat("##.####");
+
+    @Test
+    public void latLongToNVector() {
+        double lat = 52.024535;
+        double lon = -0.490155;
+        double[] n = NVectorUtil.latLongToNVector(lat, lon);
+        double[] ll = NVectorUtil.NVectorToLatLong(n);
+        assertSimilar(lat, ll[0]);
+        assertSimilar(lon, ll[1]);
+    }
+
+    void assertSimilar(double expected, double actual) {

Review Comment:
   Why do we need to do string formatting? Wouldn't it be more clear to use the `assertEquals(double, double, delta)` form with delta being... 0.0001 it looks like you want?



##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {

Review Comment:
   I think it's confusing to have two classes named NVector in different packages.



##########
solr/core/src/test/org/apache/solr/util/FastInvTrigTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.solr.util;
+
+import org.apache.solr.SolrTestCase;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static org.apache.solr.util.NVectorUtil.EARTH_RADIUS;
+
+public class FastInvTrigTest extends SolrTestCase {
+
+    final static int num_points = 100000;
+    final static double EPSILON = 0.0001;
+    static final Random r = new Random();
+    private static final double TEN_METERS = 0.01;
+
+    static double[][] points = new double[num_points][2];
+
+    @Before
+    public void initAll() {
+        for (int i = 0; i < num_points; i++) {
+            points[i] = generateRandomPoint();
+        }
+    }
+
+    public static double deg2rad(double deg) {
+        return deg * (Math.PI / 180);
+    }
+
+    public static double[] generateRandomPoint() {
+        double u = r.nextDouble();
+        double v = r.nextDouble();
+
+        double latitude = deg2rad(Math.toDegrees(Math.acos(u * 2 - 1)) - 90);
+        double longitude = deg2rad(360 * v - 180);
+        return new double[]{latitude, longitude};
+    }
+
+    @Test
+    public void acos() {
+        for (double i = -1; i <= 1; i = i + 0.00001) {
+            assertTrue(FastInvTrig.acos(i) - Math.acos(i) <= EPSILON);

Review Comment:
   nit: use assertEquals to get better error messages



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.solr.util;
+
+public class NVectorUtil {
+
+    public static final double EARTH_RADIUS = 6173.008;//km google:standard mean earth radius;

Review Comment:
   https://www.wolframalpha.com/input?i=earth+radius+km puts it at 6371.009 - transcription error?



##########
solr/core/src/java/org/apache/solr/schema/NVector.java:
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class NVector extends CoordinateFieldType {
+
+    @Override
+    protected void init(IndexSchema schema, Map<String, String> args) {
+        super.init(schema, args);
+        dimension = 3;
+        createSuffixCache(3);
+    }
+
+    @Override
+    public List<IndexableField> createFields(SchemaField field, Object value) {
+        String externalVal = value.toString();
+        String[] point = parseCommaSeparatedList(externalVal, dimension);
+        String[] nvector = NVectorUtil.latLongToNVector(point);
+
+        List<IndexableField> f = new ArrayList<>((dimension * 2) + 1);
+
+        if (field.indexed()) {
+            for (int i = 0; i < dimension; i++) {
+                SchemaField sf = subField(field, i, schema);
+                f.addAll(sf.createFields(nvector[i]));
+            }
+        }
+
+        if (field.stored()) {
+            f.add(createField(field.getName(), externalVal, StoredField.TYPE));
+        }
+        return f;
+    }
+
+    @Override
+    public ValueSource getValueSource(SchemaField field, QParser parser) {
+        ArrayList<ValueSource> vs = new ArrayList<>(dimension);
+        for (int i = 0; i < dimension; i++) {
+            SchemaField sub = subField(field, i, schema);
+            vs.add(sub.getType()
+                .getValueSource(sub, parser));
+        }
+        return new NVectorValueSource(vs);
+    }
+
+
+    /**
+     * Given a string containing <i>dimension</i> values encoded in it, separated by commas,
+     * return a String array of length <i>dimension</i> containing the values.
+     *
+     * @param externalVal The value to parse
+     * @param dimension   The expected number of values for the point
+     * @return An array of the values that make up the point (aka vector)
+     * @throws SolrException if the dimension specified does not match the number found
+     */
+    public static String[] parseCommaSeparatedList(String externalVal, int dimension) throws SolrException {
+        //TODO: Should we support sparse vectors?
+        String[] out = new String[dimension];
+        int idx = externalVal.indexOf(',');
+        int end = idx;
+        int start = 0;
+        int i = 0;
+        if (idx == -1 && dimension == 1 && externalVal.length() > 0) {//we have a single point, dimension better be 1
+            out[0] = externalVal.trim();
+            i = 1;
+        } else if (idx > 0) {//if it is zero, that is an error
+            //Parse out a comma separated list of values, as in: 73.5,89.2,7773.4
+            for (; i < dimension; i++) {
+                while (start < end && externalVal.charAt(start) == ' ') start++;
+                while (end > start && externalVal.charAt(end - 1) == ' ') end--;
+                if (start == end) {
+                    break;
+                }
+                out[i] = externalVal.substring(start, end);
+                start = idx + 1;
+                end = externalVal.indexOf(',', start);

Review Comment:
   Numbers will always be stored as decimals with `.` and separator as `,`? Need to make sure this doesn't mess up in other locales that use comma as the decimal separator.



##########
solr/core/src/test-files/solr/collection1/conf/schema-nvector.xml:
##########
@@ -0,0 +1,930 @@
+<?xml version="1.0" ?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+<!-- The Solr schema file. This file should be named "schema.xml" and
+     should be located where the classloader for the Solr webapp can find it.
+
+     This schema is used for testing, and as such has everything and the
+     kitchen sink thrown in. See example/solr/conf/schema.xml for a
+     more concise example.
+
+  -->
+
+<schema name="test" version="1.0">

Review Comment:
   can we minimize this schema to only the fields that it needs instead of the full 1000 lines of everything?



##########
solr/core/src/java/org/apache/solr/util/FastInvTrig.java:
##########
@@ -0,0 +1,108 @@
+
+/*
+ * 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.solr.util;
+
+public class FastInvTrig {

Review Comment:
   Did you write this code? Is it borrowed from a library? Is it an implementation of something found in a text book?



##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {
+
+    private final MultiValueSource p1;
+    private final MultiValueSource p2;
+    private final double radius;
+
+    public NVector(MultiValueSource p1, MultiValueSource p2, double radius) {
+        this.p1 = p1;
+        this.p2 = p2;
+        if (p1.dimension() != 3 || p2.dimension() != 3) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Illegal dimension for value sources");
+        }
+        this.radius = radius;
+    }
+
+    @Override
+    public FunctionValues getValues(Map<Object,Object> context, LeafReaderContext readerContext) throws IOException {
+
+        final FunctionValues vals1 = p1.getValues(context, readerContext);
+        final FunctionValues vals2 = p2.getValues(context, readerContext);
+
+        return new DoubleDocValues(this) {
+
+            @Override
+            public double doubleVal(int doc) throws IOException {
+                double[] dv1 = new double[p1.dimension()];
+                double[] dv2 = new double[p2.dimension()];
+                vals1.doubleVal(doc, dv1);
+                vals2.doubleVal(doc, dv2);
+                return  NVectorDist(dv1, dv2, radius);
+            }
+
+            @Override
+            public String toString(int doc) throws IOException {
+                return name() +
+                    ',' +
+                    vals1.toString(doc) +
+                    ',' +
+                    vals2.toString(doc) +
+                    ')';
+            }
+        };
+    }
+
+    protected String name() {
+        return "nvector";
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this.getClass() != o.getClass()) return false;

Review Comment:
   `o` could be null



##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {
+
+    private final MultiValueSource p1;
+    private final MultiValueSource p2;

Review Comment:
   I'm unclear what these are - I don't think they're lat/long. Are they some offset and phase? Two separate vectors? I don't think p1 and p2 are descriptive enough for non-domain experts (hi! that's me!)



##########
solr/core/src/test/org/apache/solr/util/FastInvTrigTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.solr.util;
+
+import org.apache.solr.SolrTestCase;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static org.apache.solr.util.NVectorUtil.EARTH_RADIUS;
+
+public class FastInvTrigTest extends SolrTestCase {
+
+    final static int num_points = 100000;
+    final static double EPSILON = 0.0001;
+    static final Random r = new Random();

Review Comment:
   we don't use this random, use the one provided by the test framework so that seeds are reproducible.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1190903304

   I found your source `FastInvTrig` repo and ran the benchmarks from there, with a few tweaks to improve measurement accuracy and also added Lucene's [SloppyMath](https://javadoc.io/doc/org.apache.lucene/lucene-core/latest/org/apache/lucene/util/SloppyMath.html) into the competition.
   
   My results are directionally the same as yours - 
   
   ```
   Benchmark                                    Mode  Cnt    Score    Error  Units
   FastInvTrigBenchmark.acosBM                  avgt    3   17.576 ±  1.243  ns/op
   FastInvTrigBenchmark.fastMathAcosBM          avgt    3   87.301 ±  4.303  ns/op
   FastInvTrigBenchmark.haversineBM             avgt    3   95.062 ± 17.137  ns/op
   FastInvTrigBenchmark.mathAcosBM              avgt    3  140.391 ±  5.508  ns/op
   FastInvTrigBenchmark.sloppyHaversineMeters   avgt    3   46.647 ±  8.927  ns/op
   FastInvTrigBenchmark.sloppyHaversineSortKey  avgt    3   33.637 ±  6.364  ns/op
   ```
   
   One concern for comparison that I have would be that the current Lucene implementation (which is only 2x slower than yours) has an upper error bound of 40cm. I think we can get there with your series expansions by adding additional terms, but I don't remember my calculus well enough to know how many we need. If we do that, how much does performance suffer? Are we still competitive? You're currently at 10m, which I suspect might be too large of a delta to be useful for the applications that I am familiar with.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r941366391


##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = Math.toRadians(lat);
+    double lonRad = Math.toRadians(lon);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {
+    return new double[] {
+      Math.asin(n[2]) * (180 / Math.PI), Math.atan(n[1] / n[0]) * (180 / Math.PI)

Review Comment:
   changed



##########
solr/core/src/test/org/apache/solr/search/function/distance/NVectorDistTest.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Locale;
+
+public class NVectorDistTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
+    initCore("solrconfig-nvector.xml", "schema-nvector.xml");
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Locale.setDefault(Locale.ENGLISH);//for parsing lat/log correctly in this test
+    super.setUp();
+    assertU(delQ("*:*"));
+    assertU(commit());
+  }
+
+  @Test
+  public void testNVector() throws Exception {
+    assertU(adoc("id", "0", "nvector", "52.02471051274793, -0.49007556238612354"));
+    assertU(commit());
+    assertJQ(
+        req("defType", "lucene", "q", "*:*", "fl", "id,nvector*", "sort", "id asc"),
+        "/response/docs/[0]== {"
+            + "'id':'0',"
+            + "'nvector_0_d1':0.6152990562577377,"
+            + "'nvector_1_d1':-0.005263047078845837,"
+            + "'nvector_2_d1':0.7882762026750415,"
+            + "'nvector':'52.02471051274793, -0.49007556238612354'}");
+
+    assertJQ(
+        req(
+            "defType", "lucene",
+            "q", "*:*",
+            "nvd", "nvdist(52.01966071979866, -0.4983083573742952,nvector)",

Review Comment:
   added to ValueSourceParser



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r941366158


##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = Math.toRadians(lat);
+    double lonRad = Math.toRadians(lon);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {

Review Comment:
   updated



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {

Review Comment:
   added



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1197908275

   - SloppyMath.asin now replaces FastInvTrig.acos
   - Sorting in the function query is on the dot product, values still fetch distance
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1191522782

   Thanks for looking into this more @madrob . 
   
   I wasn't aware of SloppyMath. If we assume Math.acos correct, then to get the same accuracy with FastInvTrig to within 40cm, I've tested it takes 17 terms, which after running a benchmark means that:
   
   ```
   Benchmark                                   (num_points)  Mode  Cnt    Score   Error  Units
   FastInvTrigBenchmark.acosBM10                    2000000  avgt    2   44.149          ms/op
   FastInvTrigBenchmark.acosBM17                    2000000  avgt    2   67.977          ms/op
   FastInvTrigBenchmark.fastMathAcosBM              2000000  avgt    2  252.578          ms/op
   FastInvTrigBenchmark.haversineBM                 2000000  avgt    2  288.209          ms/op
   FastInvTrigBenchmark.mathAcosBM                  2000000  avgt    2  300.509          ms/op
   FastInvTrigBenchmark.mathAsin                    2000000  avgt    2  320.407          ms/op
   FastInvTrigBenchmark.sloppyAsin                  2000000  avgt    2   50.152          ms/op
   FastInvTrigBenchmark.sloppyHaversinMeters        2000000  avgt    2  134.309          ms/op
   FastInvTrigBenchmark.sloppyHaversinSortKey       2000000  avgt    2   82.845          ms/op
   ```
   So then:
   
   SloppyMath.HaversinSortKey is 1.9x slower than FastInvTrig with 10 terms  (as you found)
   SloppyMath.HaversinSortKey is 1.2x slower than FastInvTrig with 17 terms
   
   However I also noticed that SloppyMath has an asin implementation, and with pi/2-asin(x) = acos(x)
   
   https://www.wolframalpha.com/input?i2d=true&i=%5C%2840%29Divide%5Bpi%2C2%5D-asin%5C%2840%29x%5C%2841%29%5C%2841%29-acos%5C%2840%29x%5C%2841%29
   
   after adding this into the benchmark, using nvector and this identity ^  for acos then:
   
   FastInvTrig with 17 terms is 1.3x slower than SloppyMath.asin !
   
   The caveat is that SloppyMath.asin uses more memory for caching values I think.
   
   So I'm wondering now whether to abandon the FastInvTrig series expansion, and use SloppyMath.asin for NVector? What do you think ?
   
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1191761343

   Can we do a similar trick to split the calculation to get an n-vector sort key and an n-vector meters and get even more speedup for the cases where we don't care about absolute distances?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922320371


##########
solr/core/src/test/org/apache/solr/util/FastInvTrigTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.solr.util;
+
+import org.apache.solr.SolrTestCase;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Random;
+
+import static org.apache.solr.util.NVectorUtil.EARTH_RADIUS;
+
+public class FastInvTrigTest extends SolrTestCase {
+
+    final static int num_points = 100000;
+    final static double EPSILON = 0.0001;
+    static final Random r = new Random();
+    private static final double TEN_METERS = 0.01;
+
+    static double[][] points = new double[num_points][2];
+
+    @Before
+    public void initAll() {
+        for (int i = 0; i < num_points; i++) {
+            points[i] = generateRandomPoint();
+        }
+    }
+
+    public static double deg2rad(double deg) {
+        return deg * (Math.PI / 180);
+    }
+
+    public static double[] generateRandomPoint() {
+        double u = r.nextDouble();
+        double v = r.nextDouble();
+
+        double latitude = deg2rad(Math.toDegrees(Math.acos(u * 2 - 1)) - 90);
+        double longitude = deg2rad(360 * v - 180);
+        return new double[]{latitude, longitude};
+    }
+
+    @Test
+    public void acos() {
+        for (double i = -1; i <= 1; i = i + 0.00001) {
+            assertTrue(FastInvTrig.acos(i) - Math.acos(i) <= EPSILON);

Review Comment:
   changed to assertEquals



##########
solr/core/src/java/org/apache/solr/util/FastInvTrig.java:
##########
@@ -0,0 +1,108 @@
+
+/*
+ * 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.solr.util;
+
+public class FastInvTrig {

Review Comment:
   Yes I wrote this, original in my github repo. It's a Maclaurin series expansion of acos. TABLE stores the coefficients that can be re-used for subsequent calculations, it also reuses x^2 for x^3,x^5 etc. Also initially my implementation required a lot of terms for convergence, until I found this https://stackoverflow.com/questions/20196000/own-asin-function-with-taylor-series-not-accurate which allows for faster convergence near -1,1. I have a benchmark in my repo to show it more performant than Math.acos or FastMath.acos, accuracy appears OK in my tests. NVector with acos faster than Haversine.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922321688


##########
solr/core/src/test/org/apache/solr/util/NVectorUtilTest.java:
##########
@@ -0,0 +1,65 @@
+
+/*
+ * 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.solr.util;
+
+import org.junit.Test;
+
+import java.text.DecimalFormat;
+
+import static org.junit.Assert.assertEquals;
+
+public class NVectorUtilTest {
+
+    DecimalFormat df = new DecimalFormat("##.####");
+
+    @Test
+    public void latLongToNVector() {
+        double lat = 52.024535;
+        double lon = -0.490155;
+        double[] n = NVectorUtil.latLongToNVector(lat, lon);
+        double[] ll = NVectorUtil.NVectorToLatLong(n);
+        assertSimilar(lat, ll[0]);
+        assertSimilar(lon, ll[1]);
+    }
+
+    void assertSimilar(double expected, double actual) {

Review Comment:
   changed to `assertEquals(double, double, delta)`



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r940411276


##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = Math.toRadians(lat);
+    double lonRad = Math.toRadians(lon);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {

Review Comment:
   nit: method names should start with lowercase (here and others)



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {

Review Comment:
   can we add javadoc for the class and methods?



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = Math.toRadians(lat);
+    double lonRad = Math.toRadians(lon);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {
+    return new double[] {
+      Math.asin(n[2]) * (180 / Math.PI), Math.atan(n[1] / n[0]) * (180 / Math.PI)

Review Comment:
   Math.toDegrees



##########
solr/core/src/test/org/apache/solr/search/function/distance/NVectorDistTest.java:
##########
@@ -0,0 +1,180 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Locale;
+
+public class NVectorDistTest extends SolrTestCaseJ4 {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    System.setProperty("enable.update.log", "false"); // schema12 doesn't support _version_
+    initCore("solrconfig-nvector.xml", "schema-nvector.xml");
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    Locale.setDefault(Locale.ENGLISH);//for parsing lat/log correctly in this test
+    super.setUp();
+    assertU(delQ("*:*"));
+    assertU(commit());
+  }
+
+  @Test
+  public void testNVector() throws Exception {
+    assertU(adoc("id", "0", "nvector", "52.02471051274793, -0.49007556238612354"));
+    assertU(commit());
+    assertJQ(
+        req("defType", "lucene", "q", "*:*", "fl", "id,nvector*", "sort", "id asc"),
+        "/response/docs/[0]== {"
+            + "'id':'0',"
+            + "'nvector_0_d1':0.6152990562577377,"
+            + "'nvector_1_d1':-0.005263047078845837,"
+            + "'nvector_2_d1':0.7882762026750415,"
+            + "'nvector':'52.02471051274793, -0.49007556238612354'}");
+
+    assertJQ(
+        req(
+            "defType", "lucene",
+            "q", "*:*",
+            "nvd", "nvdist(52.01966071979866, -0.4983083573742952,nvector)",

Review Comment:
   I missed where 'nvdist' is defined as a function. Do we need to add to `ValueSourceParser` static init block?



##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {

Review Comment:
   Ok, fair enough about not extending PointType. As for range queries... does NVector efficiently support shape intersections? I'm under the impression that it doesn't - that's part of the way that we get such a speed up on distance...



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r940115742


##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = lat * (Math.PI / 180);
+    double lonRad = lon * (Math.PI / 180);

Review Comment:
   updated



##########
solr/core/src/java/org/apache/solr/util/NVectorUtil.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.solr.util;
+
+import org.apache.lucene.util.SloppyMath;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+
+import static org.locationtech.spatial4j.distance.DistanceUtils.EARTH_MEAN_RADIUS_KM;
+
+public class NVectorUtil {
+
+  private static final double pip2 = Math.PI/2;
+
+  public static double[] latLongToNVector(double lat, double lon) {
+    double latRad = lat * (Math.PI / 180);
+    double lonRad = lon * (Math.PI / 180);
+    double x = Math.cos(latRad) * Math.cos(lonRad);
+    double y = Math.cos(latRad) * Math.sin(lonRad);
+    double z = Math.sin(latRad);
+    return new double[] {x, y, z};
+  }
+
+  public static String[] latLongToNVector(String lat, String lon) {
+    double[] nvec = latLongToNVector(Double.parseDouble(lat), Double.parseDouble(lon));
+    return new String[] {
+      Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static String[] latLongToNVector(String[] latlon) {
+    return latLongToNVector(latlon[0], latlon[1]);
+  }
+
+  public static String[] latLongToNVector(String[] point, NumberFormat formatter) throws ParseException {
+    double[] nvec = latLongToNVector(formatter.parse(point[0]).doubleValue(),formatter.parse(point[1]).doubleValue());
+    return new String[] {
+            Double.toString(nvec[0]), Double.toString(nvec[1]), Double.toString(nvec[2])
+    };
+  }
+
+  public static double[] NVectorToLatLong(double[] n) {
+    return new double[] {
+      Math.asin(n[2]) * (180 / Math.PI), Math.atan(n[1] / n[0]) * (180 / Math.PI)
+    };
+  }
+
+  public static double[] NVectorToLatLong(String[] n) {
+    return NVectorToLatLong(
+        new double[] {
+          Double.parseDouble(n[0]), Double.parseDouble(n[1]), Double.parseDouble(n[2])
+        });
+  }
+
+  public static double NVectorDotProduct(double[] a, double[] b) {
+    return a[0] * b[0] + a[1] * b[1] + a[2] * b[2];
+  }
+
+  public static double NVectorDist(double[] a, double[] b) {
+    return NVectorDist(a, b, EARTH_MEAN_RADIUS_KM);
+  }
+
+  public static double NVectorDist(double[] a, double[] b, double radius) {
+    return radius * (pip2 - SloppyMath.asin(a[0] * b[0] + a[1] * b[1] + a[2] * b[2]));

Review Comment:
   thanks for suggestion, updated



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r922321442


##########
solr/core/src/java/org/apache/solr/search/function/distance/NVector.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.solr.search.function.distance;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.solr.util.NVectorUtil.NVectorDist;
+
+public class NVector extends ValueSource {

Review Comment:
   changed now



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1192560375

   > I'm a little confused as to what the sloppyAsin results are...
   > 
   > From your tests, we should still prefer NVector with MacLaurian expansion at 17 terms over using Sloppy Haversine, right? Is Sloppy Asin using NVector with no series expansion for the trig and instead the lookup tables from Sloppy Math? Which gives us the 40cm accuracy at _almost_ the original 10 term expansion performance level?
   
   We calculate the Great circle distance as `d=R*acos(a.b)` where d = distance, R = radius, and a,b are NVectors (a.b is the scalar dot product)
   
   We also know `acos(x) = pi/2-asin(x)`
   
   So I compared FastMath.acos (with 17 terms) with SloppyMath.asin, and found that FastMath.acos with 17 terms is 1.3x slower than SloppyMath.asin.
   
   This is what I meant with FastInvTrigBenchmark.sloppyAsin
   
   so SloppyMath.asin, at the required precision, is faster than FastMath.acos, 
   
   So I was thinking of abandoning FastMath.acos in favour of SloppyMath.asin, what do you think?
   
   > Can we do a similar trick to split the calculation to get an n-vector sort key and an n-vector meters and get even more speedup for the cases where we don't care about absolute distances?
   
   Yes! From looking at the acos plot ( https://www.wolframalpha.com/input?i2d=true&i=acos%5C%2840%29x%5C%2841%29 ) it's a 1 to 1 function, so well suited for comparison. I did a quick test which confirmed that the dot product is enough for comparison between values (which is all we need to do, as we cache the nvectors in the NVectorField). So we then have the following benchmark (with NVectorSortKey as this comparison and the fastest) :
   
   ```
   Benchmark                                   (num_points)  Mode  Cnt    Score   Error  Units
   FastInvTrigBenchmark.NVectorSortKey              2000000  avgt    2   13.846          ms/op
   FastInvTrigBenchmark.acosBM10                    2000000  avgt    2   45.620          ms/op
   FastInvTrigBenchmark.acosBM17                    2000000  avgt    2   68.673          ms/op
   FastInvTrigBenchmark.fastMathAcosBM              2000000  avgt    2  249.075          ms/op
   FastInvTrigBenchmark.haversineBM                 2000000  avgt    2  289.814          ms/op
   FastInvTrigBenchmark.mathAcosBM                  2000000  avgt    2  304.411          ms/op
   FastInvTrigBenchmark.mathAsin                    2000000  avgt    2  323.126          ms/op
   FastInvTrigBenchmark.sloppyAsin                  2000000  avgt    2   49.749          ms/op
   FastInvTrigBenchmark.sloppyHaversinMeters        2000000  avgt    2  135.847          ms/op
   FastInvTrigBenchmark.sloppyHaversinSortKey       2000000  avgt    2   82.149          ms/op
   ```
   
   So in Solr perhaps we can use NVectorSortKey for sort comparisons then.
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] madrob commented on pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
madrob commented on PR #940:
URL: https://github.com/apache/solr/pull/940#issuecomment-1184550250

   please also run `./gradlew tidy` to make sure that your code adhered to our formatting conventions. :)


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] danrosher commented on a diff in pull request #940: SOLR-16292 : NVector for alternative great-circle distance calculations

Posted by GitBox <gi...@apache.org>.
danrosher commented on code in PR #940:
URL: https://github.com/apache/solr/pull/940#discussion_r940110603


##########
solr/core/src/java/org/apache/solr/schema/NVectorField.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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.solr.schema;
+
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.valuesource.MultiValueSource;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
+import org.apache.solr.uninverting.UninvertingReader;
+import org.apache.solr.util.NVectorUtil;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+public class NVectorField extends CoordinateFieldType {

Review Comment:
   NVectorField is different I think to PointType. NVector replaces lat/lon and overrides most of the important methods in PointType anyway. NVectorField would need a specialized getSpecializedRangeQuery too (not sure how to implement this for nvector yet)



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

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


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