Dear Shawn,
see attachment for my first "brute force" no-compression attempt.
Kind regards,
Jochen
Zitat von Shawn Heisey <s...@elyograg.org>:
On 4/29/2014 4:20 AM, Jochen Barth wrote:
BTW: stored field compression:
are all "stored fields" within a document are put into one compressed chunk,
or by per-field basis?
Here's the issue that added the compression to Lucene:
https://issues.apache.org/jira/browse/LUCENE-4226
It was made the default stored field format for Lucene, which also made
it the default for Solr. At this time, there is no way to remove
compression on Solr without writing custom code. I filed an issue to
make it configurable, but I don't know how to do it. Nobody else has
offered a solution either. One day I might find some time to take a
look at the issue and see if I can solve it myself.
https://issues.apache.org/jira/browse/SOLR-4375
Here's the author's blog post that goes into more detail than the LUCENE
issue:
http://blog.jpountz.net/post/33247161884/efficient-compressed-stored-fields-with-lucene
Thanks,
Shawn
diff -c -r solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
*** solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java 2013-11-01 07:03:52.000000000 +0100
--- solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java 2014-04-29 13:58:27.000000000 +0200
***************
*** 38,43 ****
--- 38,44 ----
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
+ import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
***************
*** 56,62 ****
@Deprecated
public class Lucene41Codec extends Codec {
// TODO: slightly evil
! private final StoredFieldsFormat fieldsFormat = new CompressingStoredFieldsFormat("Lucene41StoredFields", CompressionMode.FAST, 1 << 14) {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
--- 57,63 ----
@Deprecated
public class Lucene41Codec extends Codec {
// TODO: slightly evil
! private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat() {
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
diff -c -r solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
*** solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java 2013-11-01 07:03:52.000000000 +0100
--- solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java 2014-04-29 13:57:08.000000000 +0200
***************
*** 32,38 ****
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
! import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentWriteState;
--- 32,38 ----
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
! import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentWriteState;
***************
*** 53,59 ****
// (it writes a minor version, etc).
@Deprecated
public class Lucene42Codec extends Codec {
! private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
--- 53,59 ----
// (it writes a minor version, etc).
@Deprecated
public class Lucene42Codec extends Codec {
! private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
diff -c -r solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java
*** solr-4.8.0.original/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java 2013-11-01 07:03:52.000000000 +0100
--- solr-4.8.0/lucene/core/src/java/org/apache/lucene/codecs/lucene46/Lucene46Codec.java 2014-04-29 13:56:58.000000000 +0200
***************
*** 28,34 ****
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
! import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
--- 28,34 ----
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
! import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
***************
*** 39,45 ****
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
! * {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene46 package documentation for file format details.
* @lucene.experimental
--- 39,45 ----
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
!
*
* @see org.apache.lucene.codecs.lucene46 package documentation for file format details.
* @lucene.experimental
***************
*** 48,54 ****
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
public class Lucene46Codec extends Codec {
! private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();
--- 48,54 ----
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
public class Lucene46Codec extends Codec {
! private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene46FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene46SegmentInfoFormat();