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

svn commit: r1228693 [4/5] - in /lucene/dev/branches/solrcloud: ./ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/contrib/ dev-tools/idea/modules/benchmark/ dev-tools/maven/modules/benchmark/ lucene/ lucene/contrib/ lucene/contrib/highlighter/sr...

Modified: lucene/dev/branches/solrcloud/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/CHANGES.txt?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/CHANGES.txt (original)
+++ lucene/dev/branches/solrcloud/solr/CHANGES.txt Sat Jan  7 18:23:08 2012
@@ -217,12 +217,6 @@ Optimizations
 * SOLR-1904: When facet.enum.cache.minDf > 0 and the base doc set is a
   SortedIntSet, convert to HashDocSet for better performance. (yonik)
 
-* SOLR-1843: A new "rootName" attribute is now available when
-  configuring <jmx/> in solrconfig.xml.  If this attribute is set,
-  Solr will use it as the root name for all MBeans Solr exposes via
-  JMX.  The default root name is "solr" followed by the core name.
-  (Constantijn Visinescu, hossman)
-
 * SOLR-2092: Speed up single-valued and multi-valued "fc" faceting. Typical
   improvement is 5%, but can be much greater (up to 10x faster) when facet.offset
   is very large (deep paging). (yonik)
@@ -422,6 +416,19 @@ New Features
 * SOLR-2982: Added BeiderMorseFilterFactory for Beider-Morse (BMPM) phonetic encoder. Upgrades
   commons-codec to version 1.6  (Brooke Schreier Ganz, rmuir)
 
+* SOLR-1843: A new "rootName" attribute is now available when
+  configuring <jmx/> in solrconfig.xml.  If this attribute is set,
+  Solr will use it as the root name for all MBeans Solr exposes via
+  JMX.  The default root name is "solr" followed by the core name.
+  (Constantijn Visinescu, hossman)
+
+Optimizations
+----------------------
+* SOLR-1931 Speedup for LukeRequestHandler and admin/schema browser. New parameter
+  reportDocCount defaults to 'false'. Old behavior still possible by specifying this as 'true'
+  (Erick Erickson)
+
+
 Bug Fixes
 ----------------------
 * SOLR-2912: Fixed File descriptor leak in ShowFileRequestHandler (Michael Ryan, shalin)

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java Sat Jan  7 18:23:08 2012
@@ -20,33 +20,16 @@ package org.apache.solr.handler.admin;
 import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.*;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_ONLY;
 import static org.apache.lucene.index.FieldInfo.IndexOptions.DOCS_AND_FREQS;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.MultiFields;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermRangeQuery;
-import org.apache.lucene.search.TopDocs;
+
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
@@ -72,6 +55,7 @@ import org.apache.solr.schema.SchemaFiel
 import org.apache.solr.search.SolrIndexSearcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.lucene.index.DocsEnum;
 
 /**
  * This handler exposes the internal lucene index.  It is inspired by and 
@@ -82,33 +66,45 @@ import org.slf4j.LoggerFactory;
  * that it works nicely with an XSLT transformation.  Until we have a nice
  * XSLT front end for /admin, the format is still open to change.
  * </p>
- * 
+ *
  * For more documentation see:
  *  http://wiki.apache.org/solr/LukeRequestHandler
- * 
+ *
  *
  * @since solr 1.2
  */
-public class LukeRequestHandler extends RequestHandlerBase 
+public class LukeRequestHandler extends RequestHandlerBase
 {
   private static Logger log = LoggerFactory.getLogger(LukeRequestHandler.class);
-  
+
   public static final String NUMTERMS = "numTerms";
   public static final String DOC_ID = "docId";
   public static final String ID = "id";
   public static final int DEFAULT_COUNT = 10;
   
+  static final int HIST_ARRAY_SIZE = 33;
+
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
-  {    
+  {
     IndexSchema schema = req.getSchema();
     SolrIndexSearcher searcher = req.getSearcher();
     IndexReader reader = searcher.getIndexReader();
     SolrParams params = req.getParams();
     int numTerms = params.getInt( NUMTERMS, DEFAULT_COUNT );
-        
+
     // Always show the core lucene info
-    rsp.add("index", getIndexInfo(reader, numTerms>0 ) );
+    Map<String, TopTermQueue> topTerms = new TreeMap<String, TopTermQueue>();
+
+    // If no doc is given, show all fields and top terms
+    Set<String> fields = null;
+    if( params.get( CommonParams.FL ) != null ) {
+      fields = new TreeSet<String>(Arrays.asList(params.getParams( CommonParams.FL )));
+    }
+    if ( "schema".equals( params.get( "show" ))) {
+      numTerms = 0; // Abort any statistics gathering.
+    }
+    rsp.add("index", getIndexInfo(reader, numTerms, topTerms, fields ));
 
     Integer docId = params.getInt( DOC_ID );
     if( docId == null && params.get( ID ) != null ) {
@@ -121,7 +117,7 @@ public class LukeRequestHandler extends 
         throw new SolrException( SolrException.ErrorCode.NOT_FOUND, "Can't find document: "+params.get( ID ) );
       }
     }
-        
+
     // Read the document from the index
     if( docId != null ) {
       Document doc = null;
@@ -132,9 +128,9 @@ public class LukeRequestHandler extends 
       if( doc == null ) {
         throw new SolrException( SolrException.ErrorCode.NOT_FOUND, "Can't find document: "+docId );
       }
-      
+
       SimpleOrderedMap<Object> info = getDocumentFieldsInfo( doc, docId, reader, schema );
-      
+
       SimpleOrderedMap<Object> docinfo = new SimpleOrderedMap<Object>();
       docinfo.add( "docId", docId );
       docinfo.add( "lucene", info );
@@ -145,26 +141,18 @@ public class LukeRequestHandler extends 
       rsp.add( "schema", getSchemaInfo( req.getSchema() ) );
     }
     else {
-      // If no doc is given, show all fields and top terms
-      Set<String> fields = null;
-      if( params.get( CommonParams.FL ) != null ) {
-        fields = new HashSet<String>();
-        for( String f : params.getParams( CommonParams.FL ) ) {
-          fields.add( f );
-        }
-      }
-      rsp.add( "fields", getIndexedFieldsInfo( searcher, fields, numTerms ) ) ;
+      rsp.add( "fields", getIndexedFieldsInfo( searcher, fields, numTerms, topTerms) ) ;
     }
 
     // Add some generally helpful information
     NamedList<Object> info = new SimpleOrderedMap<Object>();
     info.add( "key", getFieldFlagsKey() );
-    info.add( "NOTE", "Document Frequency (df) is not updated when a document is marked for deletion.  df values include deleted documents." ); 
+    info.add( "NOTE", "Document Frequency (df) is not updated when a document is marked for deletion.  df values include deleted documents." );
     rsp.add( "info", info );
     rsp.setHttpCaching(false);
   }
 
-  
+
 
   /**
    * @return a string representing a IndexableField's flags.  
@@ -184,11 +172,11 @@ public class LukeRequestHandler extends 
     flags.append( (f != null && f.fieldType().storeTermVectorPositions()) ? FieldFlag.TERM_VECTOR_POSITION.getAbbreviation() : '-' );
     flags.append( (f != null && f.fieldType().omitNorms())                  ? FieldFlag.OMIT_NORMS.getAbbreviation() : '-' );
 
-    flags.append( (f != null && DOCS_ONLY == opts ) ? 
-                  FieldFlag.OMIT_TF.getAbbreviation() : '-' );
-    
-    flags.append( (f != null && DOCS_AND_FREQS == opts ) ?
-                  FieldFlag.OMIT_POSITIONS.getAbbreviation() : '-' );
+    flags.append( (f != null && DOCS_ONLY == opts ) ?
+        FieldFlag.OMIT_TF.getAbbreviation() : '-' );
+
+    flags.append((f != null && DOCS_AND_FREQS == opts) ?
+        FieldFlag.OMIT_POSITIONS.getAbbreviation() : '-');
 
     flags.append( (f != null && f.getClass().getSimpleName().equals("LazyField")) ? FieldFlag.LAZY.getAbbreviation() : '-' );
     flags.append( (f != null && f.binaryValue()!=null)                      ? FieldFlag.BINARY.getAbbreviation() : '-' );
@@ -196,18 +184,18 @@ public class LukeRequestHandler extends 
     flags.append( (false)                                          ? FieldFlag.SORT_MISSING_LAST.getAbbreviation() : '-' ); // SchemaField Specific
     return flags.toString();
   }
-  
+
   /**
    * @return a string representing a SchemaField's flags.  
    */
   private static String getFieldFlags( SchemaField f )
   {
     FieldType t = (f==null) ? null : f.getType();
-    
+
     // see: http://www.nabble.com/schema-field-properties-tf3437753.html#a9585549
     boolean lazy = false; // "lazy" is purely a property of reading fields
     boolean binary = false; // Currently not possible
-    
+
     StringBuilder flags = new StringBuilder();
     flags.append( (f != null && f.indexed())             ? FieldFlag.INDEXED.getAbbreviation() : '-' );
     flags.append( (t != null && t.isTokenized())         ? FieldFlag.TOKENIZED.getAbbreviation() : '-' );
@@ -217,8 +205,8 @@ public class LukeRequestHandler extends 
     flags.append( (f != null && f.storeTermOffsets() )   ? FieldFlag.TERM_VECTOR_OFFSET.getAbbreviation() : '-' );
     flags.append( (f != null && f.storeTermPositions() ) ? FieldFlag.TERM_VECTOR_POSITION.getAbbreviation() : '-' );
     flags.append( (f != null && f.omitNorms())           ? FieldFlag.OMIT_NORMS.getAbbreviation() : '-' );
-    flags.append( (f != null && 
-                   f.omitTermFreqAndPositions() )        ? FieldFlag.OMIT_TF.getAbbreviation() : '-' );
+    flags.append( (f != null &&
+        f.omitTermFreqAndPositions() )        ? FieldFlag.OMIT_TF.getAbbreviation() : '-' );
     flags.append( (f != null && f.omitPositions() )      ? FieldFlag.OMIT_POSITIONS.getAbbreviation() : '-' );
     flags.append( (lazy)                                 ? FieldFlag.LAZY.getAbbreviation() : '-' );
     flags.append( (binary)                               ? FieldFlag.BINARY.getAbbreviation() : '-' );
@@ -226,7 +214,7 @@ public class LukeRequestHandler extends 
     flags.append( (f != null && f.sortMissingLast() )    ? FieldFlag.SORT_MISSING_LAST.getAbbreviation() : '-' );
     return flags.toString();
   }
-  
+
   /**
    * @return a key to what each character means
    */
@@ -237,15 +225,15 @@ public class LukeRequestHandler extends 
     }
     return key;
   }
