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

[GitHub] [lucene-solr] jtibshirani commented on a change in pull request #2444: LUCENE-9705: Create Lucene90StoredFieldsFormat

jtibshirani commented on a change in pull request #2444:
URL: https://github.com/apache/lucene-solr/pull/2444#discussion_r585829585



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/compressing/Lucene90CompressingStoredFieldsFormat.java
##########
@@ -40,7 +40,7 @@
  *
  * @lucene.experimental
  */
-public class CompressingStoredFieldsFormat extends StoredFieldsFormat {
+public class Lucene90CompressingStoredFieldsFormat extends StoredFieldsFormat {

Review comment:
       I think these should be moved to the package `lucene90.compressing`.

##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene87/Lucene87Codec.java
##########
@@ -101,14 +102,22 @@ public DocValuesFormat getDocValuesFormatForField(String field) {
 
   /** Instantiates a new codec. */
   public Lucene87Codec() {
+    this(Mode.BEST_COMPRESSION);

Review comment:
       Maybe we could avoid adding a default constructor here? I think it's only used in `Lucene87RWCodec`.

##########
File path: lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene87/Lucene87RWCodec.java
##########
@@ -37,6 +38,17 @@ public PostingsFormat getPostingsFormatForField(String field) {
           return defaultPF;
         }
       };
+  private final Mode mode;
+
+  public Lucene87RWCodec() {
+    super();
+    this.mode = Mode.BEST_COMPRESSION;

Review comment:
       I think we should default to `BEST_SPEED` instead of `BEST_COMPRESSION`, otherwise `BEST_SPEED` won't be exercised in unit tests. We could factor this like the other `Lucene86RWCodec` classes and just create a default `Lucene87StoredFieldsFormat` here?

##########
File path: lucene/core/src/java/org/apache/lucene/codecs/compressing/LZ4WithPresetDictCompressionMode.java
##########
@@ -14,12 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.codecs.lucene87;
+package org.apache.lucene.codecs.compressing;

Review comment:
       It seems like these classes should also be duplicated into 87 and 90 versions, with the 90 versions living in the `lucene90.compressing` package. In other places we decided not to share encoder/ decoder helper classes, for example we duplicated `ForUtil` instead of sharing it across postings formats.

##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsFormat.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.backward_codecs.lucene50.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.StoredFieldsReader;
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.packed.DirectMonotonicWriter;
+
+/**
+ * A {@link StoredFieldsFormat} that compresses documents in chunks in order to improve the
+ * compression ratio.
+ *
+ * <p>For a chunk size of <var>chunkSize</var> bytes, this {@link StoredFieldsFormat} does not
+ * support documents larger than (<code>2<sup>31</sup> - chunkSize</code>) bytes.
+ *
+ * <p>For optimal performance, you should use a {@link MergePolicy} that returns segments that have
+ * the biggest byte size first.
+ *
+ * @lucene.experimental
+ */
+public class Lucene50CompressingStoredFieldsFormat extends StoredFieldsFormat {
+
+  /** format name */
+  protected final String formatName;

Review comment:
       No strong opinion, but instead of making these protected we could hardcode certain values in `Lucene50RWCompressingStoredFieldsFormat`. This is what we do in `Lucene50RWPostingsFormat`.

##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsReader.java
##########
@@ -654,15 +669,7 @@ public void readBytes(byte[] b, int offset, int len) throws IOException {
 
               @Override
               public void skipBytes(long numBytes) throws IOException {
-                if (numBytes < 0) {
-                  throw new IllegalArgumentException("numBytes must be >= 0, got " + numBytes);
-                }
-                while (numBytes > bytes.length) {
-                  numBytes -= bytes.length;
-                  fillBuffer();
-                }
-                bytes.offset += numBytes;
-                bytes.length -= numBytes;
+                skipBytesSlowly(numBytes);

Review comment:
       Why did this logic change?

##########
File path: lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50RWCompressingStoredFieldsFormat.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.backward_codecs.lucene50.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/** RW impersonation of Lucene50CompressingStoredFieldsFormat. */
+public class Lucene50RWCompressingStoredFieldsFormat extends Lucene50CompressingStoredFieldsFormat {
+
+  /** Sole constructor. */
+  public Lucene50RWCompressingStoredFieldsFormat(

Review comment:
       I think this constructor is unused (also the comment seems inaccurate).

##########
File path: lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene87/Lucene87RWCodec.java
##########
@@ -57,4 +69,10 @@ public PostingsFormat postingsFormat() {
   public TermVectorsFormat termVectorsFormat() {
     return new Lucene50RWTermVectorsFormat();
   }
+
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    // TODO needs to consider compression mode?

Review comment:
       Could you explain details of this TODO?




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