You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/01/14 21:07:08 UTC

svn commit: r1433088 - in /lucene/dev/branches/lucene4547/lucene: codecs/src/java/org/apache/lucene/codecs/diskdv/ codecs/src/java/org/apache/lucene/codecs/memory/ codecs/src/resources/META-INF/services/ core/src/java/org/apache/lucene/codecs/lucene41/...

Author: rmuir
Date: Mon Jan 14 20:07:06 2013
New Revision: 1433088

URL: http://svn.apache.org/viewvc?rev=1433088&view=rev
Log:
lucene41 -> disk

Added:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java   (with props)
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java   (with props)
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java   (with props)
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/package.html   (with props)
Removed:
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SimpleDocValuesConsumer.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SimpleDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41SimpleDocValuesProducer.java
Modified:
    lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java
    lucene/dev/branches/lucene4547/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat
    lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
    lucene/dev/branches/lucene4547/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat
    lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java
    lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java

Added: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java?rev=1433088&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesConsumer.java Mon Jan 14 20:07:06 2013
@@ -0,0 +1,152 @@
+package org.apache.lucene.codecs.diskdv;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SimpleDVConsumer;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
+
+// nocommit fix exception handling (make sure tests find problems first)
+class DiskDocValuesConsumer extends SimpleDVConsumer {
+  final IndexOutput data, meta;
+  final int maxDoc;
+  
+  DiskDocValuesConsumer(SegmentWriteState state) throws IOException {
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "dvd");
+    data = state.directory.createOutput(dataName, state.context);
+    CodecUtil.writeHeader(data, DiskDocValuesFormat.DATA_CODEC, 
+                                DiskDocValuesFormat.VERSION_CURRENT);
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "dvm");
+    meta = state.directory.createOutput(metaName, state.context);
+    CodecUtil.writeHeader(meta, DiskDocValuesFormat.METADATA_CODEC, 
+                                DiskDocValuesFormat.VERSION_CURRENT);
+    maxDoc = state.segmentInfo.getDocCount();
+  }
+  
+  @Override
+  public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
+    meta.writeVInt(field.number);
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    int count = 0;
+    for(Number nv : values) {
+      long v = nv.longValue();
+      minValue = Math.min(minValue, v);
+      maxValue = Math.max(maxValue, v);
+      count++;
+    }
+    meta.writeLong(minValue);
+    long delta = maxValue - minValue;
+    final int bitsPerValue;
+    if (delta < 0) {
+      bitsPerValue = 64;
+    } else {
+      bitsPerValue = PackedInts.bitsRequired(delta);
+    }
+    FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(count, bitsPerValue, PackedInts.COMPACT);
+    
+    // nocommit: refactor this crap in PackedInts.java
+    // e.g. Header.load()/save() or something rather than how it works now.
+    CodecUtil.writeHeader(meta, PackedInts.CODEC_NAME, PackedInts.VERSION_CURRENT);
+    meta.writeVInt(bitsPerValue);
+    meta.writeVInt(count);
+    meta.writeVInt(formatAndBits.format.getId());
+    
+    meta.writeLong(data.getFilePointer());
+    
+    final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, count, formatAndBits.bitsPerValue, 0);
+    for(Number nv : values) {
+      writer.add(nv.longValue() - minValue);
+    }
+    writer.finish();
+  }
+
+  @Override
+  public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
+    // write the byte[] data
+    meta.writeVInt(field.number);
+    int minLength = Integer.MAX_VALUE;
+    int maxLength = Integer.MIN_VALUE;
+    final long startFP = data.getFilePointer();
+    int count = 0;
+    for(BytesRef v : values) {
+      minLength = Math.min(minLength, v.length);
+      maxLength = Math.max(maxLength, v.length);
+      data.writeBytes(v.bytes, v.offset, v.length);
+      count++;
+    }
+    meta.writeVInt(minLength);
+    meta.writeVInt(maxLength);
+    meta.writeVInt(count);
+    meta.writeLong(startFP);
+    
+    // if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
+    // otherwise, we need to record the length fields...
+    // TODO: make this more efficient. this is just as inefficient as 4.0 codec.... we can do much better.
+    if (minLength != maxLength) {
+      addNumericField(field, new Iterable<Number>() {
+        @Override
+        public Iterator<Number> iterator() {
+          final Iterator<BytesRef> inner = values.iterator();
+          return new Iterator<Number>() {
+            long addr = 0;
+
+            @Override
+            public boolean hasNext() {
+              return inner.hasNext();
+            }
+
+            @Override
+            public Number next() {
+              BytesRef b = inner.next();
+              addr += b.length;
+              return Long.valueOf(addr);
+            }
+
+            @Override
+            public void remove() { throw new UnsupportedOperationException(); } 
+          };
+        }
+      });
+    }
+  }
+
+  @Override
+  public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
+    addBinaryField(field, values);
+    addNumericField(field, docToOrd);
+  }
+  
+  @Override
+  public void close() throws IOException {
+    // nocommit: just write this to a RAMfile or something and flush it here, with #fields first.
+    // this meta is a tiny file so this hurts nobody
+    meta.writeVInt(-1);
+    IOUtils.close(data, meta);
+  }
+}