-  
+
   private static SimpleOrderedMap<Object> getDocumentFieldsInfo( Document doc, int docId, IndexReader reader, IndexSchema schema ) throws IOException
-  { 
+  {
     final CharsRef spare = new CharsRef();
     SimpleOrderedMap<Object> finfo = new SimpleOrderedMap<Object>();
     for( Object o : doc.getFields() ) {
       Field field = (Field)o;
       SimpleOrderedMap<Object> f = new SimpleOrderedMap<Object>();
-      
+
       SchemaField sfield = schema.getFieldOrNull( field.name() );
       FieldType ftype = (sfield==null)?null:sfield.getType();
 
@@ -266,7 +254,7 @@ public class LukeRequestHandler extends 
       }
       f.add( "boost", field.boost() );
       f.add( "docFreq", t.text()==null ? 0 : reader.docFreq( t ) ); // this can be 0 for non-indexed fields
-            
+
       // If we have a term vector, return that
       if( field.fieldType().storeTermVectors() ) {
         try {
@@ -287,101 +275,118 @@ public class LukeRequestHandler extends 
           log.warn( "error writing term vector", ex );
         }
       }
-      
+
       finfo.add( field.name(), f );
     }
     return finfo;
   }
-  
+
   @SuppressWarnings("unchecked")
