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 2015/03/01 21:30:50 UTC

svn commit: r1663157 - in /lucene/dev/trunk: lucene/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/src/java/org/apache/lucene/codecs/lucene50/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucen...

Author: rmuir
Date: Sun Mar  1 20:30:49 2015
New Revision: 1663157

URL: http://svn.apache.org/r1663157
Log:
LUCENE-6317: Fix readStringStringMap API, reduce memory usage

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataInput.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
    lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
    lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Mar  1 20:30:49 2015
@@ -189,6 +189,9 @@ API Changes
 * LUCENE-6303: Replaced FilterCache with QueryCache and CachingWrapperFilter
   with CachingWrapperQuery. (Adrien Grand)
 
+* LUCENE-6317: Deprecate DataOutput.writeStringSet and writeStringStringMap.
+  Use writeSetOfStrings/Maps instead. (Mike McCandless, Robert Muir)
+
 Other
 
 * LUCENE-6248: Remove unused odd constants from StandardSyntaxParser.jj

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java Sun Mar  1 20:30:49 2015
@@ -140,7 +140,7 @@ public class SimpleTextSegmentInfoFormat
       SimpleTextUtil.checkFooter(input);
 
       SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount,
-                                         isCompoundFile, null, diagnostics, id, Collections.unmodifiableMap(attributes));
+                                         isCompoundFile, null, Collections.unmodifiableMap(diagnostics), id, Collections.unmodifiableMap(attributes));
       info.setFiles(files);
       return info;
     }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50FieldInfosFormat.java Sun Mar  1 20:30:49 2015
@@ -50,7 +50,7 @@ import org.apache.lucene.store.IndexOutp
  *   <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
  *   <li>FieldBits, IndexOptions, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
  *   <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
- *   <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
+ *   <li>Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
  *   <li>DocValuesGen --&gt; {@link DataOutput#writeLong(long) Int64}</li>
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
@@ -112,7 +112,7 @@ public final class Lucene50FieldInfosFor
       Throwable priorE = null;
       FieldInfo infos[] = null;
       try {
-        CodecUtil.checkIndexHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
+        int format = CodecUtil.checkIndexHeader(input, Lucene50FieldInfosFormat.CODEC_NAME, 
                                      Lucene50FieldInfosFormat.FORMAT_START, 
                                      Lucene50FieldInfosFormat.FORMAT_CURRENT,
                                      segmentInfo.getId(), segmentSuffix);
@@ -136,10 +136,15 @@ public final class Lucene50FieldInfosFor
           // DV Types are packed in one byte
           final DocValuesType docValuesType = getDocValuesType(input, input.readByte());
           final long dvGen = input.readLong();
-          final Map<String,String> attributes = input.readStringStringMap();
+          final Map<String,String> attributes;
+          if (format >= FORMAT_SAFE_MAPS) {
+            attributes = input.readMapOfStrings();
+          } else {
+            attributes = Collections.unmodifiableMap(input.readStringStringMap());
+          }
           try {
             infos[i] = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, 
-                                     indexOptions, docValuesType, dvGen, Collections.unmodifiableMap(attributes));
+                                     indexOptions, docValuesType, dvGen, attributes);
             infos[i].checkConsistency();
           } catch (IllegalStateException e) {
             throw new CorruptIndexException("invalid fieldinfo for field: " + name + ", fieldNumber=" + fieldNumber, input, e);
@@ -264,7 +269,7 @@ public final class Lucene50FieldInfosFor
         // pack the DV type and hasNorms in one byte
         output.writeByte(docValuesByte(fi.getDocValuesType()));
         output.writeLong(fi.getDocValuesGen());
-        output.writeStringStringMap(fi.attributes());
+        output.writeMapOfStrings(fi.attributes());
       }
       CodecUtil.writeFooter(output);
     }
@@ -276,7 +281,8 @@ public final class Lucene50FieldInfosFor
   // Codec header
   static final String CODEC_NAME = "Lucene50FieldInfos";
   static final int FORMAT_START = 0;
-  static final int FORMAT_CURRENT = FORMAT_START;
+  static final int FORMAT_SAFE_MAPS = 1;
+  static final int FORMAT_CURRENT = FORMAT_SAFE_MAPS;
   
   // Field flags
   static final byte STORE_TERMVECTOR = 0x1;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java Sun Mar  1 20:30:49 2015