Added: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java?rev=1433088&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java (added)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesFormat.java Mon Jan 14 20:07:06 2013
@@ -0,0 +1,60 @@
+package org.apache.lucene.codecs.diskdv;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.SimpleDVConsumer;
+import org.apache.lucene.codecs.SimpleDVProducer;
+import org.apache.lucene.codecs.SimpleDocValuesFormat;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+
+// nocommit fix this
+/**
+ * Internally there are only 2 field types:
+ * BINARY: a big byte[]
+ * NUMERIC: packed ints
+ *
+ * NumericField = NUMERIC
+ * fixedLength BinaryField = BINARY
+ * variableLength BinaryField = BINARY + NUMERIC (addresses)
+ * fixedLength SortedField = BINARY + NUMERIC (ords)
+ * variableLength SortedField = BINARY + NUMERIC (addresses) + NUMERIC (ords) 
+ */
+public class DiskDocValuesFormat extends SimpleDocValuesFormat {
+
+  public DiskDocValuesFormat() {
+    super("Disk");
+  }
+
+  @Override
+  public SimpleDVConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new DiskDocValuesConsumer(state);
+  }
+
+  @Override
+  public SimpleDVProducer fieldsProducer(SegmentReadState state) throws IOException {
+    return new DiskDocValuesProducer(state);
+  }
+  
+  static final String DATA_CODEC = "DiskDocValuesData";
+  static final String METADATA_CODEC = "DiskDocValuesMetadata";
+  static final int VERSION_START = 0;
+  static final int VERSION_CURRENT = VERSION_START;
+}

