You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/04/29 12:41:26 UTC

[GitHub] [lucene] msokolov opened a new pull request #114: LUCENE-9905: PerFieldVectorFormat

msokolov opened a new pull request #114:
URL: https://github.com/apache/lucene/pull/114


   This emulates the approach taken for per-field customization of postings and doc-values formats and applies that to numeric vectors, ie VectorFormat. It registers discoverable services for Lucen90HnswVectorFormat and a new AssertingVectorFormat, to enable testing akin to what we have for the other formats. The asserting format doesn't assert very much, but it's a start.


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

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



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


[GitHub] [lucene] rmuir commented on pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #114:
URL: https://github.com/apache/lucene/pull/114#issuecomment-831983465


   is the plan to do a separate followup to break out euclidean and dot product into codec parameter and remove from FieldInfo? as these are hnsw-specific parameters, they really belong in that codec versus FieldInfo.


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

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



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


[GitHub] [lucene] msokolov commented on pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #114:
URL: https://github.com/apache/lucene/pull/114#issuecomment-831919682


   no comments here it seems; Anyway, we're really just moving the deck chairs around to be more future-extensible; I'll take the silence as consensus and merge later today.


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

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



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


[GitHub] [lucene] jpountz commented on a change in pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
jpountz commented on a change in pull request #114:
URL: https://github.com/apache/lucene/pull/114#discussion_r625821205



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90HnswVectorFormat.java
##########
@@ -77,7 +77,9 @@
   static final int VERSION_CURRENT = VERSION_START;
 
   /** Sole constructor */