@@ -48,8 +48,8 @@ import org.apache.lucene.util.Version;
  *   <li>Header --&gt; {@link CodecUtil#writeIndexHeader IndexHeader}</li>
  *   <li>SegSize --&gt; {@link DataOutput#writeInt Int32}</li>
  *   <li>SegVersion --&gt; {@link DataOutput#writeString String}</li>
- *   <li>Files --&gt; {@link DataOutput#writeStringSet Set&lt;String&gt;}</li>
- *   <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
+ *   <li>Files --&gt; {@link DataOutput#writeSetOfStrings Set&lt;String&gt;}</li>
+ *   <li>Diagnostics,Attributes --&gt; {@link DataOutput#writeMapOfStrings Map&lt;String,String&gt;}</li>
  *   <li>IsCompoundFile --&gt; {@link DataOutput#writeByte Int8}</li>
  *   <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
@@ -83,7 +83,7 @@ public class Lucene50SegmentInfoFormat e
       Throwable priorE = null;
       SegmentInfo si = null;
       try {
-        CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
+        int format = CodecUtil.checkIndexHeader(input, Lucene50SegmentInfoFormat.CODEC_NAME,
                                           Lucene50SegmentInfoFormat.VERSION_START,
                                           Lucene50SegmentInfoFormat.VERSION_CURRENT,
                                           segmentID, "");
@@ -94,11 +94,22 @@ public class Lucene50SegmentInfoFormat e
           throw new CorruptIndexException("invalid docCount: " + docCount, input);
         }
         final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
-        final Map<String,String> diagnostics = input.readStringStringMap();
-        final Set<String> files = input.readStringSet();
-        final Map<String,String> attributes = input.readStringStringMap();
         
-        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, Collections.unmodifiableMap(attributes));
+        final Map<String,String> diagnostics;
+        final Set<String> files;
+        final Map<String,String> attributes;
+        
+        if (format >= VERSION_SAFE_MAPS) {
+          diagnostics = input.readMapOfStrings();
+          files = input.readSetOfStrings();
+          attributes = input.readMapOfStrings();
+        } else {
+          diagnostics = Collections.unmodifiableMap(input.readStringStringMap());
+          files = Collections.unmodifiableSet(input.readStringSet());
+          attributes = Collections.unmodifiableMap(input.readStringStringMap());
+        }
+        
+        si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes);
         si.setFiles(files);
       } catch (Throwable exception) {
         priorE = exception;
@@ -132,15 +143,15 @@ public class Lucene50SegmentInfoFormat e
       output.writeInt(si.getDocCount());
 
       output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
-      output.writeStringStringMap(si.getDiagnostics());
+      output.writeMapOfStrings(si.getDiagnostics());
       Set<String> files = si.files();
       for (String file : files) {
         if (!IndexFileNames.parseSegmentName(file).equals(si.name)) {
           throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files);
         }
       }
-      output.writeStringSet(files);
-      output.writeStringStringMap(si.getAttributes());
+      output.writeSetOfStrings(files);
+      output.writeMapOfStrings(si.getAttributes());
       CodecUtil.writeFooter(output);
     }
   }
@@ -149,5 +160,6 @@ public class Lucene50SegmentInfoFormat e
   public final static String SI_EXTENSION = "si";
   static final String CODEC_NAME = "Lucene50SegmentInfo";
   static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_SAFE_MAPS = 1;
+  static final int VERSION_CURRENT = VERSION_SAFE_MAPS;
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Mar  1 20:30:49 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.text.NumberFormat;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Locale;
@@ -177,7 +178,7 @@ class DocumentsWriterPerThread {
     pendingUpdates.clear();
     deleteSlice = deleteQueue.newSlice();
    
-    segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     assert numDocsInRAM == 0;
     if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
       infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue);  

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Sun Mar  1 20:30:49 2015
@@ -17,8 +17,8 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  *  Access to the Field Info file that describes document fields and whether or
@@ -44,7 +44,7 @@ public final class FieldInfo {
   private IndexOptions indexOptions = IndexOptions.NONE;
   private boolean storePayloads; // whether this field stores payloads together with term positions
 
-  private Map<String,String> attributes;
+  private final Map<String,String> attributes;
 
   private long dvGen;
   /**
@@ -55,16 +55,10 @@ public final class FieldInfo {
   public FieldInfo(String name, int number, boolean storeTermVector, boolean omitNorms, 
       boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues,
       long dvGen, Map<String,String> attributes) {
-    if (docValues == null) {
-      throw new NullPointerException("DocValuesType cannot be null (field: \"" + name + "\")");
-    }
-    if (indexOptions == null) {
-      throw new NullPointerException("IndexOptions cannot be null (field: \"" + name + "\")");
-    }
-    this.name = name;
+    this.name = Objects.requireNonNull(name);
     this.number = number;
-    this.docValuesType = docValues;
-    this.indexOptions = indexOptions;
+    this.docValuesType = Objects.requireNonNull(docValues, "DocValuesType cannot be null (field: \"" + name + "\")");
+    this.indexOptions = Objects.requireNonNull(indexOptions, "IndexOptions cannot be null (field: \"" + name + "\")");
     if (indexOptions != IndexOptions.NONE) {
       this.storeTermVector = storeTermVector;
       this.storePayloads = storePayloads;
@@ -75,7 +69,7 @@ public final class FieldInfo {
       this.omitNorms = false;
     }
     this.dvGen = dvGen;
-    this.attributes = attributes;
+    this.attributes = Objects.requireNonNull(attributes);
     assert checkConsistency();
   }
 
@@ -246,11 +240,7 @@ public final class FieldInfo {
    * Get a codec attribute value, or null if it does not exist
    */
   public String getAttribute(String key) {
-    if (attributes == null) {
-      return null;
-    } else {
-      return attributes.get(key);
-    }
+    return attributes.get(key);
   }
   
   /**
@@ -264,14 +254,11 @@ public final class FieldInfo {
    * the new value.
    */
   public String putAttribute(String key, String value) {
-    if (attributes == null) {
-      attributes = new HashMap<>();
-    }
     return attributes.put(key, value);
   }
   
   /**
-   * Returns internal codec attributes map. May be null if no mappings exist.
+   * Returns internal codec attributes map.
    */
   public Map<String,String> attributes() {
     return attributes;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java Sun Mar  1 20:30:49 2015
@@ -287,7 +287,7 @@ public class FieldInfos implements Itera
         // before then we'll get the same name and number,
         // else we'll allocate a new one:
         final int fieldNumber = globalFieldNumbers.addOrGet(name, -1, DocValuesType.NONE);
-        fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, null);
+        fi = new FieldInfo(name, fieldNumber, false, false, false, IndexOptions.NONE, DocValuesType.NONE, -1, new HashMap<>());
         assert !byName.containsKey(fi.name);
         globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, DocValuesType.NONE);
         byName.put(fi.name, fi);
@@ -310,7 +310,7 @@ public class FieldInfos implements Itera
         // before then we'll get the same name and number,
         // else we'll allocate a new one:
         final int fieldNumber = globalFieldNumbers.addOrGet(name, preferredFieldNumber, docValues);
-        fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, -1, null);
+        fi = new FieldInfo(name, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, -1, new HashMap<>());
         assert !byName.containsKey(fi.name);
         globalFieldNumbers.verifyConsistent(Integer.valueOf(fi.number), fi.name, fi.getDocValuesType());
         byName.put(fi.name, fi);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sun Mar  1 20:30:49 2015
@@ -2483,7 +2483,7 @@ public class IndexWriter implements Clos
       TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
 
       SegmentInfo info = new SegmentInfo(directory, Version.LATEST, mergedName, -1,
-                                         false, codec, null, StringHelper.randomId(), new HashMap<>());
+                                         false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
       SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir,
                                                globalFieldNumberMap, 
@@ -3712,7 +3712,7 @@ public class IndexWriter implements Clos
     // ConcurrentMergePolicy we keep deterministic segment
     // names.
     final String mergeSegmentName = newSegmentName();
-    SegmentInfo si = new SegmentInfo(directory, Version.LATEST, mergeSegmentName, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    SegmentInfo si = new SegmentInfo(directory, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     Map<String,String> details = new HashMap<>();
     details.put("mergeMaxNumSegments", "" + merge.maxNumSegments);
     details.put("mergeFactor", Integer.toString(merge.segments.size()));

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java Sun Mar  1 20:30:49 2015
@@ -487,10 +487,8 @@ class ReadersAndUpdates {
         for (FieldInfo fi : reader.getFieldInfos()) {
           FieldInfo clone = builder.add(fi);
           // copy the stuff FieldInfos.Builder doesn't copy
-          if (fi.attributes() != null) {
-            for (Entry<String,String> e : fi.attributes().entrySet()) {
-              clone.putAttribute(e.getKey(), e.getValue());
-            }
+          for (Entry<String,String> e : fi.attributes().entrySet()) {
+            clone.putAttribute(e.getKey(), e.getValue());
           }
           clone.setDocValuesGen(fi.getDocValuesGen());
         }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java Sun Mar  1 20:30:49 2015
@@ -17,11 +17,9 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Objects;
@@ -68,7 +66,7 @@ public final class SegmentInfo {
 
   private Map<String,String> diagnostics;
   
-  private Map<String,String> attributes;
+  private final Map<String,String> attributes;
 
   // Tracks the Lucene version this segment was created with, since 3.1. Null
   // indicates an older than 3.0 index, and it's used to detect a too old index.
@@ -78,11 +76,11 @@ public final class SegmentInfo {
   private Version version;
 
   void setDiagnostics(Map<String, String> diagnostics) {
-    this.diagnostics = diagnostics;
+    this.diagnostics = Objects.requireNonNull(diagnostics);
   }
 
   /** Returns diagnostics saved into the segment when it was
-   *  written. */
+   *  written. The map is immutable. */
   public Map<String, String> getDiagnostics() {
     return diagnostics;
   }
@@ -96,13 +94,13 @@ public final class SegmentInfo {
                      boolean isCompoundFile, Codec codec, Map<String,String> diagnostics,
                      byte[] id, Map<String,String> attributes) {
     assert !(dir instanceof TrackingDirectoryWrapper);
-    this.dir = dir;
-    this.version = version;
-    this.name = name;
+    this.dir = Objects.requireNonNull(dir);
+    this.version = Objects.requireNonNull(version);
+    this.name = Objects.requireNonNull(name);
     this.docCount = docCount;
     this.isCompoundFile = isCompoundFile;
     this.codec = codec;
-    this.diagnostics = diagnostics;
+    this.diagnostics = Objects.requireNonNull(diagnostics);
     this.id = id;
     if (id.length != StringHelper.ID_LENGTH) {
       throw new IllegalArgumentException("invalid id: " + Arrays.toString(id));

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Sun Mar  1 20:30:49 2015
@@ -73,10 +73,10 @@ import org.apache.lucene.util.StringHelp
  * <li>HasSegID --&gt; {@link DataOutput#writeByte Int8}</li>
  * <li>SegID --&gt; {@link DataOutput#writeByte Int8<sup>ID_LENGTH</sup>}</li>
  * <li>SegName, SegCodec --&gt; {@link DataOutput#writeString String}</li>
- * <li>CommitUserData --&gt; {@link DataOutput#writeStringStringMap
+ * <li>CommitUserData --&gt; {@link DataOutput#writeMapOfStrings
  * Map&lt;String,String&gt;}</li>
  * <li>UpdatesFiles --&gt; Map&lt;{@link DataOutput#writeInt Int32},
- * {@link DataOutput#writeStringSet(Set) Set&lt;String&gt;}&gt;</li>
+ * {@link DataOutput#writeSetOfStrings(Set) Set&lt;String&gt;}&gt;</li>
  * <li>Footer --&gt; {@link CodecUtil#writeFooter CodecFooter}</li>
  * </ul>
  * Field Descriptions:
@@ -116,6 +116,9 @@ public final class SegmentInfos implemen
 
   /** The file format version for the segments_N codec header, since 5.0+ */
   public static final int VERSION_50 = 4;
+  /** The file format version for the segments_N codec header, since 5.1+ */
+  public static final int VERSION_51 = 5; // use safe maps
+  static final int VERSION_CURRENT = VERSION_51;
 
   /** Used to name new segments. */
   // TODO: should this be a long ...?
@@ -269,7 +272,7 @@ public final class SegmentInfos implemen
       if (magic != CodecUtil.CODEC_MAGIC) {
         throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
       }
-      CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_50);
+      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
       byte id[] = new byte[StringHelper.ID_LENGTH];
       input.readBytes(id, 0, id.length);
       CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
@@ -307,21 +310,34 @@ public final class SegmentInfos implemen
         long fieldInfosGen = input.readLong();
         long dvGen = input.readLong();
         SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
-        siPerCommit.setFieldInfosFiles(input.readStringSet());
+        if (format >= VERSION_51) {
+          siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
+        } else {
+          siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
+        }
         final Map<Integer,Set<String>> dvUpdateFiles;
         final int numDVFields = input.readInt();
         if (numDVFields == 0) {
           dvUpdateFiles = Collections.emptyMap();
         } else {
-          dvUpdateFiles = new HashMap<>(numDVFields);
+          Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
           for (int i = 0; i < numDVFields; i++) {
-            dvUpdateFiles.put(input.readInt(), input.readStringSet());
+            if (format >= VERSION_51) {
+              map.put(input.readInt(), input.readSetOfStrings());
+            } else {
+              map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
+            }
           }
+          dvUpdateFiles = Collections.unmodifiableMap(map);
         }
         siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
         infos.add(siPerCommit);
       }
-      infos.userData = input.readStringStringMap();
+      if (format >= VERSION_51) {
+        infos.userData = input.readMapOfStrings();
+      } else {
+        infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
+      }
 
       CodecUtil.checkFooter(input);
 
@@ -364,7 +380,7 @@ public final class SegmentInfos implemen
 
     try {
       segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
-      CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_50, 
+      CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_CURRENT, 
                                    StringHelper.randomId(), Long.toString(nextGeneration, Character.MAX_RADIX));
       segnOutput.writeLong(version); 
       segnOutput.writeInt(counter); // write counter
@@ -392,16 +408,16 @@ public final class SegmentInfos implemen
         segnOutput.writeInt(delCount);
         segnOutput.writeLong(siPerCommit.getFieldInfosGen());
         segnOutput.writeLong(siPerCommit.getDocValuesGen());
-        segnOutput.writeStringSet(siPerCommit.getFieldInfosFiles());
+        segnOutput.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
         final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
         segnOutput.writeInt(dvUpdatesFiles.size());
         for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
           segnOutput.writeInt(e.getKey());
-          segnOutput.writeStringSet(e.getValue());
+          segnOutput.writeSetOfStrings(e.getValue());
         }
         assert si.dir == directory;
       }
-      segnOutput.writeStringStringMap(userData);
+      segnOutput.writeMapOfStrings(userData);
       CodecUtil.writeFooter(segnOutput);
       segnOutput.close();
       directory.sync(Collections.singleton(segmentFileName));

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataInput.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataInput.java Sun Mar  1 20:30:49 2015
@@ -19,10 +19,13 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.lucene.util.BitUtil;
 
@@ -254,7 +257,11 @@ public abstract class DataInput implemen
   }
 
   /** Reads a Map&lt;String,String&gt; previously written
-   *  with {@link DataOutput#writeStringStringMap(Map)}. */
+   *  with {@link DataOutput#writeStringStringMap(Map)}. 
+   *  @deprecated Only for reading existing formats. Encode maps with 
+   *  {@link DataOutput#writeMapOfStrings(Map)} instead.
+   */
+  @Deprecated
   public Map<String,String> readStringStringMap() throws IOException {
     final Map<String,String> map = new HashMap<>();
     final int count = readInt();
@@ -266,9 +273,34 @@ public abstract class DataInput implemen
 
     return map;
   }
+  
+  /** 
+   * Reads a Map&lt;String,String&gt; previously written
+   * with {@link DataOutput#writeMapOfStrings(Map)}. 
+   * @return An immutable map containing the written contents.
+   */
+  public Map<String,String> readMapOfStrings() throws IOException {
+    int count = readVInt();
+    if (count == 0) {
+      return Collections.emptyMap();
+    } else if (count == 1) {
+      return Collections.singletonMap(readString(), readString());
+    } else {
+      Map<String,String> map = count > 10 ? new HashMap<>() : new TreeMap<>();
+      for (int i = 0; i < count; i++) {
+        final String key = readString();
+        final String val = readString();
+        map.put(key, val);
+      }
+      return Collections.unmodifiableMap(map);
+    }
+  }
 
   /** Reads a Set&lt;String&gt; previously written
-   *  with {@link DataOutput#writeStringSet(Set)}. */
+   *  with {@link DataOutput#writeStringSet(Set)}. 
+   *  @deprecated Only for reading existing formats. Encode maps with 
+   *  {@link DataOutput#writeSetOfStrings(Set)} instead. */
+  @Deprecated
   public Set<String> readStringSet() throws IOException {
     final Set<String> set = new HashSet<>();
     final int count = readInt();
@@ -278,6 +310,26 @@ public abstract class DataInput implemen
 
     return set;
   }
+  
+  /** 
+   * Reads a Set&lt;String&gt; previously written
+   * with {@link DataOutput#writeSetOfStrings(Set)}. 
+   * @return An immutable set containing the written contents.
+   */
+  public Set<String> readSetOfStrings() throws IOException {
+    int count = readVInt();
+    if (count == 0) {
+      return Collections.emptySet();
+    } else if (count == 1) {
+      return Collections.singleton(readString());
+    } else {
+      Set<String> set = count > 10 ? new HashSet<>() : new TreeSet<>();
+      for (int i = 0; i < count; i++) {
+        set.add(readString());
+      }
+      return Collections.unmodifiableSet(set);
+    }
+  }
 
   /**
    * Skip over <code>numBytes</code> bytes. The contract on this method is that it

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/DataOutput.java Sun Mar  1 20:30:49 2015
@@ -286,7 +286,9 @@ public abstract class DataOutput {
    * {@link #writeString(String) String}s.
    * 
    * @param map Input map. May be null (equivalent to an empty map)
+   * @deprecated Use {@link #writeMapOfStrings(Map)} instead.
    */
+  @Deprecated
   public void writeStringStringMap(Map<String,String> map) throws IOException {
     if (map == null) {
       writeInt(0);
@@ -298,6 +300,24 @@ public abstract class DataOutput {
       }
     }
   }
+  
+  /**
+   * Writes a String map.
+   * <p>
+   * First the size is written as an {@link #writeVInt(int) vInt},
+   * followed by each key-value pair written as two consecutive 
+   * {@link #writeString(String) String}s.
+   * 
+   * @param map Input map.
+   * @throws NullPointerException if {@code map} is null.
+   */
+  public void writeMapOfStrings(Map<String,String> map) throws IOException {
+    writeVInt(map.size());
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      writeString(entry.getKey());
+      writeString(entry.getValue());
+    }
+  }
 
   /**
    * Writes a String set.
@@ -307,7 +327,9 @@ public abstract class DataOutput {
    * {@link #writeString(String) String}.
    * 
    * @param set Input set. May be null (equivalent to an empty set)
+   * @deprecated Use {@link #writeMapOfStrings(Map)} instead.
    */
+  @Deprecated
   public void writeStringSet(Set<String> set) throws IOException {
     if (set == null) {
       writeInt(0);
@@ -318,4 +340,21 @@ public abstract class DataOutput {
       }
     }
   }
+  
+  /**
+   * Writes a String set.
+   * <p>
+   * First the size is written as an {@link #writeVInt(int) vInt},
+   * followed by each value written as a
+   * {@link #writeString(String) String}.
+   * 
+   * @param set Input set.
+   * @throws NullPointerException if {@code set} is null.
+   */
+  public void writeSetOfStrings(Set<String> set) throws IOException {
+    writeVInt(set.size());
+    for (String value : set) {
+      writeString(value);
+    }
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Sun Mar  1 20:30:49 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -221,7 +222,7 @@ public class TestCodecs extends LuceneTe
     final FieldInfos fieldInfos = builder.finish();
     final Directory dir = newDirectory();
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     
     this.write(si, fieldInfos, dir, fields);
     final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random())));
@@ -278,7 +279,7 @@ public class TestCodecs extends LuceneTe
     }
 
     Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     this.write(si, fieldInfos, dir, fields);
 
     if (VERBOSE) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Sun Mar  1 20:30:49 2015
@@ -28,6 +28,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -220,7 +221,7 @@ public class TestDoc extends LuceneTestC
 
     final Codec codec = Codec.getDefault();
     TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir);
-    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(r1, r2),
                                              si, InfoStream.getDefault(), trackingDir,

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Sun Mar  1 20:30:49 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 
 import org.apache.lucene.codecs.Codec;
@@ -82,7 +83,7 @@ public class TestSegmentMerger extends L
 
   public void testMerge() throws IOException {
     final Codec codec = Codec.getDefault();
-    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
     SegmentMerger merger = new SegmentMerger(Arrays.<CodecReader>asList(reader1, reader2),
                                              si, InfoStream.getDefault(), mergedDir,

Modified: lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java (original)
+++ lucene/dev/trunk/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java Sun Mar  1 20:30:49 2015
@@ -79,7 +79,7 @@ public class TermVectorLeafReader extend
     }
     FieldInfo fieldInfo = new FieldInfo(field, 0,
         true, true, terms.hasPayloads(),
-        indexOptions, DocValuesType.NONE, -1, null);
+        indexOptions, DocValuesType.NONE, -1, Collections.emptyMap());
     fieldInfos = new FieldInfos(new FieldInfo[]{fieldInfo});
   }
 

Modified: lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sun Mar  1 20:30:49 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.index.memory;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
@@ -443,7 +444,7 @@ public class MemoryIndex {
         fieldInfo = new FieldInfo(fieldName, fields.size(), false, false, this.storePayloads,
             this.storeOffsets
                 ? IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS : IndexOptions.DOCS_AND_FREQS_AND_POSITIONS,
-            DocValuesType.NONE, -1, null);
+            DocValuesType.NONE, -1, Collections.emptyMap());
         sliceArray = new SliceByteStartArray(BytesRefHash.DEFAULT_CAPACITY);
         terms = new BytesRefHash(byteBlockPool, BytesRefHash.DEFAULT_CAPACITY, sliceArray);
       }

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/UninvertingReader.java Sun Mar  1 20:30:49 2015
@@ -19,6 +19,7 @@ package org.apache.lucene.uninverting;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Map;
 
 import org.apache.lucene.document.BinaryDocValuesField; // javadocs
@@ -215,7 +216,7 @@ public class UninvertingReader extends F
         }
       }
       filteredInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
-                                      fi.hasPayloads(), fi.getIndexOptions(), type, -1, null));
+                                      fi.hasPayloads(), fi.getIndexOptions(), type, -1, Collections.emptyMap()));
     }
     fieldInfos = new FieldInfos(filteredInfos.toArray(new FieldInfo[filteredInfos.size()]));
   }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java Sun Mar  1 20:30:49 2015
@@ -648,7 +648,7 @@ public abstract class BaseCompoundFormat
   
   /** Returns a new fake segment */
   protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
-    return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), null, StringHelper.randomId(), new HashMap<>());
+    return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
   }
   
   /** Creates a file of the specified size with random data. */

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java Sun Mar  1 20:30:49 2015
@@ -18,8 +18,10 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
@@ -66,6 +68,32 @@ public abstract class BaseFieldInfoForma
     dir.close();
   }
   