Added: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java?rev=1433088&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java (added)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/DiskDocValuesProducer.java Mon Jan 14 20:07:06 2013
@@ -0,0 +1,244 @@
+package org.apache.lucene.codecs.diskdv;
+
+/*
+ * 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.
+ */
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.SimpleDVProducer;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+class DiskDocValuesProducer extends SimpleDVProducer {
+  private final Map<Integer,NumericEntry> numerics;
+  private final Map<Integer,NumericEntry> ords;
+  private final Map<Integer,BinaryEntry> binaries;
+  private final IndexInput data;
+  
+  DiskDocValuesProducer(SegmentReadState state) throws IOException {
+    String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "dvm");
+    // read in the entries from the metadata file.
+    IndexInput in = state.directory.openInput(metaName, state.context);
+    boolean success = false;
+    try {
+      CodecUtil.checkHeader(in, DiskDocValuesFormat.METADATA_CODEC, 
+                                DiskDocValuesFormat.VERSION_START,
+                                DiskDocValuesFormat.VERSION_START);
+      numerics = new HashMap<Integer,NumericEntry>();
+      ords = new HashMap<Integer,NumericEntry>();
+      binaries = new HashMap<Integer,BinaryEntry>();
+      readFields(in, state.fieldInfos);
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(in);
+      } else {
+        IOUtils.closeWhileHandlingException(in);
+      }
+    }
+    
+    String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, "dvd");
+    data = state.directory.openInput(dataName, state.context);
+    CodecUtil.checkHeader(data, DiskDocValuesFormat.DATA_CODEC, 
+                                DiskDocValuesFormat.VERSION_START,
+                                DiskDocValuesFormat.VERSION_START);
+  }
+  
+  private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
+    int fieldNumber = meta.readVInt();
+    while (fieldNumber != -1) {
+      DocValues.Type type = infos.fieldInfo(fieldNumber).getDocValuesType();
+      if (DocValues.isNumber(type) || DocValues.isFloat(type)) {
+        numerics.put(fieldNumber, readNumericEntry(meta));
+      } else if (DocValues.isBytes(type)) {
+        BinaryEntry b = readBinaryEntry(meta);
+        binaries.put(fieldNumber, b);
+        if (b.minLength != b.maxLength) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("binary entry for field: " + fieldNumber + " is corrupt");
+          }
+          // variable length byte[]: read addresses as a numeric dv field
+          numerics.put(fieldNumber, readNumericEntry(meta));
+        }
+      } else if (DocValues.isSortedBytes(type)) {
+        BinaryEntry b = readBinaryEntry(meta);
+        binaries.put(fieldNumber, b);
+        if (b.minLength != b.maxLength) {
+          if (meta.readVInt() != fieldNumber) {
+            throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
+          }
+          // variable length byte[]: read addresses as a numeric dv field
+          numerics.put(fieldNumber, readNumericEntry(meta));
+        }
+        // sorted byte[]: read ords as a numeric dv field
+        if (meta.readVInt() != fieldNumber) {
+          throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
+        }
+        ords.put(fieldNumber, readNumericEntry(meta));
+      }
+      fieldNumber = meta.readVInt();
+    }
+  }
+  
+  static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
+    NumericEntry entry = new NumericEntry();
+    entry.minValue = meta.readLong();
+    entry.header = PackedInts.readHeader(meta);
+    entry.offset = meta.readLong();
+    return entry;
+  }
+  
+  static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
+    BinaryEntry entry = new BinaryEntry();
+    entry.minLength = meta.readVInt();
+    entry.maxLength = meta.readVInt();
+    entry.count = meta.readVInt();
+    entry.offset = meta.readLong();
+    return entry;
+  }
+
+  @Override
+  public NumericDocValues getNumeric(FieldInfo field) throws IOException {
+    NumericEntry entry = numerics.get(field.number);
+    return getNumeric(field, entry);
+  }
+  
+  private NumericDocValues getNumeric(FieldInfo field, final NumericEntry entry) throws IOException {
+    final IndexInput data = this.data.clone();
+    data.seek(entry.offset);
+    final PackedInts.Reader reader = PackedInts.getDirectReaderNoHeader(data, entry.header);
+    return new NumericDocValues() {
+      @Override
+      public long get(int docID) {
+        return entry.minValue + reader.get(docID);
+      }
+    };
+  }
+
+  @Override
+  public BinaryDocValues getBinary(FieldInfo field) throws IOException {
+    BinaryEntry bytes = binaries.get(field.number);
+    if (bytes.minLength == bytes.maxLength) {
+      return getFixedBinary(field, bytes);
+    } else {
+      return getVariableBinary(field, bytes);
+    }
+  }
+  
+  private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
+    final IndexInput data = this.data.clone();
+    return new BinaryDocValues() {
+      @Override
+      public void get(int docID, BytesRef result) {
+        long address = bytes.offset + docID * (long)bytes.maxLength;
+        try {
+          data.seek(address);
+          if (result.length < bytes.maxLength) {
+            result.offset = 0;
+            result.bytes = new byte[bytes.maxLength];
+          }
+          data.readBytes(result.bytes, result.offset, bytes.maxLength);
+          result.length = bytes.maxLength;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+  
+  private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
+    final IndexInput data = this.data.clone();
+    final NumericDocValues addresses = getNumeric(field);
+    return new BinaryDocValues() {
+      @Override
+      public void get(int docID, BytesRef result) {
+        long startAddress = docID == 0 ? bytes.offset : bytes.offset + addresses.get(docID-1);
+        long endAddress = bytes.offset + addresses.get(docID);
+        int length = (int) (endAddress - startAddress);
+        try {
+          data.seek(startAddress);
+          if (result.length < length) {
+            result.offset = 0;
+            result.bytes = new byte[length];
+          }
+          data.readBytes(result.bytes, result.offset, length);
+          result.length = length;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  @Override
+  public SortedDocValues getSorted(FieldInfo field) throws IOException {
+    final int valueCount = binaries.get(field.number).count;
+    final BinaryDocValues binary = getBinary(field);
+    final NumericDocValues ordinals = getNumeric(field, ords.get(field.number));
+    return new SortedDocValues() {
+
+      @Override
+      public int getOrd(int docID) {
+        return (int) ordinals.get(docID);
+      }
+
+      @Override
+      public void lookupOrd(int ord, BytesRef result) {
+        binary.get(ord, result);
+      }
+
+      @Override
+      public int getValueCount() {
+        return valueCount;
+      }
+    };
+  }
+
+  @Override
+  public void close() throws IOException {
+    data.close();
+  }
+  
+  static class NumericEntry {
+    long offset;
+    
+    long minValue;
+    PackedInts.Header header;
+  }
+  
+  static class BinaryEntry {
+    long offset;
+
+    int count;
+    int minLength;
+    int maxLength;
+  }
+}

Added: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/package.html?rev=1433088&view=auto
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/package.html (added)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/diskdv/package.html Mon Jan 14 20:07:06 2013
@@ -0,0 +1,25 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+DocValuesFormat that accesses values directly from disk.
+</body>
+</html>

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryDocValuesFormat.java Mon Jan 14 20:07:06 2013
@@ -36,7 +36,7 @@ import org.apache.lucene.util.packed.Pac
 /** Indexes doc values to disk and loads them in RAM at
  *  search time. */
 
-// nocommit: nuke this wrapper and just make a nice impl (e.g. FST for sortedbytes)
+// nocommit: nuke this wrapper and just make a nice impl for 4.1 (e.g. FST for sortedbytes)
 public class MemoryDocValuesFormat extends SimpleDocValuesFormat {
 
   public MemoryDocValuesFormat() {

Modified: lucene/dev/branches/lucene4547/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat (original)
+++ lucene/dev/branches/lucene4547/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat Mon Jan 14 20:07:06 2013
@@ -13,5 +13,6 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
+org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
 org.apache.lucene.codecs.memory.MemoryDocValuesFormat
 org.apache.lucene.codecs.simpletext.SimpleTextSimpleDocValuesFormat

Modified: lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Mon Jan 14 20:07:06 2013
@@ -144,7 +144,7 @@ public class Lucene41Codec extends Codec
 
   private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
   // nocommit
-  private final SimpleDocValuesFormat defaultDVFormat = SimpleDocValuesFormat.forName("Lucene41");
+  private final SimpleDocValuesFormat defaultDVFormat = SimpleDocValuesFormat.forName("Disk");
 
   private final SimpleNormsFormat simpleNormsFormat = new Lucene41SimpleNormsFormat();
 

Modified: lucene/dev/branches/lucene4547/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.SimpleDocValuesFormat Mon Jan 14 20:07:06 2013
@@ -13,6 +13,4 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
-org.apache.lucene.codecs.lucene41.Lucene41SimpleDocValuesFormat
-org.apache.lucene.codecs.memory.MemoryDocValuesFormat
-org.apache.lucene.codecs.simpletext.SimpleTextSimpleDocValuesFormat
+#nocommit org.apache.lucene.codecs.lucene41.Lucene41SimpleDocValuesFormat

Modified: lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java (original)
+++ lucene/dev/branches/lucene4547/lucene/core/src/test/org/apache/lucene/search/TestComplexExplanations.java Mon Jan 14 20:07:06 2013
@@ -27,6 +27,8 @@ import org.apache.lucene.search.spans.*;
  * on the assumption that if the explanations work out right for them,
  * they should work for anything.
  */
+// nocommit: fix this slow-wrapper sortedDV fail:
+// ant test  -Dtestcase=TestComplexExplanations -Dtests.method=testMPQ7 -Dtests.seed=8FB070EE0C4130E9 -Dtests.slow=true -Dtests.locale=it_CH -Dtests.timezone=America/Grand_Turk -Dtests.file.encoding=UTF-8
 public class TestComplexExplanations extends TestExplanations {
 
   /**

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java?rev=1433088&r1=1433087&r2=1433088&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/index/RandomCodec.java Mon Jan 14 20:07:06 2013
@@ -32,9 +32,9 @@ import org.apache.lucene.codecs.SimpleDo
 import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41Codec;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41SimpleDocValuesFormat;
 import org.apache.lucene.codecs.lucene41ords.Lucene41WithOrds;
 import org.apache.lucene.codecs.bloom.TestBloomFilteredLucene41Postings;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.codecs.memory.DirectPostingsFormat;
 import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
 import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
@@ -142,7 +142,7 @@ public class RandomCodec extends Lucene4
         new MemoryPostingsFormat(false, random.nextFloat()));
     
     addDocValues(avoidCodecs,
-        new Lucene41SimpleDocValuesFormat(),
+        new DiskDocValuesFormat(),
         new SimpleTextSimpleDocValuesFormat(),
         new MemoryDocValuesFormat());