-  private static SimpleOrderedMap<Object> getIndexedFieldsInfo( 
-    final SolrIndexSearcher searcher, final Set<String> fields, final int numTerms ) 
-    throws Exception {
+  private static SimpleOrderedMap<Object> getIndexedFieldsInfo(
+      final SolrIndexSearcher searcher, final Set<String> fields, final int numTerms, Map<String,TopTermQueue> ttinfo)
+      throws Exception {
 
     IndexReader reader = searcher.getIndexReader();
     IndexSchema schema = searcher.getSchema();
-    
+
     // Walk the term enum and keep a priority queue for each map in our set
-    Map<String,TopTermQueue> ttinfo = null;
-    if( numTerms > 0 ) {
-      ttinfo = getTopTerms(reader, fields, numTerms, null );
-    }
     SimpleOrderedMap<Object> finfo = new SimpleOrderedMap<Object>();
-    Collection<String> fieldNames = reader.getFieldNames(IndexReader.FieldOption.ALL);
+    Fields theFields = MultiFields.getFields(reader);
+
+    Set<String> fieldNames = new TreeSet<String>(reader.getFieldNames(IndexReader.FieldOption.ALL));
     for (String fieldName : fieldNames) {
-      if( fields != null && !fields.contains( fieldName ) ) {
-        continue; // if a field is specified, only them
+      if (fields != null && ! fields.contains(fieldName)) {
+        continue; // we're not interested in this term
       }
-      
+
       SimpleOrderedMap<Object> f = new SimpleOrderedMap<Object>();
-      
+
       SchemaField sfield = schema.getFieldOrNull( fieldName );
       FieldType ftype = (sfield==null)?null:sfield.getType();
 
       f.add( "type", (ftype==null)?null:ftype.getTypeName() );
       f.add( "schema", getFieldFlags( sfield ) );
       if (sfield != null && schema.isDynamicField(sfield.getName()) && schema.getDynamicPattern(sfield.getName()) != null) {
-    	  f.add("dynamicBase", schema.getDynamicPattern(sfield.getName()));
+        f.add("dynamicBase", schema.getDynamicPattern(sfield.getName()));
       }
 
+      Terms terms = theFields.terms(fieldName);
+      if (terms == null) { // Not indexed, so we need to report what we can (it made it through the fl param if specified)
+        finfo.add( fieldName, f );
+        continue;
+      }
+
+      TopTermQueue topTerms = ttinfo.get( fieldName );
       // If numTerms==0, the call is just asking for a quick field list
       if( ttinfo != null && sfield != null && sfield.indexed() ) {
-        Query q = new TermRangeQuery(fieldName,null,null,false,false); 
-        TopDocs top = searcher.search( q, 1 );
-        if( top.totalHits > 0 ) {
-          // Find a document with this field
-          try {
-            Document doc = searcher.doc( top.scoreDocs[0].doc );
-            IndexableField fld = doc.getField( fieldName );
-            if( fld != null ) {
-              f.add( "index", getFieldFlags( fld ) );
-            }
-            else {
-              // it is a non-stored field...
-              f.add( "index", "(unstored field)" );
+        if (numTerms > 0) { // Read the actual field from the index and report that too.
+          Document doc = null;
+          if (topTerms != null && topTerms.getTopTermInfo() != null) {
+            Term term = topTerms.getTopTermInfo().term;
+            DocsEnum docsEnum = MultiFields.getTermDocsEnum(reader,
+                MultiFields.getLiveDocs(reader),
+                term.field(),
+                new BytesRef(term.text()),
+                false);
+            if (docsEnum != null) {
+              int docId;
+              if ((docId = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+                doc = reader.document(docId);
+              }
             }
           }
-          catch( Exception ex ) {
-            log.warn( "error reading field: "+fieldName );
+          if( doc != null ) {
+            // Found a document with this field
+            try {
+              IndexableField fld = doc.getField( fieldName );
+              if( fld != null ) {
+                f.add( "index", getFieldFlags( fld ) );
+              }
+              else {
+                // it is a non-stored field...
+                f.add( "index", "(unstored field)" );
+              }
+            }
+            catch( Exception ex ) {
+              log.warn( "error reading field: "+fieldName );
+            }
           }
+          f.add("docs", terms.getDocCount());
         }
-        f.add( "docs", top.totalHits );
-        
-        TopTermQueue topTerms = ttinfo.get( fieldName );
         if( topTerms != null ) {
           f.add( "distinct", topTerms.distinctTerms );
-          
+
           // Include top terms
           f.add( "topTerms", topTerms.toNamedList( searcher.getSchema() ) );
-  
+
           // Add a histogram
           f.add( "histogram", topTerms.histogram.toNamedList() );
         }
       }
-      
       // Add the field
       finfo.add( fieldName, f );
     }
     return finfo;
   }
-    
+
   /**
    * Return info from the index
    */
   private static SimpleOrderedMap<Object> getSchemaInfo( IndexSchema schema ) {
-    Map<String, List<String>> typeusemap = new HashMap<String, List<String>>();
-    SimpleOrderedMap<Object> fields = new SimpleOrderedMap<Object>();
+    Map<String, List<String>> typeusemap = new TreeMap<String, List<String>>();
+    Map<String, Object> fields = new TreeMap<String, Object>();
     SchemaField uniqueField = schema.getUniqueKeyField();
     for( SchemaField f : schema.getFields().values() ) {
       populateFieldInfo(schema, typeusemap, fields, uniqueField, f);
     }
-    
-    SimpleOrderedMap<Object> dynamicFields = new SimpleOrderedMap<Object>();
+
+    Map<String, Object> dynamicFields = new TreeMap<String, Object>();
     for (SchemaField f : schema.getDynamicFieldPrototypes()) {
-    	populateFieldInfo(schema, typeusemap, dynamicFields, uniqueField, f);
+      populateFieldInfo(schema, typeusemap, dynamicFields, uniqueField, f);
     }
     SimpleOrderedMap<Object> types = new SimpleOrderedMap<Object>();
-    for( FieldType ft : schema.getFieldTypes().values() ) {
+    Map<String, FieldType> sortedTypes = new TreeMap<String, FieldType>(schema.getFieldTypes());
+    for( FieldType ft : sortedTypes.values() ) {
       SimpleOrderedMap<Object> field = new SimpleOrderedMap<Object>();
       field.add("fields", typeusemap.get( ft.getTypeName() ) );
       field.add("tokenized", ft.isTokenized() );
@@ -391,23 +396,35 @@ public class LukeRequestHandler extends 
       types.add( ft.getTypeName(), field );
     }
 
+    // Must go through this to maintain binary compatbility. Putting a TreeMap into a resp leads to casting errors
     SimpleOrderedMap<Object> finfo = new SimpleOrderedMap<Object>();
-    finfo.add("fields", fields);
-    finfo.add("dynamicFields", dynamicFields);
-    finfo.add("uniqueKeyField", 
-              null == uniqueField ? null : uniqueField.getName());
+
+    SimpleOrderedMap<Object> fieldsSimple = new SimpleOrderedMap<Object>();
+    for (Map.Entry<String, Object> ent : fields.entrySet()) {
+      fieldsSimple.add(ent.getKey(), ent.getValue());
+    }
+    finfo.add("fields", fieldsSimple);
+
+    SimpleOrderedMap<Object> dynamicSimple = new SimpleOrderedMap<Object>();
+    for (Map.Entry<String, Object> ent : dynamicFields.entrySet()) {
+      dynamicSimple.add(ent.getKey(), ent.getValue());
+    }
+    finfo.add("dynamicFields", dynamicSimple);
+
+    finfo.add("uniqueKeyField",
+        null == uniqueField ? null : uniqueField.getName());
     finfo.add("defaultSearchField", schema.getDefaultSearchFieldName());
     finfo.add("types", types);
     return finfo;
   }
 
-  
+
   private static SimpleOrderedMap<Object> getAnalyzerInfo(Analyzer analyzer) {
-	  SimpleOrderedMap<Object> aninfo = new SimpleOrderedMap<Object>();
-	  aninfo.add("className", analyzer.getClass().getName());
-	  if (analyzer instanceof TokenizerChain) {
+    SimpleOrderedMap<Object> aninfo = new SimpleOrderedMap<Object>();
+    aninfo.add("className", analyzer.getClass().getName());
+    if (analyzer instanceof TokenizerChain) {
 
-	    TokenizerChain tchain = (TokenizerChain)analyzer;
+      TokenizerChain tchain = (TokenizerChain)analyzer;
 
       CharFilterFactory[] cfiltfacs = tchain.getCharFilterFactories();
       SimpleOrderedMap<Map<String, Object>> cfilters = new SimpleOrderedMap<Map<String, Object>>();
@@ -421,7 +438,7 @@ public class LukeRequestHandler extends 
       if (cfilters.size() > 0) {
         aninfo.add("charFilters", cfilters);
       }
-      
+
       SimpleOrderedMap<Object> tokenizer = new SimpleOrderedMap<Object>();
       TokenizerFactory tfac = tchain.getTokenizerFactory();
       tokenizer.add("className", tfac.getClass().getName());
@@ -440,68 +457,102 @@ public class LukeRequestHandler extends 
       if (filters.size() > 0) {
         aninfo.add("filters", filters);
       }
-	  }
-	  return aninfo;
+    }
+    return aninfo;
   }
 
   private static void populateFieldInfo(IndexSchema schema,
-		Map<String, List<String>> typeusemap, SimpleOrderedMap<Object> fields,
-		SchemaField uniqueField, SchemaField f) {
-      FieldType ft = f.getType();
-      SimpleOrderedMap<Object> field = new SimpleOrderedMap<Object>();
-      field.add( "type", ft.getTypeName() );
-      field.add( "flags", getFieldFlags(f) );
-      if( f.isRequired() ) {
-        field.add( "required", f.isRequired() );
-      }
-      if( f.getDefaultValue() != null ) {
-        field.add( "default", f.getDefaultValue() );
-      }
-      if (f == uniqueField){
-        field.add("uniqueKey", true);
-      }
-      if (ft.getAnalyzer().getPositionIncrementGap(f.getName()) != 0) {
-    	  field.add("positionIncrementGap", ft.getAnalyzer().getPositionIncrementGap(f.getName()));
-      }
-      field.add("copyDests", schema.getCopyFieldsList(f.getName()));
-      field.add("copySources", schema.getCopySources(f.getName()));
-
-      
-      fields.add( f.getName(), field );
-      
-      List<String> v = typeusemap.get( ft.getTypeName() );
-      if( v == null ) {
-        v = new ArrayList<String>();
-      }
-      v.add( f.getName() );
-      typeusemap.put( ft.getTypeName(), v );
+                                        Map<String, List<String>> typeusemap, Map<String, Object> fields,
+                                        SchemaField uniqueField, SchemaField f) {
+    FieldType ft = f.getType();
+    SimpleOrderedMap<Object> field = new SimpleOrderedMap<Object>();
+    field.add( "type", ft.getTypeName() );
+    field.add( "flags", getFieldFlags(f) );
+    if( f.isRequired() ) {
+      field.add( "required", f.isRequired() );
+    }
+    if( f.getDefaultValue() != null ) {
+      field.add( "default", f.getDefaultValue() );
+    }
+    if (f == uniqueField){
+      field.add("uniqueKey", true);
+    }
+    if (ft.getAnalyzer().getPositionIncrementGap(f.getName()) != 0) {
+      field.add("positionIncrementGap", ft.getAnalyzer().getPositionIncrementGap(f.getName()));
+    }
+    field.add("copyDests", schema.getCopyFieldsList(f.getName()));
+    field.add("copySources", schema.getCopySources(f.getName()));
+
+
+    fields.put( f.getName(), field );
+
+    List<String> v = typeusemap.get( ft.getTypeName() );
+    if( v == null ) {
+      v = new ArrayList<String>();
+    }
+    v.add( f.getName() );
+    typeusemap.put( ft.getTypeName(), v );
   }
-  
-  public static SimpleOrderedMap<Object> getIndexInfo( IndexReader reader, boolean countTerms ) throws IOException {
+  public static SimpleOrderedMap<Object> getIndexInfo(IndexReader reader, boolean countTerms) throws IOException {
+    return getIndexInfo(reader, countTerms ? 1 : 0, null, null);
+  }
+  public static SimpleOrderedMap<Object> getIndexInfo( IndexReader reader, int numTerms,
+                                                       Map<String, TopTermQueue> topTerms,
+                                                       Set<String> fieldList) throws IOException {
     Directory dir = reader.directory();
     SimpleOrderedMap<Object> indexInfo = new SimpleOrderedMap<Object>();
-    
+
     indexInfo.add("numDocs", reader.numDocs());
     indexInfo.add("maxDoc", reader.maxDoc());
-    
-    if( countTerms ) {
+    final CharsRef spare = new CharsRef();
+    if( numTerms > 0 ) {
       Fields fields = MultiFields.getFields(reader);
-      int numTerms = 0;
+      long totalTerms = 0;
       if (fields != null) {
         FieldsEnum fieldsEnum = fields.iterator();
-        while(fieldsEnum.next() != null) {
+        String field;
+        while ((field = fieldsEnum.next()) != null) {
           Terms terms = fieldsEnum.terms();
-          if (terms != null) {
-            TermsEnum termsEnum = terms.iterator(null);
-            while(termsEnum.next() != null) {
-              numTerms++;
+          if (terms == null) {
+            continue;
+          }
+          totalTerms += terms.getUniqueTermCount();
+
+          if (fieldList != null && !fieldList.contains(field)) {
+            continue;
+          }
+
+          TermsEnum termsEnum = terms.iterator(null);
+          BytesRef text;
+          int[] buckets = new int[HIST_ARRAY_SIZE];
+          TopTermQueue tiq = topTerms.get(field);
+          if (tiq == null) {
+            tiq = new TopTermQueue(numTerms + 1);   // Allocating slots for the top N terms to collect freqs.
+            topTerms.put(field, tiq);
+          }
+          while ((text = termsEnum.next()) != null) {
+            int freq = termsEnum.docFreq();  // This calculation seems odd, but it gives the same results as it used to.
+            int slot = 32 - Integer.numberOfLeadingZeros(Math.max(0, freq - 1));
+            buckets[slot] = buckets[slot] + 1;
+            if (freq > tiq.minFreq) {
+              UnicodeUtil.UTF8toUTF16(text, spare);
+              String t = spare.toString();
+              tiq.distinctTerms = new Long(fieldsEnum.terms().getUniqueTermCount()).intValue();
+
+              tiq.add(new TopTermQueue.TermInfo(new Term(field, t), termsEnum.docFreq()));
+              if (tiq.size() > numTerms) { // if tiq full
+                tiq.pop(); // remove lowest in tiq
+                tiq.minFreq  = tiq.getTopTermInfo().docFreq;
+              }
             }
           }
+          tiq.histogram.add(buckets);
         }
       }
-      indexInfo.add("numTerms", numTerms );
-    }
+      //Clumsy, but I'm tired.
+      indexInfo.add("numTerms", (new Long(totalTerms)).intValue());
 
+    }
     indexInfo.add("version", reader.getVersion());  // TODO? Is this different then: IndexReader.getCurrentVersion( dir )?
     indexInfo.add("segmentCount", reader.getSequentialSubReaders().length);
     indexInfo.add("current", reader.isCurrent() );
@@ -510,7 +561,6 @@ public class LukeRequestHandler extends 
     indexInfo.add("lastModified", new Date(IndexReader.lastModified(dir)) );
     return indexInfo;
   }
-  
   //////////////////////// SolrInfoMBeans methods //////////////////////
 
   @Override
@@ -542,51 +592,33 @@ public class LukeRequestHandler extends 
   }
 
   ///////////////////////////////////////////////////////////////////////////////////////
-  
-  static class TermHistogram 
+
+  static class TermHistogram
   {
-    int maxBucket = -1;
-    public Map<Integer,Integer> hist = new HashMap<Integer, Integer>();
-    
-    public static int getPowerOfTwoBucket( int num )
-    {
-      return Math.max(1, Integer.highestOneBit(num-1) << 1);
-    }
-    
-    public void add( int df )
-    {
-      Integer bucket = getPowerOfTwoBucket( df );
-      if( bucket > maxBucket ) {
-        maxBucket = bucket;
-      }
-      Integer old = hist.get( bucket );
-      if( old == null ) {
-        hist.put( bucket, 1 );
+    int _maxBucket = -1;
+    int _buckets[] = new int[HIST_ARRAY_SIZE];
+    public void add(int[] buckets) {
+      for (int idx = 0; idx < buckets.length; ++idx) {
+        if (buckets[idx] != 0) _maxBucket = idx;
       }
-      else {
-        hist.put( bucket, old+1 );
+      for (int idx = 0; idx <= _maxBucket; ++idx) {
+        _buckets[idx] = buckets[idx];
       }
     }
-    
     // TODO? should this be a list or a map?
     public NamedList<Integer> toNamedList()
     {
       NamedList<Integer> nl = new NamedList<Integer>();
-      for( int bucket = 1; bucket <= maxBucket; bucket *= 2 ) {
-        Integer val = hist.get( bucket );
-        if( val == null ) {
-          val = 0;
-        }
-        nl.add( ""+bucket, val );
+      for( int bucket = 0; bucket <= _maxBucket; bucket++ ) {
+        nl.add( ""+ (1 << bucket), _buckets[bucket] );
       }
       return nl;
     }
   }
-  
   /**
    * Private internal class that counts up frequent terms
    */
-  private static class TopTermQueue extends PriorityQueue 
+  private static class TopTermQueue extends PriorityQueue
   {
     static class TermInfo {
       TermInfo(Term t, int df) {
@@ -596,23 +628,24 @@ public class LukeRequestHandler extends 
       int docFreq;
       Term term;
     }
-    
+
     public int minFreq = 0;
     public int distinctTerms = 0;
     public TermHistogram histogram;
-    
+
+
     TopTermQueue(int size) {
       super(size);
       histogram = new TermHistogram();
     }
-    
+
     @Override
     protected final boolean lessThan(Object a, Object b) {
       TermInfo termInfoA = (TermInfo)a;
       TermInfo termInfoB = (TermInfo)b;
       return termInfoA.docFreq < termInfoB.docFreq;
     }
-    
+
     /**
      * This is a destructive call... the queue is empty at the end
      */
@@ -623,7 +656,7 @@ public class LukeRequestHandler extends 
       while( size() > 0 ) {
         aslist.add( 0, (TermInfo)pop() );
       }
-      
+
       NamedList<Integer> list = new NamedList<Integer>();
       for (TermInfo i : aslist) {
         String txt = i.term.text();
@@ -635,65 +668,8 @@ public class LukeRequestHandler extends 
       }
       return list;
     }
-  }
-
-  private static Map<String,TopTermQueue> getTopTerms( IndexReader reader, Set<String> fields, int numTerms, Set<String> junkWords ) throws Exception 
-  {
-    Map<String,TopTermQueue> info = new HashMap<String, TopTermQueue>();
-    final CharsRef spare = new CharsRef();
-    Fields fieldsC = MultiFields.getFields(reader);
-    if (fieldsC != null) {
-      FieldsEnum fieldsEnum = fieldsC.iterator();
-      String field;
-      while((field = fieldsEnum.next()) != null) {
-
-        Terms terms = fieldsEnum.terms();
-        if (terms == null) {
-          continue;
-        }
-        TermsEnum termsEnum = terms.iterator(null);
-        BytesRef text;
-        while((text = termsEnum.next()) != null) {
-          UnicodeUtil.UTF8toUTF16(text, spare);
-          String t = spare.toString();
-  
-          // Compute distinct terms for every field
-          TopTermQueue tiq = info.get( field );
-          if( tiq == null ) {
-            tiq = new TopTermQueue( numTerms+1 );
-            info.put( field, tiq );
-          }
-
-          tiq.distinctTerms++;
-          tiq.histogram.add( termsEnum.docFreq() );  // add the term to the histogram
-        
-          // Only save the distinct terms for fields we worry about
-          if (fields != null && fields.size() > 0) {
-            if( !fields.contains( field ) ) {
-              continue;
-            }
-          }
-          if( junkWords != null && junkWords.contains( t ) ) {
-            continue;
-          }
-        
-          if( termsEnum.docFreq() > tiq.minFreq ) {
-            tiq.add(new TopTermQueue.TermInfo(new Term(field, t), termsEnum.docFreq()));
-            if (tiq.size() > numTerms) { // if tiq full
-              tiq.pop(); // remove lowest in tiq
-              tiq.minFreq = ((TopTermQueue.TermInfo)tiq.top()).docFreq; // reset minFreq
-            }
-          }
-        }
-      }
+    public TermInfo getTopTermInfo() {
+      return (TermInfo)top();
     }
-    return info;
   }
 }
-
-
-
-
-
-
-

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/MinimalSchemaTest.java Sat Jan  7 18:23:08 2012
@@ -89,7 +89,6 @@ public class MinimalSchemaTest extends S
             req("qt", "/admin/luke",
                 "show","schema")
             ,"//int[@name='numDocs'][.='2']"
-            ,"//int[@name='numTerms'][.='5']"
             ,"//null[@name='uniqueKeyField']"
             ,"//null[@name='defaultSearchField']"
             );

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/analysis/TestSynonymFilterFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/analysis/TestSynonymFilterFactory.java?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/analysis/TestSynonymFilterFactory.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/analysis/TestSynonymFilterFactory.java Sat Jan  7 18:23:08 2012
@@ -21,6 +21,7 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.StringReader;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -65,6 +66,25 @@ public class TestSynonymFilterFactory ex
         new int[] { 1, 0, 0, 0 });
   }
   
+  /** test multiword offsets with the old impl
+   * @deprecated Remove this test in Lucene 5.0 */
+  @Deprecated
+  public void testMultiwordOffsetsOld() throws Exception {
+    SynonymFilterFactory factory = new SynonymFilterFactory();
+    Map<String,String> args = new HashMap<String,String>();
+    args.put("luceneMatchVersion", Version.LUCENE_33.toString());
+    args.put("synonyms", "synonyms.txt");
+    factory.init(args);
+    factory.inform(new StringMockSolrResourceLoader("national hockey league, nhl"));
+    TokenStream ts = factory.create(new MockTokenizer(new StringReader("national hockey league"), MockTokenizer.WHITESPACE, false));
+    // WTF?
+    assertTokenStreamContents(ts, 
+        new String[] { "national", "nhl", "hockey", "league" },
+        new int[] { 0, 0, 0, 0 },
+        new int[] { 22, 22, 22, 22 },
+        new int[] { 1, 0, 1, 1 });
+  }
+  
   /** if the synonyms are completely empty, test that we still analyze correctly */
   public void testEmptySynonyms() throws Exception {
     SynonymFilterFactory factory = new SynonymFilterFactory();
@@ -85,7 +105,7 @@ public class TestSynonymFilterFactory ex
     }
 
     public List<String> getLines(String resource) throws IOException {
-      return null;
+      return Arrays.asList(text.split("\n"));
     }
 
     public Object newInstance(String cname, String... subpackages) {

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java Sat Jan  7 18:23:08 2012
@@ -60,7 +60,6 @@ import org.junit.BeforeClass;
  *
  * @since 1.4
  */
-@LuceneTestCase.Nightly
 // TODO: can this test be sped up? it used to not be so slow...
 public class TestReplicationHandler extends SolrTestCaseJ4 {
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/handler/admin/LukeRequestHandlerTest.java Sat Jan  7 18:23:08 2012
@@ -19,6 +19,7 @@ package org.apache.solr.handler.admin;
 
 import org.apache.solr.common.luke.FieldFlag;
 import org.apache.solr.util.AbstractSolrTestCase;
+import org.junit.Test;
 
 import java.util.EnumSet;
 import java.util.Arrays;
@@ -38,33 +39,31 @@ public class LukeRequestHandlerTest exte
     return "solrconfig.xml";
   }
 
-  /** tests some simple edge cases */
-  public void doTestHistogramPowerOfTwoBucket() {
-    assertHistoBucket(1,  1);
-    assertHistoBucket(2,  2);
-    assertHistoBucket(4,  3);
-    assertHistoBucket(4,  4);
-    assertHistoBucket(8,  5);
-    assertHistoBucket(8,  6);
-    assertHistoBucket(8,  7);
-    assertHistoBucket(8,  8);
-    assertHistoBucket(16, 9);
+  public void testHistogramBucket() {
+    assertHistoBucket(0, 1);
+    assertHistoBucket(1, 2);
+    assertHistoBucket(2, 3);
+    assertHistoBucket(2, 4);
+    assertHistoBucket(3, 5);
+    assertHistoBucket(3, 6);
+    assertHistoBucket(3, 7);
+    assertHistoBucket(3, 8);
+    assertHistoBucket(4, 9);
 
     final int MAX_VALID = ((Integer.MAX_VALUE/2)+1)/2;
-    
-    assertHistoBucket(MAX_VALID,   MAX_VALID-1 );
-    assertHistoBucket(MAX_VALID,   MAX_VALID   );
-    assertHistoBucket(MAX_VALID*2, MAX_VALID+1 );
-    
+
+    assertHistoBucket(29,   MAX_VALID-1 );
+    assertHistoBucket(29,   MAX_VALID   );
+    assertHistoBucket(30, MAX_VALID+1 );
+
   }
 
-  private void assertHistoBucket(int expected, int in) {
-    assertEquals("histobucket: " + in, expected,
-                 LukeRequestHandler.TermHistogram.getPowerOfTwoBucket( in ));
+  private void assertHistoBucket(int slot, int in) {
+    assertEquals("histobucket: " + in, slot, 32 - Integer.numberOfLeadingZeros(Math.max(0, in - 1)));
   }
 
+  @Test
   public void testLuke() {
-    doTestHistogramPowerOfTwoBucket();
 
     assertU(adoc("id","SOLR1000", "name","Apache Solr",
       "solr_si", "10",
@@ -99,7 +98,7 @@ public class LukeRequestHandlerTest exte
     assertQ(req("qt","/admin/luke", "id","SOLR1000"));
 
     final int numFlags = EnumSet.allOf(FieldFlag.class).size();
-    
+
     assertQ("Not all flags ("+numFlags+") mentioned in info->key",
             req("qt","/admin/luke"),
             numFlags+"=count(//lst[@name='info']/lst[@name='key']/str)");
@@ -116,7 +115,7 @@ public class LukeRequestHandlerTest exte
 
     }
 
-    // diff loop for checking 'index' flags, 
+    // diff loop for checking 'index' flags,
     // only valid for fields that are indexed & stored
     for (String f : Arrays.asList("solr_t","solr_s","solr_ti",
                                   "solr_td","solr_pl","solr_dt","solr_b")) {
@@ -125,10 +124,17 @@ public class LukeRequestHandlerTest exte
       assertQ("Not as many index flags as expected ("+numFlags+") for " + f,
               req("qt","/admin/luke", "fl", f),
               numFlags+"=string-length("+xp+"[@name='index'])");
-    }
 
+    final String hxp = getFieldXPathHistogram(f);
+    assertQ("Historgram field should be present for field "+f,
+        req("qt", "/admin/luke", "fl", f),
+        hxp+"[@name='histogram']");
+    }
   }
 
+  private static String getFieldXPathHistogram(String field) {
+    return "//lst[@name='fields']/lst[@name='"+field+"']/lst";
+  }
   private static String getFieldXPathPrefix(String field) {
     return "//lst[@name='fields']/lst[@name='"+field+"']/str";
   }

Modified: lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/site.xml?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/site.xml (original)
+++ lucene/dev/branches/solrcloud/solr/site-src/src/documentation/content/xdocs/site.xml Sat Jan  7 18:23:08 2012
@@ -39,6 +39,7 @@ See http://forrest.apache.org/docs/linki
   <about label="About">
     <index label="Welcome" href="index.html" description="Welcome to Solr"/>
     <index label="Who We Are" href="ext:lucene_who" description="Lucene/Solr Committers"/>
+    <privacy label="Privacy Policy" href="ext:privacy"/>
   </about>
 
   <docs label="Documentation">
@@ -81,7 +82,7 @@ See http://forrest.apache.org/docs/linki
     </forrest>
     <cocoon href="http://cocoon.apache.org/"/>
     <xml.apache.org href="http://xml.apache.org/"/>
-
+    <privacy href="http://lucene.apache.org/privacy.html"/>
     <lucene      href="http://lucene.apache.org/java/" />
     <lucene_who  href="http://lucene.apache.org/java/docs/whoweare.html" />
     <nutch     href="http://lucene.apache.org/nutch/" />

Modified: lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skins/lucene/xslt/html/site-to-xhtml.xsl
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skins/lucene/xslt/html/site-to-xhtml.xsl?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skins/lucene/xslt/html/site-to-xhtml.xsl (original)
+++ lucene/dev/branches/solrcloud/solr/site-src/src/documentation/skins/lucene/xslt/html/site-to-xhtml.xsl Sat Jan  7 18:23:08 2012
@@ -379,6 +379,19 @@ footer, searchbar, css etc.  As input, i
     |end bottomstrip
     +</xsl:comment>
         </div>
+        <script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
       </body>
     </html>
   </xsl:template>

Modified: lucene/dev/branches/solrcloud/solr/site/features.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/features.html?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/features.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/features.html Sat Jan  7 18:23:08 2012
@@ -3,7 +3,7 @@
 <head>
 <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
 <meta content="Apache Forrest" name="Generator">
-<meta name="Forrest-version" content="0.8">
+<meta name="Forrest-version" content="0.9">
 <meta name="Forrest-skin-name" content="lucene">
 <title>Introduction to The Solr Enterprise Search Server</title>
 <link type="text/css" href="skin/basic.css" rel="stylesheet">
@@ -109,6 +109,9 @@ document.write("Last Published: " + docu
 <div class="menuitem">
 <a href="http://lucene.apache.org/java/docs/whoweare.html" title="Lucene/Solr Committers">Who We Are</a>
 </div>
+<div class="menuitem">
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>
+</div>
 </div>
 <div onclick="SwitchMenu('menu_selected_1.2', 'skin/')" id="menu_selected_1.2Title" class="menutitle" style="background-image: url('skin/images/chapter_open.gif');">Documentation</div>
 <div id="menu_selected_1.2" class="selectedmenuitemgroup" style="display: block;">
@@ -255,7 +258,7 @@ document.write("Last Published: " + docu
 </div>
 
 
-<a name="N1000D"></a><a name="Solr+in+a+Nutshell"></a>
+<a name="N1000E"></a><a name="Solr+in+a+Nutshell"></a>
 <h2 class="boxed">Solr in a Nutshell</h2>
 <div class="section">
 <p>
@@ -283,7 +286,7 @@ document.write("Last Published: " + docu
 </div>
 
 
-<a name="N10032"></a><a name="Solr+Uses+the+Lucene+Search+Library+and+Extends+it%21"></a>
+<a name="N10033"></a><a name="Solr+Uses+the+Lucene+Search+Library+and+Extends+it%21"></a>
 <h2 class="boxed">Solr Uses the Lucene Search Library and Extends it!</h2>
 <div class="section">
 <ul>
@@ -326,10 +329,10 @@ document.write("Last Published: " + docu
 </div>
 
 
-<a name="N1006F"></a><a name="Detailed+Features"></a>
+<a name="N10070"></a><a name="Detailed+Features"></a>
 <h2 class="boxed">Detailed Features</h2>
 <div class="section">
-<a name="N10075"></a><a name="Schema"></a>
+<a name="N10076"></a><a name="Schema"></a>
 <h3 class="boxed">Schema</h3>
 <ul>
       
@@ -350,7 +353,7 @@ document.write("Last Published: " + docu
 <li>Many additional text analysis components including word splitting, regex and sounds-like filters</li>
     
 </ul>
-<a name="N10096"></a><a name="Query"></a>
+<a name="N10097"></a><a name="Query"></a>
 <h3 class="boxed">Query</h3>
 <ul>
       
@@ -390,7 +393,7 @@ document.write("Last Published: " + docu
 <li>Performance Optimizations</li>
     
 </ul>
-<a name="N100D2"></a><a name="Core"></a>
+<a name="N100D3"></a><a name="Core"></a>
 <h3 class="boxed">Core</h3>
 <ul>
       
@@ -415,7 +418,7 @@ document.write("Last Published: " + docu
 <li>"Luke" request handler for corpus information</li>
     
 </ul>
-<a name="N100F9"></a><a name="Caching"></a>
+<a name="N100FA"></a><a name="Caching"></a>
 <h3 class="boxed">Caching</h3>
 <ul>
       
@@ -449,7 +452,7 @@ document.write("Last Published: " + docu
 <li>User level caching with autowarming support</li>
     
 </ul>
-<a name="N1011E"></a><a name="Replication"></a>
+<a name="N1011F"></a><a name="Replication"></a>
 <h3 class="boxed">Replication</h3>
 <ul>
       
@@ -462,7 +465,7 @@ document.write("Last Published: " + docu
 <li>Replication and automatic reloading of configuration files</li>
     
 </ul>
-<a name="N10133"></a><a name="Admin+Interface"></a>
+<a name="N10134"></a><a name="Admin+Interface"></a>
 <h3 class="boxed">Admin Interface</h3>
 <ul>
       
@@ -516,5 +519,18 @@ document.write("Last Published: " + docu
     |end bottomstrip
     +-->
 </div>
+<script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
 </body>
 </html>

Modified: lucene/dev/branches/solrcloud/solr/site/features.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/features.pdf?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/index.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/index.html?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/index.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/index.html Sat Jan  7 18:23:08 2012
@@ -3,7 +3,7 @@
 <head>
 <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
 <meta content="Apache Forrest" name="Generator">
-<meta name="Forrest-version" content="0.8">
+<meta name="Forrest-version" content="0.9">
 <meta name="Forrest-skin-name" content="lucene">
 <title>Welcome to Solr</title>
 <link type="text/css" href="skin/basic.css" rel="stylesheet">
@@ -109,6 +109,9 @@ document.write("Last Published: " + docu
 <div class="menuitem">
 <a href="http://lucene.apache.org/java/docs/whoweare.html" title="Lucene/Solr Committers">Who We Are</a>
 </div>
+<div class="menuitem">
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>
+</div>
 </div>
 <div onclick="SwitchMenu('menu_1.2', 'skin/')" id="menu_1.2Title" class="menutitle">Documentation</div>
 <div id="menu_1.2" class="menuitemgroup">
@@ -320,7 +323,7 @@ document.write("Last Published: " + docu
 </ul>
 </div> 
     
-<a name="N1000D"></a><a name="intro"></a>
+<a name="N1000E"></a><a name="intro"></a>
 <h2 class="boxed">What Is Solr?</h2>
 <div class="section">
 <p>
@@ -351,7 +354,7 @@ customization is required.
       </p>
 </div>
     
-<a name="N10030"></a><a name="getstarted"></a>
+<a name="N10031"></a><a name="getstarted"></a>
 <h2 class="boxed">Get Started</h2>
 <div class="section">
 <ul>
@@ -373,10 +376,10 @@ customization is required.
 </ul>
 </div>
     
-<a name="N10059"></a><a name="news"></a>
+<a name="N1005A"></a><a name="news"></a>
 <h2 class="boxed">News</h2>
 <div class="section">
-<a name="N1005F"></a><a name="27+November+2011+-+Solr+3.5.0+Released"></a>
+<a name="N10060"></a><a name="27+November+2011+-+Solr+3.5.0+Released"></a>
 <h3 class="boxed">27 November 2011 - Solr 3.5.0 Released</h3>
 <p>
           The Lucene PMC is pleased to announce the release of
@@ -434,7 +437,7 @@ customization is required.
           </li>
         
 </ul>
-<a name="N100C5"></a><a name="18+November+2011+-+2nd+edition+of+the+first+book+on+Solr+published%21"></a>
+<a name="N100C6"></a><a name="18+November+2011+-+2nd+edition+of+the+first+book+on+Solr+published%21"></a>
 <h3 class="boxed">18 November 2011 - 2nd edition of the first book on Solr published!</h3>
 <p>
 <img alt="Apache Solr 3 Enterprise Search Server cover" class="float-right" src="images/as3ess_book.jpg">
@@ -461,7 +464,7 @@ customization is required.
           and at <a href="http://www.solrenterprisesearchserver.com">the authors' site</a>, including a free chapter and
           search parameter quick-reference sheet (the appendix).
         </p>
-<a name="N100E9"></a><a name="26+October+2011+-+Java+7u1+fixes+index+corruption+and+crash+bugs+in+Apache+Lucene+Core+and+Apache+Solr"></a>
+<a name="N100EA"></a><a name="26+October+2011+-+Java+7u1+fixes+index+corruption+and+crash+bugs+in+Apache+Lucene+Core+and+Apache+Solr"></a>
 <h3 class="boxed">26 October 2011 - Java 7u1 fixes index corruption and crash bugs in Apache Lucene Core and Apache Solr</h3>
 <p>Oracle released <a href="http://www.oracle.com/technetwork/java/javase/7u1-relnotes-507962.html">Java 7u1</a> on October 19.
           According to the release notes and tests done by the Lucene committers, all bugs reported on July 28 are fixed in this release,
@@ -475,7 +478,7 @@ customization is required.
           version shipped with Java 7 changed and tokenization behaves differently
           (e.g. lowercasing). For more information, read <span class="codefrag">JRE_VERSION_MIGRATION.txt</span>
           in your distribution package!</p>
-<a name="N10112"></a><a name="14+September+2011+-+Solr+3.4.0+Released"></a>
+<a name="N10113"></a><a name="14+September+2011+-+Solr+3.4.0+Released"></a>
 <h3 class="boxed">14 September 2011 - Solr 3.4.0 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.4.0</a>!
 	</p>
@@ -539,7 +542,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_4_0/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
 	</p>
-<a name="N1019F"></a><a name="28+July+2011+-+WARNING%3A+Index+corruption+and+crashes+in+Apache+Lucene+Core+%2F+Apache+Solr+with+Java+7"></a>
+<a name="N101A0"></a><a name="28+July+2011+-+WARNING%3A+Index+corruption+and+crashes+in+Apache+Lucene+Core+%2F+Apache+Solr+with+Java+7"></a>
 <h3 class="boxed">28 July 2011 - WARNING: Index corruption and crashes in Apache Lucene Core / Apache Solr with Java 7</h3>
 <p>Oracle released <a href="http://www.oracle.com/technetwork/java/javase/jdk7-relnotes-429209.html">Java 7</a> today.
   Unfortunately it contains hotspot compiler optimizations, which miscompile some loops.
@@ -577,13 +580,13 @@ customization is required.
   version shipped with Java 7 changed and tokenization behaves differently
   (e.g. lowercasing). For more information, read <span class="codefrag">JRE_VERSION_MIGRATION.txt</span>
   in your distribution package!</p>
-<a name="N101EF"></a><a name="22+July+2011+-+Solr+3.1+cookbook+published%21"></a>
+<a name="N101F0"></a><a name="22+July+2011+-+Solr+3.1+cookbook+published%21"></a>
 <h3 class="boxed">22 July 2011 - Solr 3.1 cookbook published!</h3>
 <p>
 <img alt="Solr Cookbook cover" class="float-right" src="images/solr_31_cookbook.jpg">Rafał Kuć is proud to introduce a new book on Solr, <a href="http://www.packtpub.com/solr-3-1-enterprise-search-server-cookbook/book" title="PACKT:Apache Solr 3.1 Cookbook">"Apache Solr 3.1 Cookbook"</a> from Packt Publishing.</p>
 <p>The Solr 3.1 Cookbook will make your everyday work easier by using real-life examples that show you how to deal with the most common problems that can arise while using the Apache Solr search engine.</p>
 <p>This cookbook will show you how to get the most out of your search engine. Each chapter covers a different aspect of working with Solr from analyzing your text data through querying, performance improvement, and developing your own modules. The practical recipes will help you to quickly solve common problems with data analysis, show you how to use faceting to collect data and to speed up the performance of Solr. You will learn about functionalities that most newbies are unaware of, such as sorting results by a function value, highlighting matched words, and computing statistics to make your work with Solr easy and stress free.</p>
-<a name="N10208"></a><a name="July+2011+-+Solr+3.3+Released"></a>
+<a name="N10209"></a><a name="July+2011+-+Solr+3.3+Released"></a>
 <h3 class="boxed">July 2011 - Solr 3.3 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.3</a>!
   </p>
@@ -613,7 +616,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_3/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N1023C"></a><a name="May+2011+-+Solr+3.2+Released"></a>
+<a name="N1023D"></a><a name="May+2011+-+Solr+3.2+Released"></a>
 <h3 class="boxed">May 2011 - Solr 3.2 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.2</a>!
   </p>
@@ -639,7 +642,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_2/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N10269"></a><a name="March+2011+-+Solr+3.1+Released"></a>
+<a name="N1026A"></a><a name="March+2011+-+Solr+3.1+Released"></a>
 <h3 class="boxed">March 2011 - Solr 3.1 Released</h3>
 <p>The Lucene PMC is pleased to announce the release of <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr">Apache Solr 3.1</a>!
   </p>
@@ -687,7 +690,7 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_1/solr/CHANGES.txt">release notes</a> for a more complete list of all the new features, improvements, and bugfixes.
  </p>
-<a name="N102D4"></a><a name="25+June+2010+-+Solr+1.4.1+Released"></a>
+<a name="N102D5"></a><a name="25+June+2010+-+Solr+1.4.1+Released"></a>
 <h3 class="boxed">25 June 2010 - Solr 1.4.1 Released</h3>
 <p>
            Solr 1.4.1 has been released and is now available for public
@@ -700,7 +703,7 @@ customization is required.
           See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.4.1/CHANGES.txt">release notes</a>
           for more details.
           </p>
-<a name="N102E9"></a><a name="7+May+2010+-+Apache+Lucene+Eurocon+2010+Coming+to+Prague+May+18-21"></a>
+<a name="N102EA"></a><a name="7+May+2010+-+Apache+Lucene+Eurocon+2010+Coming+to+Prague+May+18-21"></a>
 <h3 class="boxed">7 May 2010 - Apache Lucene Eurocon 2010 Coming to Prague May 18-21</h3>
 <p>
           On May 18th to the 21st Prague will play host to the first
@@ -772,7 +775,7 @@ customization is required.
 </li>
         
 </ul>
-<a name="N10335"></a><a name="10+November+2009+-+Solr+1.4+Released"></a>
+<a name="N10336"></a><a name="10+November+2009+-+Solr+1.4+Released"></a>
 <h3 class="boxed">10 November 2009 - Solr 1.4 Released</h3>
 <p>
      Solr 1.4 has been released and is now available for public <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr/">download</a>!
@@ -804,12 +807,12 @@ customization is required.
 </ul>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.4.0/CHANGES.txt">release notes</a> for more details.
  </p>
-<a name="N10365"></a><a name="20+August+2009+-+Solr%27s+first+book+is+published%21"></a>
+<a name="N10366"></a><a name="20+August+2009+-+Solr%27s+first+book+is+published%21"></a>
 <h3 class="boxed">20 August 2009 - Solr's first book is published!</h3>
 <p>
    
-<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&utm_medium=spons&utm_content=pod&utm_campaign=mdb_000275" title="PACKT:Solr 1.4 Enterprise Search Server"><img alt="Solr book cover" class="float-right" src="images/solr-book-image.jpg"></a>
-   David Smiley and Eric Pugh are proud to introduce the first book on Solr, <strong><a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&utm_medium=spons&utm_content=pod&utm_campaign=mdb_000275">"Solr 1.4 Enterprise Search Server"</a></strong> from Packt Publishing.
+<a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275" title="PACKT:Solr 1.4 Enterprise Search Server"><img alt="Solr book cover" class="float-right" src="images/solr-book-image.jpg"></a>
+   David Smiley and Eric Pugh are proud to introduce the first book on Solr, <strong><a href="http://www.packtpub.com/solr-1-4-enterprise-search-server?utm_source=http%3A%2F%2Flucene.apache.org%2Fsolr%2F&amp;utm_medium=spons&amp;utm_content=pod&amp;utm_campaign=mdb_000275">"Solr 1.4 Enterprise Search Server"</a></strong> from Packt Publishing.
    </p>
 <p>This book is a comprehensive reference guide for nearly every feature Solr has to offer. It serves the reader right from initiation to development to deployment. It also comes with complete running examples to demonstrate its use and show how to integrate it with other languages and frameworks.
    </p>
@@ -819,7 +822,7 @@ customization is required.
    </p>
 <p>Finally, this book covers various deployment considerations to include indexing strategies and performance-oriented configuration that will enable you to scale Solr to meet the needs of a high-volume site.
    </p>
-<a name="N10385"></a><a name="18+August+2009+-+Lucene+at+US+ApacheCon"></a>
+<a name="N10386"></a><a name="18+August+2009+-+Lucene+at+US+ApacheCon"></a>
 <h3 class="boxed">18 August 2009 - Lucene at US ApacheCon</h3>
 <p>
  
@@ -895,7 +898,7 @@ Be sure not to miss:
     Search</a> - Jason Rutherglen @ 15:00</li>
 
 </ul>
-<a name="N103F9"></a><a name="09+February+2009+-+Lucene+at+ApacheCon+Europe+2009+in+Amsterdam"></a>
+<a name="N103FA"></a><a name="09+February+2009+-+Lucene+at+ApacheCon+Europe+2009+in+Amsterdam"></a>
 <h3 class="boxed">09 February 2009 - Lucene at ApacheCon Europe 2009 in Amsterdam</h3>
 <p>
                
@@ -933,23 +936,23 @@ Be sure not to miss:
 
 							              
 </ul>
-<a name="N10442"></a><a name="19+December+2008+-+Solr+Logo+Contest+Results"></a>
+<a name="N10443"></a><a name="19+December+2008+-+Solr+Logo+Contest+Results"></a>
 <h3 class="boxed">19 December 2008 - Solr Logo Contest Results</h3>
 <p>Many great logos were submitted, but only one could be chosen.  Congratulations Michiel,
 	the creator of the winning logo that is proudly displayed at the top of this page.  
 	</p>
-<a name="N1044B"></a><a name="03+October+2008+-+Solr+Logo+Contest"></a>
+<a name="N1044C"></a><a name="03+October+2008+-+Solr+Logo+Contest"></a>
 <h3 class="boxed">03 October 2008 - Solr Logo Contest</h3>
 <p>By popular demand, Solr is holding a contest to pick a new Solr logo.  Details about how to submit an entry can be found <a href="http://wiki.apache.org/solr/LogoContest">on the wiki</a>.  The Deadline for submissions is November 20th, 2008 @ 11:59PM GMT.
 	</p>
-<a name="N10458"></a><a name="15+September+2008+-+Solr+1.3.0+Available"></a>
+<a name="N10459"></a><a name="15+September+2008+-+Solr+1.3.0+Available"></a>
 <h3 class="boxed">15 September 2008 - Solr 1.3.0 Available</h3>
 <p>Solr 1.3.0 is available for public download.  This version contains many enhancements and bug fixes, including distributed search capabilities,
 			Lucene 2.3.x performance improvements and many others.
 		</p>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.3.0/CHANGES.txt">release notes</a> for more details.  Download is
     available from a <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr/">Apache Mirror</a>.</p>
-<a name="N1046C"></a><a name="28+August+2008+-+Lucene%2FSolr+at+ApacheCon+New+Orleans"></a>
+<a name="N1046D"></a><a name="28+August+2008+-+Lucene%2FSolr+at+ApacheCon+New+Orleans"></a>
 <h3 class="boxed">28 August 2008 - Lucene/Solr at ApacheCon New Orleans</h3>
 <p>
 	         
@@ -971,7 +974,7 @@ Be sure not to miss:
 <li>An <a href="http://us.apachecon.com/c/acus2008/schedule/2008/11/05">entire day of Lucene sessions</a> on November 5th</li>
 	        
 </ul>
-<a name="N1049A"></a><a name="03+September+2007+-+Lucene+at+ApacheCon+Atlanta"></a>
+<a name="N1049B"></a><a name="03+September+2007+-+Lucene+at+ApacheCon+Atlanta"></a>
 <h3 class="boxed">03 September 2007 - Lucene at ApacheCon Atlanta</h3>
 <p>
 <a href="http://www.us.apachecon.com"><img alt="ApacheCon US logo" class="float-right" src="http://www.apache.org/ads/ApacheCon/2007-usa-125x125.png"></a>
@@ -991,7 +994,7 @@ Be sure not to miss:
 <li>November 16, 4:00 pm: <a href="http://us.apachecon.com/us2007/program/talk/2017"> Advanced Indexing Techniques with Apache Lucene</a> by Michael Busch. Information on payloads and advanced indexing techniques.</li>
               
 </ul>
-<a name="N104D3"></a><a name="06+June+2007%3A+Release+1.2+available"></a>
+<a name="N104D4"></a><a name="06+June+2007%3A+Release+1.2+available"></a>
 <h3 class="boxed">06 June 2007: Release 1.2 available</h3>
 <p>
         This is the first release since Solr graduated from the Incubator,
@@ -1001,40 +1004,40 @@ Be sure not to miss:
         and more flexible plugins.
       </p>
 <p>See the <a href="http://svn.apache.org/repos/asf/lucene/solr/tags/release-1.2.0/CHANGES.txt">release notes</a> for more details.</p>
-<a name="N104E4"></a><a name="17+January+2007%3A+Solr+graduates+from+Incubator"></a>
+<a name="N104E5"></a><a name="17+January+2007%3A+Solr+graduates+from+Incubator"></a>
 <h3 class="boxed">17 January 2007: Solr graduates from Incubator</h3>
 <p>
         Solr has graduated from the Apache Incubator, and is now a sub-project of Lucene.
       </p>
-<a name="N104EE"></a><a name="22+December+2006%3A+Release+1.1.0+available"></a>
+<a name="N104EF"></a><a name="22+December+2006%3A+Release+1.1.0+available"></a>
 <h3 class="boxed">22 December 2006: Release 1.1.0 available</h3>
 <p>
         This is the first release since Solr joined the Incubator, and brings
         many new features and performance optimizations including highlighting,
         faceted search, and JSON/Python/Ruby response formats.
       </p>
-<a name="N104F8"></a><a name="15+August+2006%3A+Solr+at+ApacheCon+US"></a>
+<a name="N104F9"></a><a name="15+August+2006%3A+Solr+at+ApacheCon+US"></a>
 <h3 class="boxed">15 August 2006: Solr at ApacheCon US</h3>
 <p>Chris Hostetter will be presenting
         <strong><a href="http://www.apachecon.com/2006/US/html/sessions.html#FR26">"Faceted Searching With Apache Solr"</a></strong>  
         at ApacheCon US 2006, on October 13th at 4:30pm.
         See the <a href="http://www.us.apachecon.com/">ApacheCon</a> website for more details.
       </p>
-<a name="N1050B"></a><a name="21+April+2006%3A+Solr+at+ApacheCon"></a>
+<a name="N1050C"></a><a name="21+April+2006%3A+Solr+at+ApacheCon"></a>
 <h3 class="boxed">21 April 2006: Solr at ApacheCon</h3>
 <p>Yonik Seeley will be presenting
         <strong>"Apache Solr, a Full-Text Search Server based on Lucene"</strong>  
         at ApacheCon Europe 2006, on June 29th at 5:30pm.
         See the <a href="http://www.eu.apachecon.com/">ApacheCon</a> website for more details.
       </p>
-<a name="N1051C"></a><a name="21+February+2006%3A+nightly+builds"></a>
+<a name="N1051D"></a><a name="21+February+2006%3A+nightly+builds"></a>
 <h3 class="boxed">21 February 2006: nightly builds</h3>
 <p>Solr now has nightly builds.  This automatically creates a
       <a href="http://people.apache.org/builds/lucene/solr/nightly/">downloadable version of Solr every
       night</a>.  All unit tests must pass, or a message is sent to
       the developers mailing list and no new version is created.  This
       also updates the <a href="api/index.html">javadoc</a>.</p>
-<a name="N1052E"></a><a name="17+January+2006%3A+Solr+Joins+Apache+Incubator"></a>
+<a name="N1052F"></a><a name="17+January+2006%3A+Solr+Joins+Apache+Incubator"></a>
 <h3 class="boxed">17 January 2006: Solr Joins Apache Incubator</h3>
 <p>Solr, a search server based on Lucene, has been accepted into the Apache Incubator.
             Solr was originally developed by CNET Networks, and is widely used within CNET
@@ -1067,5 +1070,18 @@ document.write("Last Published: " + docu
     |end bottomstrip
     +-->
 </div>
+<script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
 </body>
 </html>

Modified: lucene/dev/branches/solrcloud/solr/site/index.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/index.pdf?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/issue_tracking.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/issue_tracking.html?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/issue_tracking.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/issue_tracking.html Sat Jan  7 18:23:08 2012
@@ -3,7 +3,7 @@
 <head>
 <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
 <meta content="Apache Forrest" name="Generator">
-<meta name="Forrest-version" content="0.8">
+<meta name="Forrest-version" content="0.9">
 <meta name="Forrest-skin-name" content="lucene">
 <title>Solr Issue Tracking</title>
 <link type="text/css" href="skin/basic.css" rel="stylesheet">
@@ -109,6 +109,9 @@ document.write("Last Published: " + docu
 <div class="menuitem">
 <a href="http://lucene.apache.org/java/docs/whoweare.html" title="Lucene/Solr Committers">Who We Are</a>
 </div>
+<div class="menuitem">
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>
+</div>
 </div>
 <div onclick="SwitchMenu('menu_1.2', 'skin/')" id="menu_1.2Title" class="menutitle">Documentation</div>
 <div id="menu_1.2" class="menuitemgroup">
@@ -251,5 +254,18 @@ document.write("Last Published: " + docu
     |end bottomstrip
     +-->
 </div>
+<script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
 </body>
 </html>

Modified: lucene/dev/branches/solrcloud/solr/site/issue_tracking.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/issue_tracking.pdf?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/linkmap.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/linkmap.html?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/linkmap.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/linkmap.html Sat Jan  7 18:23:08 2012
@@ -3,7 +3,7 @@
 <head>
 <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
 <meta content="Apache Forrest" name="Generator">
-<meta name="Forrest-version" content="0.8">
+<meta name="Forrest-version" content="0.9">
 <meta name="Forrest-skin-name" content="lucene">
 <title>Site Linkmap Table of Contents</title>
 <link type="text/css" href="skin/basic.css" rel="stylesheet">
@@ -109,6 +109,9 @@ document.write("Last Published: " + docu
 <div class="menuitem">
 <a href="http://lucene.apache.org/java/docs/whoweare.html" title="Lucene/Solr Committers">Who We Are</a>
 </div>
+<div class="menuitem">
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>
+</div>
 </div>
 <div onclick="SwitchMenu('menu_1.2', 'skin/')" id="menu_1.2Title" class="menutitle">Documentation</div>
 <div id="menu_1.2" class="menuitemgroup">
@@ -245,6 +248,12 @@ document.write("Last Published: " + docu
 <li>
 <a href="http://lucene.apache.org/java/docs/whoweare.html">Who We Are</a>&nbsp;&nbsp;___________________&nbsp;&nbsp;<em>index</em>&nbsp;: Lucene/Solr Committers</li>
 </ul>
+    
+<ul>
+<li>
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>&nbsp;&nbsp;___________________&nbsp;&nbsp;<em>privacy</em>
+</li>
+</ul>
   
 </ul>
 </ul>
@@ -393,5 +402,18 @@ document.write("Last Published: " + docu
     |end bottomstrip
     +-->
 </div>
+<script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
 </body>
 </html>

Modified: lucene/dev/branches/solrcloud/solr/site/linkmap.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/linkmap.pdf?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/mailing_lists.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/mailing_lists.html?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/mailing_lists.html (original)
+++ lucene/dev/branches/solrcloud/solr/site/mailing_lists.html Sat Jan  7 18:23:08 2012
@@ -3,7 +3,7 @@
 <head>
 <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
 <meta content="Apache Forrest" name="Generator">
-<meta name="Forrest-version" content="0.8">
+<meta name="Forrest-version" content="0.9">
 <meta name="Forrest-skin-name" content="lucene">
 <title>Solr Mailing Lists</title>
 <link type="text/css" href="skin/basic.css" rel="stylesheet">
@@ -109,6 +109,9 @@ document.write("Last Published: " + docu
 <div class="menuitem">
 <a href="http://lucene.apache.org/java/docs/whoweare.html" title="Lucene/Solr Committers">Who We Are</a>
 </div>
+<div class="menuitem">
+<a href="http://lucene.apache.org/privacy.html">Privacy Policy</a>
+</div>
 </div>
 <div onclick="SwitchMenu('menu_1.2', 'skin/')" id="menu_1.2Title" class="menutitle">Documentation</div>
 <div id="menu_1.2" class="menuitemgroup">
@@ -235,7 +238,7 @@ document.write("Last Published: " + docu
 </div>
 
     
-<a name="N1000D"></a><a name="Users"></a>
+<a name="N1000E"></a><a name="Users"></a>
 <h2 class="boxed">Users</h2>
 <div class="section">
 <p>If you use Solr, please subscribe to the Solr user mailing list.</p>
@@ -265,7 +268,7 @@ document.write("Last Published: " + docu
 </div>
 
     
-<a name="N10038"></a><a name="Developers"></a>
+<a name="N10039"></a><a name="Developers"></a>
 <h2 class="boxed">Developers</h2>
 <div class="section">
 <p>If you'd like to contribute to Solr, please subscribe to the
@@ -301,7 +304,7 @@ document.write("Last Published: " + docu
 </div>
 
     
-<a name="N10069"></a><a name="Commits"></a>
+<a name="N1006A"></a><a name="Commits"></a>
 <h2 class="boxed">Commits</h2>
 <div class="section">
 <p>If you'd like to see changes made in Solr's <a href="version_control.html">version control system</a>
@@ -343,5 +346,18 @@ document.write("Last Published: " + docu
     |end bottomstrip
     +-->
 </div>
+<script type="text/javascript">
+
+  var _gaq = _gaq || [];
+  _gaq.push(['_setAccount', 'UA-94576-12']);
+  _gaq.push(['_trackPageview']);
+
+  (function() {
+    var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+    ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+    var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+  })();
+
+        </script>
 </body>
 </html>

Modified: lucene/dev/branches/solrcloud/solr/site/mailing_lists.pdf
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/mailing_lists.pdf?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
Binary files - no diff available.

Modified: lucene/dev/branches/solrcloud/solr/site/skin/basic.css
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/skin/basic.css?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/skin/basic.css (original)
+++ lucene/dev/branches/solrcloud/solr/site/skin/basic.css Sat Jan  7 18:23:08 2012
@@ -163,4 +163,4 @@ p {
 .codefrag {
   font-family: "Courier New", Courier, monospace;
   font-size: 110%;
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/solrcloud/solr/site/skin/print.css
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/skin/print.css?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/skin/print.css (original)
+++ lucene/dev/branches/solrcloud/solr/site/skin/print.css Sat Jan  7 18:23:08 2012
@@ -51,4 +51,4 @@ a:link, a:visited {
 
 acronym {
   border: 0;
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/solrcloud/solr/site/skin/profile.css
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/skin/profile.css?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/skin/profile.css (original)
+++ lucene/dev/branches/solrcloud/solr/site/skin/profile.css Sat Jan  7 18:23:08 2012
@@ -172,4 +172,4 @@ a:hover { color:#6587ff} 
     }
       
     
-  
+  
\ No newline at end of file

Modified: lucene/dev/branches/solrcloud/solr/site/skin/screen.css
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/site/skin/screen.css?rev=1228693&r1=1228692&r2=1228693&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/site/skin/screen.css (original)
+++ lucene/dev/branches/solrcloud/solr/site/skin/screen.css Sat Jan  7 18:23:08 2012
@@ -584,4 +584,4 @@ p.instruction {
   list-style-image: url('../images/instruction_arrow.png');
   list-style-position: outside;
   margin-left: 2em;
-} 
+} 
\ No newline at end of file