+  /** Test field infos attributes coming back are not mutable */
+  public void testImmutableAttributes() throws Exception {
+    Directory dir = newDirectory();
+    Codec codec = getCodec();
+    SegmentInfo segmentInfo = newSegmentInfo(dir, "_123");
+    FieldInfos.Builder builder = new FieldInfos.Builder();
+    FieldInfo fi = builder.getOrAdd("field");
+    fi.setIndexOptions(TextField.TYPE_STORED.indexOptions());
+    addAttributes(fi);
+    fi.putAttribute("foo", "bar");
+    fi.putAttribute("bar", "baz");
+    FieldInfos infos = builder.finish();
+    codec.fieldInfosFormat().write(dir, segmentInfo, "", infos, IOContext.DEFAULT);
+    FieldInfos infos2 = codec.fieldInfosFormat().read(dir, segmentInfo, "", IOContext.DEFAULT);
+    assertEquals(1, infos2.size());
+    assertNotNull(infos2.fieldInfo("field"));
+    Map<String,String> attributes = infos2.fieldInfo("field").attributes();
+    try {
+      attributes.put("bogus", "bogus");
+      fail("shouldn't be able to modify attributes");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    dir.close();
+  }
+  
   // TODO: more tests
   
   /** Test field infos read/write with random fields, with different values. */
@@ -176,7 +204,7 @@ public abstract class BaseFieldInfoForma
   
   /** Returns a new fake segment */
   protected static SegmentInfo newSegmentInfo(Directory dir, String name) {
-    return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), null, StringHelper.randomId(), new HashMap<>());
+    return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java Sun Mar  1 20:30:49 2015
@@ -301,10 +301,10 @@ abstract class BaseIndexFileFormatTestCa
     Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors"));
     Codec codec = getCodec();
     