-  public Lucene90HnswVectorFormat() {}
+  public Lucene90HnswVectorFormat() {
+    super("Lucene90VectorFormat");

Review comment:
       historically we've used the class name as a format name, should we use
   ```suggestion
       super("Lucene90HnswVectorFormat");
   ```
   ?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldVectorFormat.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.perfield;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.TreeMap;
+import org.apache.lucene.codecs.VectorFormat;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Enables per field numeric vector support.
+ *
+ * <p>Note, when extending this class, the name ({@link #getName}) is written into the index. In
+ * order for the field to be read, the name must resolve to your implementation via {@link
+ * #forName(String)}. This method uses Java's {@link ServiceLoader Service Provider Interface} to
+ * resolve format names.
+ *
+ * <p>Files written by each numeric vectors format have an additional suffix containing the format
+ * name. For example, in a per-field configuration instead of <code>_1.dat</code> filenames would
+ * look like <code>_1_Lucene40_0.dat</code>.
+ *
+ * @see ServiceLoader
+ * @lucene.experimental
+ */
+public abstract class PerFieldVectorFormat extends VectorFormat {
+  /** Name of this {@link VectorFormat}. */
+  public static final String PER_FIELD_NAME = "PerFieldVectors90";
+
+  /** {@link FieldInfo} attribute name used to store the format name for each field. */
+  public static final String PER_FIELD_FORMAT_KEY =
+      PerFieldVectorFormat.class.getSimpleName() + ".format";
+
+  /** {@link FieldInfo} attribute name used to store the segment suffix name for each field. */
+  public static final String PER_FIELD_SUFFIX_KEY =
+      PerFieldVectorFormat.class.getSimpleName() + ".suffix";
+
+  /** Sole constructor. */
+  protected PerFieldVectorFormat() {
+    super(PER_FIELD_NAME);
+  }
+
+  @Override
+  public VectorWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new FieldsWriter(state);
+  }
+
+  @Override
+  public VectorReader fieldsReader(SegmentReadState state) throws IOException {
+    return new FieldsReader(state);
+  }
+
+  /**
+   * Returns the numeric vector format that should be used for writing new segments of <code>field
+   * </code>.
+   *
+   * <p>The field to format mapping is written to the index, so this method is only invoked when
+   * writing, not when reading.
+   */
+  public abstract VectorFormat getVectorFormatForField(String field);
+
+  private class FieldsWriter extends VectorWriter {
+    private final Map<VectorFormat, WriterAndSuffix> formats;
+    private final Map<String, Integer> suffixes = new HashMap<>();
+    private final SegmentWriteState segmentWriteState;
+
+    FieldsWriter(SegmentWriteState segmentWriteState) {
+      this.segmentWriteState = segmentWriteState;
+      formats = new HashMap<>();
+    }
+
+    @Override
+    public void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException {
+      getInstance(fieldInfo).writeField(fieldInfo, values);
+    }
+
+    @Override
+    public void finish() throws IOException {
+      for (WriterAndSuffix was : formats.values()) {
+        was.writer.finish();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      IOUtils.close(formats.values());
+    }
+
+    private VectorWriter getInstance(FieldInfo field) throws IOException {
+      VectorFormat format = null;
+      String fieldFormatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
+      if (fieldFormatName != null) {
+        format = VectorFormat.forName(fieldFormatName);
+      }
+      if (format == null) {
+        format = getVectorFormatForField(field.name);
+      }
+      if (format == null) {
+        throw new IllegalStateException(
+            "invalid null VectorFormat for field=\"" + field.name + "\"");
+      }
+      final String formatName = format.getName();
+
+      field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
+      Integer suffix = null;
+
+      WriterAndSuffix writerAndSuffix = formats.get(format);
+      if (writerAndSuffix == null) {
+        // First time we are seeing this format; create a new instance
+
+        String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
+        if (suffixAtt != null) {
+          suffix = Integer.valueOf(suffixAtt);
+        }
+
+        if (suffix == null) {
+          // bump the suffix
+          suffix = suffixes.get(formatName);
+          if (suffix == null) {
+            suffix = 0;
+          } else {
+            suffix = suffix + 1;
+          }
+        }
+        suffixes.put(formatName, suffix);
+
+        String segmentSuffix =
+            getFullSegmentSuffix(
+                segmentWriteState.segmentSuffix, getSuffix(formatName, Integer.toString(suffix)));
+        writerAndSuffix =
+            new WriterAndSuffix(
+                format.fieldsWriter(new SegmentWriteState(segmentWriteState, segmentSuffix)),
+                suffix);
+        formats.put(format, writerAndSuffix);
+      } else {
+        // we've already seen this format, so just grab its suffix
+        assert suffixes.containsKey(formatName);
+        suffix = writerAndSuffix.suffix;
+      }
+
+      field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
+      return writerAndSuffix.writer;
+    }
+  }
+
+  /** VectorReader that can wrap multiple delegate readers, selected by field. */
+  public static class FieldsReader extends VectorReader {
+
+    private final Map<String, VectorReader> fields = new TreeMap<>();
+
+    /**
+     * Create a FieldsReader over a segment, opening VectorReaders for each VectorFormat specified
+     * by the indexed numeric vector fields.
+     *
+     * @param readState defines the fields
+     * @throws IOException if one of the delegate readers throws
+     */
+    public FieldsReader(final SegmentReadState readState) throws IOException {
+
+      // Init each unique format:
+      boolean success = false;
+      Map<String, VectorReader> formats = new HashMap<>();
+      try {
+        // Read field name -> format name
+        for (FieldInfo fi : readState.fieldInfos) {
+          if (fi.hasVectorValues()) {
+            final String fieldName = fi.name;
+            final String formatName = fi.getAttribute(PER_FIELD_FORMAT_KEY);
+            if (formatName != null) {
+              // null formatName means the field is in fieldInfos, but has no vectors!
+              final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
+              if (suffix == null) {
+                throw new IllegalStateException(
+                    "missing attribute: " + PER_FIELD_SUFFIX_KEY + " for field: " + fieldName);
+              }
+              VectorFormat format = VectorFormat.forName(formatName);
+              String segmentSuffix =
+                  getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix));
+              if (!formats.containsKey(segmentSuffix)) {
+                formats.put(
+                    segmentSuffix,
+                    format.fieldsReader(new SegmentReadState(readState, segmentSuffix)));
+              }
+              fields.put(fieldName, formats.get(segmentSuffix));
+            }
+          }
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(formats.values());
+        }
+      }
+    }
+
+    /**
+     * Return the underlying VectorReader for the given field
+     *
+     * @param field the name of a numeric vector field
+     */
+    public VectorReader getFieldReader(String field) {
+      return fields.get(field);
+    }
+
+    @Override
+    public void checkIntegrity() throws IOException {
+      for (VectorReader reader : fields.values()) {
+        reader.checkIntegrity();
+      }
+    }
+
+    @Override
+    public VectorValues getVectorValues(String field) throws IOException {
+      VectorReader vectorReader = fields.get(field);
+      if (vectorReader == null) {
+        return null;
+      } else {
+        return vectorReader.getVectorValues(field);
+      }
+    }
+
+    @Override
+    public TopDocs search(String field, float[] target, int k, int fanout) throws IOException {
+      return fields.get(field).search(field, target, k, fanout);

Review comment:
       nit: It feels inconsistent that `getVectorValues` is lenient when a field doesn't have vectors while this one isn't? (I believe it doesn't matter in practice because CodecReader makes sure to only pull vectors when field infos say that there are vectors, but consistency would still be nice?)




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

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



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


[GitHub] [lucene] msokolov commented on a change in pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #114:
URL: https://github.com/apache/lucene/pull/114#discussion_r627497358



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldVectorFormat.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.codecs.perfield;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.TreeMap;
+import org.apache.lucene.codecs.VectorFormat;
+import org.apache.lucene.codecs.VectorReader;
+import org.apache.lucene.codecs.VectorWriter;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.VectorValues;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.IOUtils;
+
+/**
+ * Enables per field numeric vector support.
+ *
+ * <p>Note, when extending this class, the name ({@link #getName}) is written into the index. In
+ * order for the field to be read, the name must resolve to your implementation via {@link
+ * #forName(String)}. This method uses Java's {@link ServiceLoader Service Provider Interface} to
+ * resolve format names.
+ *
+ * <p>Files written by each numeric vectors format have an additional suffix containing the format
+ * name. For example, in a per-field configuration instead of <code>_1.dat</code> filenames would
+ * look like <code>_1_Lucene40_0.dat</code>.
+ *
+ * @see ServiceLoader
+ * @lucene.experimental
+ */
+public abstract class PerFieldVectorFormat extends VectorFormat {
+  /** Name of this {@link VectorFormat}. */
+  public static final String PER_FIELD_NAME = "PerFieldVectors90";
+
+  /** {@link FieldInfo} attribute name used to store the format name for each field. */
+  public static final String PER_FIELD_FORMAT_KEY =
+      PerFieldVectorFormat.class.getSimpleName() + ".format";
+
+  /** {@link FieldInfo} attribute name used to store the segment suffix name for each field. */
+  public static final String PER_FIELD_SUFFIX_KEY =
+      PerFieldVectorFormat.class.getSimpleName() + ".suffix";
+
+  /** Sole constructor. */
+  protected PerFieldVectorFormat() {
+    super(PER_FIELD_NAME);
+  }
+
+  @Override
+  public VectorWriter fieldsWriter(SegmentWriteState state) throws IOException {
+    return new FieldsWriter(state);
+  }
+
+  @Override
+  public VectorReader fieldsReader(SegmentReadState state) throws IOException {
+    return new FieldsReader(state);
+  }
+
+  /**
+   * Returns the numeric vector format that should be used for writing new segments of <code>field
+   * </code>.
+   *
+   * <p>The field to format mapping is written to the index, so this method is only invoked when
+   * writing, not when reading.
+   */
+  public abstract VectorFormat getVectorFormatForField(String field);
+
+  private class FieldsWriter extends VectorWriter {
+    private final Map<VectorFormat, WriterAndSuffix> formats;
+    private final Map<String, Integer> suffixes = new HashMap<>();
+    private final SegmentWriteState segmentWriteState;
+
+    FieldsWriter(SegmentWriteState segmentWriteState) {
+      this.segmentWriteState = segmentWriteState;
+      formats = new HashMap<>();
+    }
+
+    @Override
+    public void writeField(FieldInfo fieldInfo, VectorValues values) throws IOException {
+      getInstance(fieldInfo).writeField(fieldInfo, values);
+    }
+
+    @Override
+    public void finish() throws IOException {
+      for (WriterAndSuffix was : formats.values()) {
+        was.writer.finish();
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      IOUtils.close(formats.values());
+    }
+
+    private VectorWriter getInstance(FieldInfo field) throws IOException {
+      VectorFormat format = null;
+      String fieldFormatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
+      if (fieldFormatName != null) {
+        format = VectorFormat.forName(fieldFormatName);
+      }
+      if (format == null) {
+        format = getVectorFormatForField(field.name);
+      }
+      if (format == null) {
+        throw new IllegalStateException(
+            "invalid null VectorFormat for field=\"" + field.name + "\"");
+      }
+      final String formatName = format.getName();
+
+      field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
+      Integer suffix = null;
+
+      WriterAndSuffix writerAndSuffix = formats.get(format);
+      if (writerAndSuffix == null) {
+        // First time we are seeing this format; create a new instance
+
+        String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
+        if (suffixAtt != null) {
+          suffix = Integer.valueOf(suffixAtt);
+        }
+
+        if (suffix == null) {
+          // bump the suffix
+          suffix = suffixes.get(formatName);
+          if (suffix == null) {
+            suffix = 0;
+          } else {
+            suffix = suffix + 1;
+          }
+        }
+        suffixes.put(formatName, suffix);
+
+        String segmentSuffix =
+            getFullSegmentSuffix(
+                segmentWriteState.segmentSuffix, getSuffix(formatName, Integer.toString(suffix)));
+        writerAndSuffix =
+            new WriterAndSuffix(
+                format.fieldsWriter(new SegmentWriteState(segmentWriteState, segmentSuffix)),
+                suffix);
+        formats.put(format, writerAndSuffix);
+      } else {
+        // we've already seen this format, so just grab its suffix
+        assert suffixes.containsKey(formatName);
+        suffix = writerAndSuffix.suffix;
+      }
+
+      field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
+      return writerAndSuffix.writer;
+    }
+  }
+
+  /** VectorReader that can wrap multiple delegate readers, selected by field. */
+  public static class FieldsReader extends VectorReader {
+
+    private final Map<String, VectorReader> fields = new TreeMap<>();
+
+    /**
+     * Create a FieldsReader over a segment, opening VectorReaders for each VectorFormat specified
+     * by the indexed numeric vector fields.
+     *
+     * @param readState defines the fields
+     * @throws IOException if one of the delegate readers throws
+     */
+    public FieldsReader(final SegmentReadState readState) throws IOException {
+
+      // Init each unique format:
+      boolean success = false;
+      Map<String, VectorReader> formats = new HashMap<>();
+      try {
+        // Read field name -> format name
+        for (FieldInfo fi : readState.fieldInfos) {
+          if (fi.hasVectorValues()) {
+            final String fieldName = fi.name;
+            final String formatName = fi.getAttribute(PER_FIELD_FORMAT_KEY);
+            if (formatName != null) {
+              // null formatName means the field is in fieldInfos, but has no vectors!
+              final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
+              if (suffix == null) {
+                throw new IllegalStateException(
+                    "missing attribute: " + PER_FIELD_SUFFIX_KEY + " for field: " + fieldName);
+              }
+              VectorFormat format = VectorFormat.forName(formatName);
+              String segmentSuffix =
+                  getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix));
+              if (!formats.containsKey(segmentSuffix)) {
+                formats.put(
+                    segmentSuffix,
+                    format.fieldsReader(new SegmentReadState(readState, segmentSuffix)));
+              }
+              fields.put(fieldName, formats.get(segmentSuffix));
+            }
+          }
+        }
+        success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeWhileHandlingException(formats.values());
+        }
+      }
+    }
+
+    /**
+     * Return the underlying VectorReader for the given field
+     *
+     * @param field the name of a numeric vector field
+     */
+    public VectorReader getFieldReader(String field) {
+      return fields.get(field);
+    }
+
+    @Override
+    public void checkIntegrity() throws IOException {
+      for (VectorReader reader : fields.values()) {
+        reader.checkIntegrity();
+      }
+    }
+
+    @Override
+    public VectorValues getVectorValues(String field) throws IOException {
+      VectorReader vectorReader = fields.get(field);
+      if (vectorReader == null) {
+        return null;
+      } else {
+        return vectorReader.getVectorValues(field);
+      }
+    }
+
+    @Override
+    public TopDocs search(String field, float[] target, int k, int fanout) throws IOException {
+      return fields.get(field).search(field, target, k, fanout);

Review comment:
       That's fair. I kept it lenient, since that is what we do eg in DocValues.




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

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



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


[GitHub] [lucene] msokolov merged pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
msokolov merged pull request #114:
URL: https://github.com/apache/lucene/pull/114


   


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

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



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


[GitHub] [lucene] msokolov commented on pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #114:
URL: https://github.com/apache/lucene/pull/114#issuecomment-833567527


   > is the plan to do a separate followup to break out euclidean and dot product into codec parameter and remove from FieldInfo? as these are hnsw-specific parameters, they really belong in that codec versus FieldInfo.
   
   Hmm, I think the similarity function is not specific to hnsw, but is likely to apply to any vector search, for which you always need to define some similarity function.  Adrien's had earlier proposed we should move the truly HNSW-specific parameters like max-connections and beam-width, which pertain to the graph construction, out of VectorField (where we have VectorField.createHnswType) and into the codec, so I was thinking that would be the next step to make the API truly agnostic to the vector indexing codec.
   
   I'll address all the comments here (thanks!) and post a new rev today


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

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



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


[GitHub] [lucene] rmuir commented on a change in pull request #114: LUCENE-9905: PerFieldVectorFormat

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #114:
URL: https://github.com/apache/lucene/pull/114#discussion_r625823163



##########
File path: lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.VectorFormat
##########
@@ -0,0 +1,33 @@
+#
+# 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.
+#
+
+#  Licensed to the Apache Software Foundation (ASF) under one or more

Review comment:
       all these SPI files seem to have double copyrights




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

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



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