-    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
     FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field");
     FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(), 
-                                    proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), null);
+                                    proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>());
 
     FieldInfos fieldInfos = new FieldInfos(new FieldInfo[] { field } );
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Sun Mar  1 20:30:49 2015
@@ -372,7 +372,7 @@ public abstract class BasePostingsFormat
 
       fieldInfoArray[fieldUpto] = new FieldInfo(field, fieldUpto, false, false, true,
                                                 IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
-                                                DocValuesType.NONE, -1, null);
+                                                DocValuesType.NONE, -1, new HashMap<>());
       fieldUpto++;
 
       SortedMap<BytesRef,SeedAndOrd> postings = new TreeMap<>();
@@ -670,7 +670,7 @@ public abstract class BasePostingsFormat
   // randomly index at lower IndexOption
   private FieldsProducer buildIndex(Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException {
     Codec codec = getCodec();
-    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, null, StringHelper.randomId(), new HashMap<>());
+    SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>());
 
     int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed);
     if (VERBOSE) {
@@ -696,7 +696,7 @@ public abstract class BasePostingsFormat
                                                    indexOptions,
                                                    DocValuesType.NONE,
                                                    -1,
-                                                   null);
+                                                   new HashMap<>());
     }
 
     FieldInfos newFieldInfos = new FieldInfos(newFieldInfoArray);

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java Sun Mar  1 20:30:49 2015
@@ -74,6 +74,12 @@ public abstract class BaseSegmentInfoFor
     
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
     assertEquals(info.files(), info2.files());
+    try {
+      info2.files().add("bogus");
+      fail("files set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
     dir.close();
   }
   
@@ -91,6 +97,12 @@ public abstract class BaseSegmentInfoFor
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
     assertEquals(diagnostics, info2.getDiagnostics());
+    try {
+      info2.getDiagnostics().put("bogus", "bogus");
+      fail("diagnostics map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
     dir.close();
   }
   
@@ -108,6 +120,12 @@ public abstract class BaseSegmentInfoFor
     codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
     SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT);
     assertEquals(attributes, info2.getAttributes());
+    try {
+      info2.getAttributes().put("bogus", "bogus");
+      fail("attributes map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
     dir.close();
   }
   

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Sun Mar  1 20:30:49 2015
@@ -27,6 +27,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.zip.CRC32;
 
 import org.apache.lucene.index.DirectoryReader;
@@ -351,6 +352,93 @@ public abstract class BaseDirectoryTestC
     input.close();
     dir.close();
   }
+  
+  public void testSetOfStrings() throws Exception {
+    Directory dir = getDirectory(createTempDir("testSetOfStrings"));
+    
+    IndexOutput output = dir.createOutput("stringset", newIOContext(random()));
+    output.writeSetOfStrings(asSet("test1", "test2"));
+    output.writeSetOfStrings(Collections.emptySet());
+    output.writeSetOfStrings(asSet("test3"));
+    output.close();
+    
+    IndexInput input = dir.openInput("stringset", newIOContext(random()));
+    Set<String> set = input.readSetOfStrings();
+    assertEquals(asSet("test1", "test2"), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    set = input.readSetOfStrings();
+    assertEquals(Collections.emptySet(), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    set = input.readSetOfStrings();
+    assertEquals(Collections.singleton("test3"), set);
+    try {
+      set.add("bogus");
+      fail("set should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    assertEquals(input.length(), input.getFilePointer());
+    input.close();
+    dir.close();
+  }
+  
+  public void testMapOfStrings() throws Exception {
+    Map<String,String> m = new HashMap<>();
+    m.put("test1", "value1");
+    m.put("test2", "value2");
+    
+    Directory dir = getDirectory(createTempDir("testMapOfStrings"));
+    IndexOutput output = dir.createOutput("stringmap", newIOContext(random()));
+    output.writeMapOfStrings(m);
+    output.writeMapOfStrings(Collections.emptyMap());
+    output.writeMapOfStrings(Collections.singletonMap("key", "value"));
+    output.close();
+    
+    IndexInput input = dir.openInput("stringmap", newIOContext(random()));
+    Map<String,String> map = input.readMapOfStrings();
+    assertEquals(m, map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    map = input.readMapOfStrings();
+    assertEquals(Collections.emptyMap(), map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    map = input.readMapOfStrings();
+    assertEquals(Collections.singletonMap("key", "value"), map);
+    try {
+      map.put("bogus1", "bogus2");
+      fail("map should be immutable");
+    } catch (UnsupportedOperationException expected) {
+      // ok
+    }
+    
+    assertEquals(input.length(), input.getFilePointer());
+    input.close();
+    dir.close();
+  }
   
   // TODO: fold in some of the testing of o.a.l.index.TestIndexInput in here!
   public void testChecksum() throws Exception {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java?rev=1663157&r1=1663156&r2=1663157&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/Insanity.java Sun Mar  1 20:30:49 2015
@@ -66,7 +66,7 @@ public class Insanity {
       for (FieldInfo fi : in.getFieldInfos()) {
         if (fi.name.equals(insaneField)) {
           filteredInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
-                                          fi.hasPayloads(), fi.getIndexOptions(), DocValuesType.NONE, -1, null));
+                                          fi.hasPayloads(), fi.getIndexOptions(), DocValuesType.NONE, -1, Collections.emptyMap()));
         } else {
           filteredInfos.add(fi);
         }