You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/01/16 10:43:52 UTC

svn commit: r1433869 [2/3] - in /lucene/dev/trunk/lucene: ./ facet/src/examples/org/apache/lucene/facet/example/merge/ facet/src/examples/org/apache/lucene/facet/example/multiCL/ facet/src/java/org/apache/lucene/facet/associations/ facet/src/java/org/a...

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java Wed Jan 16 09:43:51 2013
@@ -28,7 +28,10 @@ import org.apache.lucene.facet.taxonomy.
 
 /**
  * A {@link FacetRequest} for weighting facets according to their float
- * association by summing the association values.
+ * association by summing the association values. Note that this class caches
+ * the associations data in-memory by default. You can override
+ * {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
+ * {@link AssociationFloatSumAggregator} which does otherwise.
  * 
  * @lucene.experimental
  */

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java Wed Jan 16 09:43:51 2013
@@ -4,6 +4,7 @@ import java.io.IOException;
 
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
+import org.apache.lucene.facet.search.aggregator.associations.AssociationFloatSumAggregator;
 import org.apache.lucene.facet.search.aggregator.associations.AssociationIntSumAggregator;
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
@@ -28,7 +29,10 @@ import org.apache.lucene.facet.taxonomy.
 
 /**
  * A {@link FacetRequest} for weighting facets according to their integer
- * association by summing the association values.
+ * association by summing the association values. Note that this class caches
+ * the associations data in-memory by default. You can override
+ * {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
+ * {@link AssociationFloatSumAggregator} which does otherwise.
  * 
  * @lucene.experimental
  */

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResult.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResult.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResult.java Wed Jan 16 09:43:51 2013
@@ -39,7 +39,7 @@ public class FacetResult {
   /**
    * Facet result node matching the root of the {@link #getFacetRequest() facet request}.
    * @see #getFacetRequest()
-   * @see FacetRequest#getCategoryPath()
+   * @see FacetRequest#categoryPath
    */
   public final FacetResultNode getFacetResultNode() {
     return this.rootNode;

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java Wed Jan 16 09:43:51 2013
@@ -4,9 +4,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.lucene.index.IndexReader;
-
-import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.ScoredDocIDs;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
@@ -16,6 +13,7 @@ import org.apache.lucene.facet.search.re
 import org.apache.lucene.facet.search.results.FacetResultNode;
 import org.apache.lucene.facet.search.results.MutableFacetResultNode;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.IndexReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -205,7 +203,7 @@ public abstract class Sampler {
   private static class OverSampledFacetRequest extends FacetRequest {
     final FacetRequest orig;
     public OverSampledFacetRequest(FacetRequest orig, int num) {
-      super(orig.getCategoryPath(), num);
+      super(orig.categoryPath, num);
       this.orig = orig;
       setDepth(orig.getDepth());
       setNumLabel(orig.getNumLabel());
@@ -215,12 +213,6 @@ public abstract class Sampler {
     }
     
     @Override
-    public CategoryListIterator createCategoryListIterator(TaxonomyReader taxo, FacetSearchParams sParams, 
-        int partition) throws IOException {
-      return orig.createCategoryListIterator(taxo, sParams, partition);
-    }
-    
-    @Override
     public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
         throws IOException {
       return orig.createAggregator(useComplements, arrays, taxonomy);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java Wed Jan 16 09:43:51 2013
@@ -1,6 +1,5 @@
 package org.apache.lucene.facet.util;
 
-import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
@@ -28,15 +27,9 @@ import org.apache.lucene.facet.taxonomy.
  */
 public final class PartitionsUtils {
 
-  /**
-   * Get the offset for a given partition.  That is, what is the minimum number an
-   * ordinal could be for a particular partition. 
-   */
-  public final static int partitionOffset(FacetIndexingParams iParams,
-      int partitionNumber, final TaxonomyReader taxonomyReader) {
-    return partitionNumber * partitionSize(iParams, taxonomyReader);
-  }
-
+  /** The prefix that is added to the name of the partition. */
+  public static final String PART_NAME_PREFIX = "$part";
+  
   /**
    * Get the partition size in this parameter, or return the size of the taxonomy, which
    * is smaller.  (Guarantees usage of as little memory as possible at search time).
@@ -58,21 +51,21 @@ public final class PartitionsUtils {
   /**
    * Partition name by category ordinal
    */
-  public final static String partitionNameByOrdinal(
-      FacetIndexingParams iParams, CategoryListParams clParams, int ordinal) {
+  public final static String partitionNameByOrdinal(FacetIndexingParams iParams, int ordinal) {
     int partition = partitionNumber(iParams, ordinal);
-    return partitionName(clParams, partition);
+    return partitionName(partition);
   }
 
-  /** 
-   * Partition name by its number
-   */
-  public final static String partitionName(CategoryListParams clParams, int partition) {
-    String term = clParams.getTerm().text();
+  /** Partition name by its number */
+  public final static String partitionName(int partition) {
+    // TODO would be good if this method isn't called when partitions are not enabled.
+    // perhaps through some specialization code.
     if (partition == 0) {
-      return term; // for backwards compatibility we do not add a partition number in this case
+      // since regular faceted search code goes through this method too,
+      // return the same value for partition 0 and when there are no partitions
+      return "";
     }
-    return term + partition;
+    return PART_NAME_PREFIX + Integer.toString(partition);
   }
 
 }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java Wed Jan 16 09:43:51 2013
@@ -266,13 +266,12 @@ public abstract class FacetTestBase exte
     FacetIndexingParams iParams = getFacetIndexingParams(Integer.MAX_VALUE);
     String delim = String.valueOf(iParams.getFacetDelimChar());
     Map<CategoryPath, Integer> res = new HashMap<CategoryPath, Integer>();
-    HashSet<Term> handledTerms = new HashSet<Term>();
+    HashSet<String> handledTerms = new HashSet<String>();
     for (CategoryListParams clp : iParams.getAllCategoryListParams()) {
-      Term baseTerm = new Term(clp.getTerm().field());
-      if (!handledTerms.add(baseTerm)) {
+      if (!handledTerms.add(clp.field)) {
         continue; // already handled this term (for another list) 
       }
-      Terms terms = MultiFields.getTerms(indexReader, baseTerm.field());
+      Terms terms = MultiFields.getTerms(indexReader, clp.field);
       if (terms == null) {
         continue;
       }
@@ -297,7 +296,7 @@ public abstract class FacetTestBase exte
       FacetResultNode topResNode = fr.getFacetResultNode();
       FacetRequest freq = fr.getFacetRequest();
       if (VERBOSE) {
-        System.out.println(freq.getCategoryPath().toString()+ "\t\t" + topResNode);
+        System.out.println(freq.categoryPath.toString()+ "\t\t" + topResNode);
       }
       assertCountsAndCardinality(facetCountsTruth, topResNode, freq.getNumResults());
     }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestUtils.java Wed Jan 16 09:43:51 2013
@@ -2,14 +2,9 @@ package org.apache.lucene.facet;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.FacetsCollector;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
@@ -23,7 +18,6 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
@@ -116,15 +110,6 @@ public class FacetTestUtils {
     return collectors;
   }
   
-  public static void add(FacetIndexingParams iParams, RandomIndexWriter iw,
-      TaxonomyWriter tw, String... strings) throws IOException {
-    Document d = new Document();
-    FacetFields facetFields = new FacetFields(tw, iParams);
-    facetFields.addFields(d, Collections.singletonList(new CategoryPath(strings)));
-    d.add(new TextField("content", "alpha", Field.Store.YES));
-    iw.addDocument(d);
-  }
-
   public static class IndexTaxonomyReaderPair {
     public DirectoryReader indexReader;
     public DirectoryTaxonomyReader taxReader;

Added: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/TestFacetsPayloadMigrationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/TestFacetsPayloadMigrationReader.java?rev=1433869&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/TestFacetsPayloadMigrationReader.java (added)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/TestFacetsPayloadMigrationReader.java Wed Jan 16 09:43:51 2013
@@ -0,0 +1,398 @@
+package org.apache.lucene.facet.index;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.index.params.FacetIndexingParams;
+import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
+import org.apache.lucene.facet.search.CategoryListIterator;
+import org.apache.lucene.facet.search.DrillDown;
+import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.search.params.CountFacetRequest;
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
+import org.apache.lucene.facet.search.results.FacetResult;
+import org.apache.lucene.facet.search.results.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.facet.util.PartitionsUtils;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollector;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TotalHitCountCollector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Tests facets index migration from payload to DocValues.*/
+public class TestFacetsPayloadMigrationReader extends LuceneTestCase {
+  
+  private static class PayloadFacetFields extends FacetFields {
+
+    private static final class CountingListStream extends TokenStream {
+      private final PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
+      private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+      private final Iterator<Entry<String,BytesRef>> categoriesData;
+      
+      CountingListStream(Map<String,BytesRef> categoriesData) {
+        this.categoriesData = categoriesData.entrySet().iterator();
+      }
+      
+      @Override
+      public boolean incrementToken() throws IOException {
+        if (!categoriesData.hasNext()) {
+          return false;
+        }
+        
+        Entry<String,BytesRef> entry = categoriesData.next();
+        termAtt.setEmpty().append(FacetsPayloadMigrationReader.PAYLOAD_TERM_TEXT + entry.getKey());
+        payloadAtt.setPayload(entry.getValue());
+        return true;
+      }
+      
+    }
+
+    private static final FieldType COUNTING_LIST_PAYLOAD_TYPE = new FieldType();
+    static {
+      COUNTING_LIST_PAYLOAD_TYPE.setIndexed(true);
+      COUNTING_LIST_PAYLOAD_TYPE.setTokenized(true);
+      COUNTING_LIST_PAYLOAD_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+      COUNTING_LIST_PAYLOAD_TYPE.setStored(false);
+      COUNTING_LIST_PAYLOAD_TYPE.setOmitNorms(true);
+      COUNTING_LIST_PAYLOAD_TYPE.freeze();
+    }
+    
+    public PayloadFacetFields(TaxonomyWriter taxonomyWriter, FacetIndexingParams params) {
+      super(taxonomyWriter, params);
+    }
+
+    @Override
+    protected FieldType drillDownFieldType() {
+      // Since the payload is indexed in the same field as the drill-down terms,
+      // we must set IndexOptions to DOCS_AND_FREQS_AND_POSITIONS
+      final FieldType type = new FieldType(TextField.TYPE_NOT_STORED);
+      type.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+      type.freeze();
+      return type;
+    }
+
+    @Override
+    protected void addCountingListData(Document doc, Map<String,BytesRef> categoriesData, String field) {
+      CountingListStream ts = new CountingListStream(categoriesData);
+      doc.add(new Field(field, ts, COUNTING_LIST_PAYLOAD_TYPE));
+    }
+  }
+
+  private static final String[] DIMENSIONS = new String[] { "dim1", "dim2", "dim3.1", "dim3.2" };
+  
+  private HashMap<String,Integer> createIndex(Directory indexDir, Directory taxoDir, FacetIndexingParams fip) 
+      throws Exception {
+    Random random = random();
+    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
+    conf.setMaxBufferedDocs(2); // force few segments
+    conf.setMergePolicy(NoMergePolicy.COMPOUND_FILES); // avoid merges so that we're left with few segments
+    IndexWriter indexWriter = new IndexWriter(indexDir, conf);
+    TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
+    
+    FacetFields facetFields = new PayloadFacetFields(taxoWriter, fip);
+    
+    HashMap<String,Integer> expectedCounts = new HashMap<String,Integer>(DIMENSIONS.length);
+    int numDocs = atLeast(10);
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      int numCategories = random.nextInt(3) + 1;
+      ArrayList<CategoryPath> categories = new ArrayList<CategoryPath>(numCategories);
+      HashSet<String> docDimensions = new HashSet<String>();
+      while (numCategories-- > 0) {
+        String dim = DIMENSIONS[random.nextInt(DIMENSIONS.length)];
+        // we should only increment the expected count by 1 per document
+        docDimensions.add(dim);
+        categories.add(new CategoryPath(dim, Integer.toString(i), Integer.toString(numCategories)));
+      }
+      facetFields.addFields(doc, categories);
+      doc.add(new StringField("docid", Integer.toString(i), Store.YES));
+      doc.add(new TextField("foo", "content" + i, Store.YES));
+      indexWriter.addDocument(doc);
+
+      // update expected count per dimension
+      for (String dim : docDimensions) {
+        Integer val = expectedCounts.get(dim);
+        if (val == null) {
+          expectedCounts.put(dim, Integer.valueOf(1));
+        } else {
+          expectedCounts.put(dim, Integer.valueOf(val.intValue() + 1));
+        }
+      }
+      
+      if (random.nextDouble() < 0.2) { // add some documents that will be deleted
+        doc = new Document();
+        doc.add(new StringField("del", "key", Store.NO));
+        facetFields.addFields(doc, Collections.singletonList(new CategoryPath("dummy")));
+        indexWriter.addDocument(doc);
+      }
+    }
+    
+    indexWriter.commit();
+    taxoWriter.commit();
+
+    // delete the docs that were marked for deletion. note that the 'dummy'
+    // category is not removed from the taxonomy, so must account for it when we
+    // verify the migrated index.
+    indexWriter.deleteDocuments(new Term("del", "key"));
+    indexWriter.commit();
+    
+    IOUtils.close(indexWriter, taxoWriter);
+    
+    return expectedCounts;
+  }
+  
+  private void migrateIndex(Directory indexDir, FacetIndexingParams fip) throws Exception {
+    final Map<String,Term> fieldTerms = FacetsPayloadMigrationReader.buildFieldTermsMap(indexDir, fip);
+    DirectoryReader reader = DirectoryReader.open(indexDir);
+    List<AtomicReaderContext> leaves = reader.leaves();
+    int numReaders = leaves.size();
+    AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];
+    for (int i = 0; i < numReaders; i++) {
+      wrappedLeaves[i] = new FacetsPayloadMigrationReader(leaves.get(i).reader(), fieldTerms);
+    }
+    
+    IndexWriter writer = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, null));
+    writer.deleteAll();
+    try {
+      writer.addIndexes(new MultiReader(wrappedLeaves));
+      writer.commit();
+    } finally {
+      reader.close();
+      writer.close();
+    }
+  }
+  
+  private void verifyMigratedIndex(Directory indexDir, Directory taxoDir, HashMap<String,Integer> expectedCounts, 
+      FacetIndexingParams fip) throws Exception {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    assertFalse("index should not have deletions", indexReader.hasDeletions());
+    
+    verifyNotFacetsData(indexReader, searcher);
+    verifyFacetedSearch(expectedCounts, fip, indexReader, taxoReader, searcher);
+    verifyDrillDown(expectedCounts, fip, indexReader, taxoReader, searcher);
+    verifyIndexOrdinals(indexReader, taxoReader, fip);
+    
+    IOUtils.close(indexReader, taxoReader);
+  }
+  
+  private void verifyNotFacetsData(DirectoryReader indexReader, IndexSearcher searcher) throws IOException {
+    // verify that non facets data was not damaged
+    TotalHitCountCollector total = new TotalHitCountCollector();
+    searcher.search(new PrefixQuery(new Term("foo", "content")), total);
+    assertEquals("invalid number of results for content query", total.getTotalHits(), indexReader.maxDoc());
+    
+    int numDocIDs = 0;
+    for (AtomicReaderContext context : indexReader.leaves()) {
+      Terms docIDs = context.reader().terms("docid");
+      assertNotNull(docIDs);
+      TermsEnum te = docIDs.iterator(null);
+      while (te.next() != null) {
+        ++numDocIDs;
+      }
+    }
+    assertEquals("invalid number of docid terms", indexReader.maxDoc(), numDocIDs);
+  }
+  
+  private void verifyFacetedSearch(Map<String,Integer> expectedCounts, FacetIndexingParams fip, 
+      DirectoryReader indexReader, TaxonomyReader taxoReader, IndexSearcher searcher) throws IOException {
+    // run faceted search and assert expected counts
+    ArrayList<FacetRequest> requests = new ArrayList<FacetRequest>(expectedCounts.size());
+    for (String dim : expectedCounts.keySet()) {
+      requests.add(new CountFacetRequest(new CategoryPath(dim), 5));
+    }
+    FacetSearchParams fsp = new FacetSearchParams(requests, fip);
+    FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
+    MatchAllDocsQuery base = new MatchAllDocsQuery();
+    searcher.search(base, fc);
+    List<FacetResult> facetResults = fc.getFacetResults();
+    assertEquals(requests.size(), facetResults.size());
+    for (FacetResult res : facetResults) {
+      FacetResultNode node = res.getFacetResultNode();
+      String dim = node.getLabel().components[0];
+      assertEquals("wrong count for " + dim, expectedCounts.get(dim).intValue(), (int) node.getValue());
+    }
+  }
+  
+  private void verifyDrillDown(Map<String,Integer> expectedCounts, FacetIndexingParams fip, DirectoryReader indexReader, 
+      TaxonomyReader taxoReader, IndexSearcher searcher) throws IOException {
+    // verify drill-down
+    for (String dim : expectedCounts.keySet()) {
+      CategoryPath drillDownCP = new CategoryPath(dim);
+      ArrayList<FacetRequest> request = new ArrayList<FacetRequest>(1);
+      request.add(new CountFacetRequest(drillDownCP, 10));
+      FacetSearchParams fsp = new FacetSearchParams(request, fip);
+      Query drillDown = DrillDown.query(fsp, new MatchAllDocsQuery(), drillDownCP);
+      TotalHitCountCollector total = new TotalHitCountCollector();
+      FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
+      searcher.search(drillDown, MultiCollector.wrap(fc, total));
+      assertTrue("no results for drill-down query " + drillDown, total.getTotalHits() > 0);
+      List<FacetResult> facetResults = fc.getFacetResults();
+      assertEquals(1, facetResults.size());
+      FacetResultNode rootNode = facetResults.get(0).getFacetResultNode();
+      assertEquals("wrong count for " + dim, expectedCounts.get(dim).intValue(), (int) rootNode.getValue());
+    }
+  }
+  
+  private void verifyIndexOrdinals(DirectoryReader indexReader, TaxonomyReader taxoReader, FacetIndexingParams fip) 
+      throws IOException {
+    // verify that the ordinals in the index match the ones in the taxonomy, and vice versa
+    
+    // collect all fields which have DocValues, to assert later that all were
+    // visited i.e. that during migration we didn't add FieldInfos with no
+    // DocValues
+    HashSet<String> docValuesFields = new HashSet<String>();
+    for (AtomicReaderContext context : indexReader.leaves()) {
+      FieldInfos infos = context.reader().getFieldInfos();
+      for (FieldInfo info : infos) {
+        if (info.hasDocValues()) {
+          docValuesFields.add(info.name);
+        }
+      }
+    }
+    
+    // check that all visited ordinals are found in the taxonomy and vice versa
+    boolean[] foundOrdinals = new boolean[taxoReader.getSize()];
+    for (int i = 0; i < foundOrdinals.length; i++) {
+      foundOrdinals[i] = false; // init to be on the safe side
+    }
+    foundOrdinals[0] = true; // ROOT ordinals isn't indexed
+    // mark 'dummy' category ordinal as seen
+    int dummyOrdinal = taxoReader.getOrdinal(new CategoryPath("dummy"));
+    if (dummyOrdinal > 0) {
+      foundOrdinals[dummyOrdinal] = true;
+    }
+    
+    int partitionSize = fip.getPartitionSize();
+    int numPartitions = (int) Math.ceil(taxoReader.getSize() / (double) partitionSize);
+    final IntsRef ordinals = new IntsRef(32);
+    for (String dim : DIMENSIONS) {
+      CategoryListParams clp = fip.getCategoryListParams(new CategoryPath(dim));
+      int partitionOffset = 0;
+      for (int partition = 0; partition < numPartitions; partition++, partitionOffset += partitionSize) {
+        final CategoryListIterator cli = clp.createCategoryListIterator(partition);
+        for (AtomicReaderContext context : indexReader.leaves()) {
+          if (cli.setNextReader(context)) { // not all fields may exist in all segments
+            // remove that field from the list of DocValues fields
+            docValuesFields.remove(clp.field + PartitionsUtils.partitionName(partition));
+            int maxDoc = context.reader().maxDoc();
+            for (int doc = 0; doc < maxDoc; doc++) {
+              cli.getOrdinals(doc, ordinals);
+              for (int j = 0; j < ordinals.length; j++) {
+                // verify that the ordinal is recognized by the taxonomy
+                int ordinal = ordinals.ints[j] + partitionOffset;
+                assertTrue("should not have received dummy ordinal (" + dummyOrdinal + ")", dummyOrdinal != ordinal);
+                assertNotNull("missing category for ordinal " + ordinal, taxoReader.getPath(ordinal));
+                foundOrdinals[ordinal] = true;
+              }
+            }
+          }
+        }
+      }
+    }
+    
+    assertTrue("some fields which have docValues were not visited: " + docValuesFields, docValuesFields.isEmpty());
+    
+    for (int i = 0; i < foundOrdinals.length; i++) {
+      assertTrue("ordinal " + i + " not visited", foundOrdinals[i]);
+    }
+  }
+  
+  private void doTestMigration(final int partitionSize) throws Exception {
+    // create a facets index with PayloadFacetFields and check it after migration
+    Directory indexDir = newDirectory();
+    Directory taxoDir = newDirectory();
+    
+    // set custom CLP fields for two dimensions and use the default ($facets) for the other two
+    HashMap<CategoryPath,CategoryListParams> params = new HashMap<CategoryPath,CategoryListParams>();
+    params.put(new CategoryPath(DIMENSIONS[0]), new CategoryListParams(DIMENSIONS[0]));
+    params.put(new CategoryPath(DIMENSIONS[1]), new CategoryListParams(DIMENSIONS[1]));
+    FacetIndexingParams fip = new PerDimensionIndexingParams(params) {
+      @Override
+      public int getPartitionSize() {
+        return partitionSize;
+      }
+    };
+    
+    HashMap<String,Integer> expectedCounts = createIndex(indexDir, taxoDir, fip);
+    migrateIndex(indexDir, fip);
+    verifyMigratedIndex(indexDir, taxoDir, expectedCounts, fip);
+    
+    IOUtils.close(indexDir, taxoDir);
+  }
+  
+  @Test
+  public void testMigration() throws Exception {
+    doTestMigration(Integer.MAX_VALUE);
+  }
+  
+  @Test
+  public void testMigrationWithPartitions() throws Exception {
+    doTestMigration(2);
+  }
+  
+}

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/CategoryListParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/CategoryListParamsTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/CategoryListParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/CategoryListParamsTest.java Wed Jan 16 09:43:51 2013
@@ -1,10 +1,7 @@
 package org.apache.lucene.facet.index.params;
 
-import org.apache.lucene.index.Term;
-import org.junit.Test;
-
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.junit.Test;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -28,7 +25,7 @@ public class CategoryListParamsTest exte
   @Test
   public void testDefaultSettings() {
     CategoryListParams clp = new CategoryListParams();
-    assertEquals("wrong default term", new Term("$facets", "$fulltree$"), clp.getTerm());
+    assertEquals("wrong default field", "$facets", clp.field);
     assertEquals("unexpected default encoder", "Sorting (Unique (DGap (VInt8)))", clp.createEncoder().toString());
     assertEquals("unexpected default decoder", "DGap (VInt8)", clp.createEncoder().createMatchingDecoder().toString());
   }
@@ -64,8 +61,8 @@ public class CategoryListParamsTest exte
         clParams1.hashCode(), clParams2.hashCode());
 
     // Test 2 CategoryListParams with the same specified Term
-    clParams1 = new CategoryListParams(new Term("test"));
-    clParams2 = new CategoryListParams(new Term("test"));
+    clParams1 = new CategoryListParams("test");
+    clParams2 = new CategoryListParams("test");
     assertEquals(
         "2 CategoryListParams with the same term should equal each other.",
         clParams1, clParams2);
@@ -73,8 +70,8 @@ public class CategoryListParamsTest exte
         clParams1.hashCode(), clParams2.hashCode());
     
     // Test 2 CategoryListParams with DIFFERENT terms
-    clParams1 = new CategoryListParams(new Term("test1"));
-    clParams2 = new CategoryListParams(new Term("test2"));
+    clParams1 = new CategoryListParams("test1");
+    clParams2 = new CategoryListParams("test2");
     assertFalse(
         "2 CategoryListParams with the different terms should NOT equal each other.",
         clParams1.equals(clParams2));

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/FacetIndexingParamsTest.java Wed Jan 16 09:43:51 2013
@@ -35,8 +35,7 @@ public class FacetIndexingParamsTest ext
     assertNotNull("Missing default category list", dfip.getAllCategoryListParams());
     assertEquals("all categories have the same CategoryListParams by default",
         dfip.getCategoryListParams(null), dfip.getCategoryListParams(new CategoryPath("a")));
-    assertEquals("Expected default category list term is $facets:$fulltree$",
-        new Term("$facets", "$fulltree$"), dfip.getCategoryListParams(null).getTerm());
+    assertEquals("Expected default category list field is $facets", "$facets", dfip.getCategoryListParams(null).field);
     String expectedDDText = "a"
         + dfip.getFacetDelimChar() + "b";
     CategoryPath cp = new CategoryPath("a", "b");
@@ -48,13 +47,13 @@ public class FacetIndexingParamsTest ext
     assertEquals("wrong drill-down term text", expectedDDText, new String(
         buf, 0, numchars));
     CategoryListParams clParams = dfip.getCategoryListParams(null);
-    assertEquals("partition for all ordinals is the first", "$fulltree$", 
-        PartitionsUtils.partitionNameByOrdinal(dfip, clParams , 250));
+    assertEquals("partition for all ordinals is the first", "", 
+        PartitionsUtils.partitionNameByOrdinal(dfip, 250));
     assertEquals("for partition 0, the same name should be returned",
-        "$fulltree$", PartitionsUtils.partitionName(clParams, 0));
+        "", PartitionsUtils.partitionName(0));
     assertEquals(
         "for any other, it's the concatenation of name + partition",
-        "$fulltree$1", PartitionsUtils.partitionName(clParams, 1));
+        PartitionsUtils.PART_NAME_PREFIX + "1", PartitionsUtils.partitionName(1));
     assertEquals("default partition number is always 0", 0, 
         PartitionsUtils.partitionNumber(dfip,100));
     assertEquals("default partition size is unbounded", Integer.MAX_VALUE,
@@ -63,11 +62,9 @@ public class FacetIndexingParamsTest ext
 
   @Test
   public void testCategoryListParamsWithDefaultIndexingParams() {
-    CategoryListParams clp = new CategoryListParams(
-        new Term("clp", "value"));
+    CategoryListParams clp = new CategoryListParams("clp");
     FacetIndexingParams dfip = new FacetIndexingParams(clp);
-    assertEquals("Expected default category list term is " + clp.getTerm(),
-        clp.getTerm(), dfip.getCategoryListParams(null).getTerm());
+    assertEquals("Expected default category list field is " + clp.field, clp.field, dfip.getCategoryListParams(null).field);
   }
 
   @Test

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/PerDimensionIndexingParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/PerDimensionIndexingParamsTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/PerDimensionIndexingParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/params/PerDimensionIndexingParamsTest.java Wed Jan 16 09:43:51 2013
@@ -32,44 +32,31 @@ public class PerDimensionIndexingParamsT
   public void testTopLevelSettings() {
     FacetIndexingParams ifip = new PerDimensionIndexingParams(Collections.<CategoryPath, CategoryListParams>emptyMap());
     assertNotNull("Missing default category list", ifip.getAllCategoryListParams());
-    assertEquals(
-        "Expected default category list term is $facets:$fulltree$",
-        new Term("$facets", "$fulltree$"), ifip.getCategoryListParams(
-            null).getTerm());
-    String expectedDDText = "a"
-        + ifip.getFacetDelimChar() + "b";
+    assertEquals("Expected default category list field is $facets", "$facets", ifip.getCategoryListParams(null).field);
+    String expectedDDText = "a" + ifip.getFacetDelimChar() + "b";
     CategoryPath cp = new CategoryPath("a", "b");
-    assertEquals("wrong drill-down term", new Term("$facets",
-        expectedDDText), DrillDown.term(ifip,cp));
+    assertEquals("wrong drill-down term", new Term("$facets", expectedDDText), DrillDown.term(ifip,cp));
     char[] buf = new char[20];
     int numchars = ifip.drillDownTermText(cp, buf);
     assertEquals("3 characters should be written", 3, numchars);
-    assertEquals("wrong drill-down term text", expectedDDText, new String(
-        buf, 0, numchars));
+    assertEquals("wrong drill-down term text", expectedDDText, new String(buf, 0, numchars));
     
     CategoryListParams clParams = ifip.getCategoryListParams(null);
-    assertEquals("partition for all ordinals is the first", "$fulltree$", 
-        PartitionsUtils.partitionNameByOrdinal(ifip, clParams , 250));
-    assertEquals("for partition 0, the same name should be returned",
-        "$fulltree$", PartitionsUtils.partitionName(clParams, 0));
-    assertEquals(
-        "for any other, it's the concatenation of name + partition",
-        "$fulltree$1", PartitionsUtils.partitionName(clParams, 1));
-    assertEquals("default partition number is always 0", 0, 
-        PartitionsUtils.partitionNumber(ifip,100));
-    
-    assertEquals("default partition size is unbounded", Integer.MAX_VALUE,
-        ifip.getPartitionSize());
+    assertEquals("partition for all ordinals is the first", "", PartitionsUtils.partitionNameByOrdinal(ifip, 250));
+    assertEquals("for partition 0, the same name should be returned", "", PartitionsUtils.partitionName(0));
+    assertEquals("for any other, it's the concatenation of name + partition", PartitionsUtils.PART_NAME_PREFIX + "1", PartitionsUtils.partitionName(1));
+    assertEquals("default partition number is always 0", 0, PartitionsUtils.partitionNumber(ifip,100));
+    assertEquals("default partition size is unbounded", Integer.MAX_VALUE, ifip.getPartitionSize());
   }
 
   @Test
   public void testCategoryListParamsAddition() {
-    CategoryListParams clp = new CategoryListParams(new Term("clp", "value"));
+    CategoryListParams clp = new CategoryListParams("clp");
     PerDimensionIndexingParams tlfip = new PerDimensionIndexingParams(
         Collections.<CategoryPath,CategoryListParams> singletonMap(new CategoryPath("a"), clp));
-    assertEquals("Expected category list term is " + clp.getTerm(), 
-        clp.getTerm(), tlfip.getCategoryListParams(new CategoryPath("a")).getTerm());
-    assertNotSame("Unexpected default category list " + clp.getTerm(), clp, tlfip.getCategoryListParams(null));
+    assertEquals("Expected category list field is " + clp.field, 
+        clp.field, tlfip.getCategoryListParams(new CategoryPath("a")).field);
+    assertNotSame("Unexpected default category list " + clp.field, clp, tlfip.getCategoryListParams(null));
   }
 
 }
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java Wed Jan 16 09:43:51 2013
@@ -1,23 +1,15 @@
 package org.apache.lucene.facet.search;
 
-import java.io.IOException;
-import java.io.Reader;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.TextField;
+import org.apache.lucene.document.StraightBytesDocValuesField;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -48,42 +40,6 @@ import org.junit.Test;
 
 public class CategoryListIteratorTest extends LuceneTestCase {
 
-  private static final class DataTokenStream extends TokenStream {
-
-    private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
-    private final BytesRef buf;
-    private final IntEncoder encoder;
-    private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
-    
-    private int idx;
-    private boolean exhausted = false;
-
-    public DataTokenStream(String text, IntEncoder encoder) {
-      this.encoder = encoder;
-      term.setEmpty().append(text);
-      buf = new BytesRef();
-      payload.setPayload(buf);
-    }
-
-    public void setIdx(int idx) {
-      this.idx = idx;
-      exhausted = false;
-    }
-
-    @Override
-    public boolean incrementToken() throws IOException {
-      if (exhausted) {
-        return false;
-      }
-
-      // must copy because encoders may change the buffer
-      encoder.encode(IntsRef.deepCopyOf(data[idx]), buf);
-      exhausted = true;
-      return true;
-    }
-
-  }
-
   static final IntsRef[] data = new IntsRef[] {
     new IntsRef(new int[] { 1, 2 }, 0, 2), 
     new IntsRef(new int[] { 3, 4 }, 0, 2),
@@ -95,13 +51,13 @@ public class CategoryListIteratorTest ex
   public void testPayloadCategoryListIteraor() throws Exception {
     Directory dir = newDirectory();
     final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
-    DataTokenStream dts = new DataTokenStream("1",encoder);
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT, 
         new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
+    BytesRef buf = new BytesRef();
     for (int i = 0; i < data.length; i++) {
-      dts.setIdx(i);
       Document doc = new Document();
-      doc.add(new TextField("f", dts));
+      encoder.encode(IntsRef.deepCopyOf(data[i]), buf);
+      doc.add(new StraightBytesDocValuesField("f", buf));
       writer.addDocument(doc);
     }
     IndexReader reader = writer.getReader();
@@ -109,9 +65,9 @@ public class CategoryListIteratorTest ex
 
     int totalCategories = 0;
     IntsRef ordinals = new IntsRef();
-    CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
+    CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
     for (AtomicReaderContext context : reader.leaves()) {
-      cli.setNextReader(context);
+      assertTrue("failed to initalize iterator", cli.setNextReader(context));
       int maxDoc = context.reader().maxDoc();
       int dataIdx = context.docBase;
       for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
@@ -136,24 +92,17 @@ public class CategoryListIteratorTest ex
   public void testPayloadIteratorWithInvalidDoc() throws Exception {
     Directory dir = newDirectory();
     final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
-    DataTokenStream dts = new DataTokenStream("1", encoder);
-    // this test requires that no payloads ever be randomly present!
-    final Analyzer noPayloadsAnalyzer = new Analyzer() {
-      @Override
-      public TokenStreamComponents createComponents(String fieldName, Reader reader) {
-        return new TokenStreamComponents(new MockTokenizer(reader, MockTokenizer.KEYWORD, false));
-      }
-    };
     // NOTE: test is wired to LogMP... because test relies on certain docids having payloads
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, 
-        newIndexWriterConfig(TEST_VERSION_CURRENT, noPayloadsAnalyzer).setMergePolicy(newLogMergePolicy()));
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
     for (int i = 0; i < data.length; i++) {
       Document doc = new Document();
       if (i == 0) {
-        dts.setIdx(i);
-        doc.add(new TextField("f", dts)); // only doc 0 has payloads!
+        BytesRef buf = new BytesRef();
+        encoder.encode(IntsRef.deepCopyOf(data[i]), buf );
+        doc.add(new StraightBytesDocValuesField("f", buf));
       } else {
-        doc.add(new TextField("f", "1", Field.Store.NO));
+        doc.add(new StraightBytesDocValuesField("f", new BytesRef()));
       }
       writer.addDocument(doc);
       writer.commit();
@@ -164,9 +113,9 @@ public class CategoryListIteratorTest ex
 
     int totalCategories = 0;
     IntsRef ordinals = new IntsRef();
-    CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
+    CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
     for (AtomicReaderContext context : reader.leaves()) {
-      cli.setNextReader(context);
+      assertTrue("failed to initalize iterator", cli.setNextReader(context));
       int maxDoc = context.reader().maxDoc();
       int dataIdx = context.docBase;
       for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
@@ -176,13 +125,13 @@ public class CategoryListIteratorTest ex
         }
         cli.getOrdinals(doc, ordinals);
         if (dataIdx == 0) {
-          assertTrue("document 0 must have a payload", ordinals.length > 0);
+          assertTrue("document 0 must have ordinals", ordinals.length > 0);
           for (int j = 0; j < ordinals.length; j++) {
             assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
           }
           totalCategories += ordinals.length;
         } else {
-          assertTrue("only document 0 should have a payload", ordinals.length == 0);
+          assertTrue("only document 0 should have ordinals", ordinals.length == 0);
         }
       }
     }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java Wed Jan 16 09:43:51 2013
@@ -60,8 +60,8 @@ public class DrillDownTest extends Lucen
   
   public DrillDownTest() {
     Map<CategoryPath,CategoryListParams> paramsMap = new HashMap<CategoryPath,CategoryListParams>();
-    paramsMap.put(new CategoryPath("a"), new CategoryListParams(new Term("testing_facets_a", "a")));
-    paramsMap.put(new CategoryPath("b"), new CategoryListParams(new Term("testing_facets_b", "b")));
+    paramsMap.put(new CategoryPath("a"), new CategoryListParams("testing_facets_a"));
+    paramsMap.put(new CategoryPath("b"), new CategoryListParams("testing_facets_b"));
     nonDefaultParams = new PerDimensionIndexingParams(paramsMap);
   }
 
@@ -113,8 +113,8 @@ public class DrillDownTest extends Lucen
   }
   
   @Test
-  public void testTermDefault() {
-    String defaultField = CategoryListParams.DEFAULT_TERM.field();
+  public void testDefaultField() {
+    String defaultField = CategoryListParams.DEFAULT_FIELD;
     
     Term termA = DrillDown.term(defaultParams, new CategoryPath("a"));
     assertEquals(new Term(defaultField, "a"), termA);

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java Wed Jan 16 09:43:51 2013
@@ -10,14 +10,18 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
 import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
 import org.apache.lucene.facet.search.params.FacetRequest;
-import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.params.FacetRequest.ResultMode;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.FacetResultNode;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
@@ -25,23 +29,19 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiCollector;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 import org.junit.Test;
 
 /*
@@ -63,6 +63,18 @@ import org.junit.Test;
 
 public class TestMultipleCategoryLists extends LuceneTestCase {
 
+  private static final CategoryPath[] CATEGORIES = new CategoryPath[] {
+    new CategoryPath("Author", "Mark Twain"),
+    new CategoryPath("Author", "Stephen King"),
+    new CategoryPath("Author", "Kurt Vonnegut"),
+    new CategoryPath("Band", "Rock & Pop", "The Beatles"),
+    new CategoryPath("Band", "Punk", "The Ramones"),
+    new CategoryPath("Band", "Rock & Pop", "U2"),
+    new CategoryPath("Band", "Rock & Pop", "REM"),
+    new CategoryPath("Band", "Rock & Pop", "Dave Matthews Band"),
+    new CategoryPath("Composer", "Bach"),
+  };
+  
   @Test
   public void testDefault() throws Exception {
     Directory[][] dirs = getDirs();
@@ -72,9 +84,6 @@ public class TestMultipleCategoryLists e
     // create and open a taxonomy writer
     TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1], OpenMode.CREATE);
 
-    /**
-     * Configure with no custom counting lists
-     */
     PerDimensionIndexingParams iParams = new PerDimensionIndexingParams(Collections.<CategoryPath, CategoryListParams>emptyMap());
 
     seedIndex(iw, tw, iParams);
@@ -88,19 +97,14 @@ public class TestMultipleCategoryLists e
     // prepare searcher to search against
     IndexSearcher searcher = newSearcher(ir);
 
-    FacetsCollector facetsCollector = performSearch(iParams, tr, ir,
-        searcher);
+    FacetsCollector facetsCollector = performSearch(iParams, tr, ir, searcher);
 
     // Obtain facets results and hand-test them
     assertCorrectResults(facetsCollector);
 
-    DocsEnum td = _TestUtil.docs(random(), ir, "$facets", new BytesRef("$fulltree$"), MultiFields.getLiveDocs(ir), null, DocsEnum.FLAG_NONE);
-    assertTrue(td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
+    assertOrdinalsExist("$facets", ir);
 
-    tr.close();
-    ir.close();
-    iw.close();
-    tw.close();
+    IOUtils.close(tr, ir, iw, tw);
     IOUtils.close(dirs[0]);
   }
 
@@ -111,12 +115,10 @@ public class TestMultipleCategoryLists e
     RandomIndexWriter iw = new RandomIndexWriter(random(), dirs[0][0], newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
     // create and open a taxonomy writer
-    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1],
-        OpenMode.CREATE);
+    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1], OpenMode.CREATE);
 
     PerDimensionIndexingParams iParams = new PerDimensionIndexingParams(
-        Collections.singletonMap(new CategoryPath("Author"),
-            new CategoryListParams(new Term("$author", "Authors"))));
+        Collections.singletonMap(new CategoryPath("Author"), new CategoryListParams("$author")));
     seedIndex(iw, tw, iParams);
 
     IndexReader ir = iw.getReader();
@@ -133,13 +135,10 @@ public class TestMultipleCategoryLists e
     // Obtain facets results and hand-test them
     assertCorrectResults(facetsCollector);
 
-    assertPostingListExists("$facets", "$fulltree$", ir);
-    assertPostingListExists("$author", "Authors", ir);
+    assertOrdinalsExist("$facets", ir);
+    assertOrdinalsExist("$author", ir);
 
-    tr.close();
-    ir.close();
-    iw.close();
-    tw.close();
+    IOUtils.close(tr, ir, iw, tw);
     IOUtils.close(dirs[0]);
   }
 
@@ -150,12 +149,11 @@ public class TestMultipleCategoryLists e
     RandomIndexWriter iw = new RandomIndexWriter(random(), dirs[0][0], newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
     // create and open a taxonomy writer
-    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1],
-        OpenMode.CREATE);
+    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1], OpenMode.CREATE);
 
     Map<CategoryPath,CategoryListParams> paramsMap = new HashMap<CategoryPath,CategoryListParams>();
-    paramsMap.put(new CategoryPath("Band"), new CategoryListParams(new Term("$music", "Bands")));
-    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams(new Term("$music", "Composers")));
+    paramsMap.put(new CategoryPath("Band"), new CategoryListParams("$music"));
+    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams("$music"));
     PerDimensionIndexingParams iParams = new PerDimensionIndexingParams(paramsMap);
     seedIndex(iw, tw, iParams);
 
@@ -168,26 +166,27 @@ public class TestMultipleCategoryLists e
     // prepare searcher to search against
     IndexSearcher searcher = newSearcher(ir);
 
-    FacetsCollector facetsCollector = performSearch(iParams, tr, ir,
-        searcher);
+    FacetsCollector facetsCollector = performSearch(iParams, tr, ir, searcher);
 
     // Obtain facets results and hand-test them
     assertCorrectResults(facetsCollector);
 
-    assertPostingListExists("$facets", "$fulltree$", ir);
-    assertPostingListExists("$music", "Bands", ir);
-    assertPostingListExists("$music", "Composers", ir);
-
-    tr.close();
-    ir.close();
-    iw.close();
-    tw.close();
+    assertOrdinalsExist("$facets", ir);
+    assertOrdinalsExist("$music", ir);
+    assertOrdinalsExist("$music", ir);
+
+    IOUtils.close(tr, ir, iw, tw);
     IOUtils.close(dirs[0]);
   }
 
-  private void assertPostingListExists(String field, String text, IndexReader ir) throws IOException {
-    DocsEnum de = _TestUtil.docs(random(), ir, field, new BytesRef(text), null, null, DocsEnum.FLAG_NONE);
-    assertTrue(de.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
+  private void assertOrdinalsExist(String field, IndexReader ir) throws IOException {
+    for (AtomicReaderContext context : ir.leaves()) {
+      AtomicReader r = context.reader();
+      if (r.docValues(field) != null) {
+        return; // not all segments must have this DocValues
+      }
+    }
+    fail("no ordinals found for " + field);
   }
 
   @Test
@@ -200,8 +199,8 @@ public class TestMultipleCategoryLists e
     TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1], OpenMode.CREATE);
 
     Map<CategoryPath,CategoryListParams> paramsMap = new HashMap<CategoryPath,CategoryListParams>();
-    paramsMap.put(new CategoryPath("Band"), new CategoryListParams(new Term("$bands", "Bands")));
-    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams(new Term("$composers", "Composers")));
+    paramsMap.put(new CategoryPath("Band"), new CategoryListParams("$bands"));
+    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams("$composers"));
     PerDimensionIndexingParams iParams = new PerDimensionIndexingParams(paramsMap);
     seedIndex(iw, tw, iParams);
 
@@ -214,18 +213,15 @@ public class TestMultipleCategoryLists e
     // prepare searcher to search against
     IndexSearcher searcher = newSearcher(ir);
 
-    FacetsCollector facetsCollector = performSearch(iParams, tr, ir,
-        searcher);
+    FacetsCollector facetsCollector = performSearch(iParams, tr, ir, searcher);
 
     // Obtain facets results and hand-test them
     assertCorrectResults(facetsCollector);
-    assertPostingListExists("$facets", "$fulltree$", ir);
-    assertPostingListExists("$bands", "Bands", ir);
-    assertPostingListExists("$composers", "Composers", ir);
-    tr.close();
-    ir.close();
-    iw.close();
-    tw.close();
+    assertOrdinalsExist("$facets", ir);
+    assertOrdinalsExist("$bands", ir);
+    assertOrdinalsExist("$composers", ir);
+
+    IOUtils.close(tr, ir, iw, tw);
     IOUtils.close(dirs[0]);
   }
 
@@ -236,13 +232,12 @@ public class TestMultipleCategoryLists e
     RandomIndexWriter iw = new RandomIndexWriter(random(), dirs[0][0], newIndexWriterConfig(
         TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)));
     // create and open a taxonomy writer
-    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1],
-        OpenMode.CREATE);
+    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dirs[0][1], OpenMode.CREATE);
 
     Map<CategoryPath,CategoryListParams> paramsMap = new HashMap<CategoryPath,CategoryListParams>();
-    paramsMap.put(new CategoryPath("Band"), new CategoryListParams(new Term("$music", "music")));
-    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams(new Term("$music", "music")));
-    paramsMap.put(new CategoryPath("Author"), new CategoryListParams(new Term("$literature", "Authors")));
+    paramsMap.put(new CategoryPath("Band"), new CategoryListParams("$music"));
+    paramsMap.put(new CategoryPath("Composer"), new CategoryListParams("$music"));
+    paramsMap.put(new CategoryPath("Author"), new CategoryListParams("$literature"));
     PerDimensionIndexingParams iParams = new PerDimensionIndexingParams(paramsMap);
 
     seedIndex(iw, tw, iParams);
@@ -256,18 +251,14 @@ public class TestMultipleCategoryLists e
     // prepare searcher to search against
     IndexSearcher searcher = newSearcher(ir);
 
-    FacetsCollector facetsCollector = performSearch(iParams, tr, ir,
-        searcher);
+    FacetsCollector facetsCollector = performSearch(iParams, tr, ir, searcher);
 
     // Obtain facets results and hand-test them
     assertCorrectResults(facetsCollector);
-    assertPostingListExists("$music", "music", ir);
-    assertPostingListExists("$literature", "Authors", ir);
+    assertOrdinalsExist("$music", ir);
+    assertOrdinalsExist("$literature", ir);
 
-    tr.close();
-    ir.close();
-    iw.close();
-    tw.close();
+    IOUtils.close(tr, ir, iw, tw);
     IOUtils.close(dirs[0]);
   }
 
@@ -275,14 +266,12 @@ public class TestMultipleCategoryLists e
     return FacetTestUtils.createIndexTaxonomyDirs(1);
   }
 
-  private void assertCorrectResults(FacetsCollector facetsCollector)
-  throws IOException {
+  private void assertCorrectResults(FacetsCollector facetsCollector) throws IOException {
     List<FacetResult> res = facetsCollector.getFacetResults();
 
     FacetResult results = res.get(0);
     FacetResultNode resNode = results.getFacetResultNode();
-    Iterable<? extends FacetResultNode> subResults = resNode
-    .getSubResults();
+    Iterable<? extends FacetResultNode> subResults = resNode.getSubResults();
     Iterator<? extends FacetResultNode> subIter = subResults.iterator();
 
     checkResult(resNode, "Band", 5.0);
@@ -325,9 +314,8 @@ public class TestMultipleCategoryLists e
     checkResult(subIter.next(), "Band/Rock & Pop/The Beatles", 1.0);
   }
 
-  private FacetsCollector performSearch(FacetIndexingParams iParams,
-                                        TaxonomyReader tr, IndexReader ir,
-                                        IndexSearcher searcher) throws IOException {
+  private FacetsCollector performSearch(FacetIndexingParams iParams, TaxonomyReader tr, IndexReader ir, 
+      IndexSearcher searcher) throws IOException {
     // step 1: collect matching documents into a collector
     Query q = new MatchAllDocsQuery();
     TopScoreDocCollector topDocsCollector = TopScoreDocCollector.create(10, true);
@@ -344,7 +332,6 @@ public class TestMultipleCategoryLists e
 
     // Faceted search parameters indicate which facets are we interested in
     FacetSearchParams facetSearchParams = new FacetSearchParams(facetRequests, iParams);
-    
 
     // perform documents search and facets accumulation
     FacetsCollector facetsCollector = new FacetsCollector(facetSearchParams, ir, tr);
@@ -352,27 +339,19 @@ public class TestMultipleCategoryLists e
     return facetsCollector;
   }
 
-  private void seedIndex(RandomIndexWriter iw, TaxonomyWriter tw,
-                          FacetIndexingParams iParams) throws IOException {
-    FacetTestUtils.add(iParams, iw, tw, "Author", "Mark Twain");
-    FacetTestUtils.add(iParams, iw, tw, "Author", "Stephen King");
-    FacetTestUtils.add(iParams, iw, tw, "Author", "Kurt Vonnegut");
-    FacetTestUtils.add(iParams, iw, tw, "Band", "Rock & Pop",
-    "The Beatles");
-    FacetTestUtils.add(iParams, iw, tw, "Band", "Punk", "The Ramones");
-    FacetTestUtils.add(iParams, iw, tw, "Band", "Rock & Pop", "U2");
-    FacetTestUtils.add(iParams, iw, tw, "Band", "Rock & Pop", "REM");
-    FacetTestUtils.add(iParams, iw, tw, "Band", "Rock & Pop",
-    "Dave Matthews Band");
-    FacetTestUtils.add(iParams, iw, tw, "Composer", "Bach");
+  private void seedIndex(RandomIndexWriter iw, TaxonomyWriter tw, FacetIndexingParams iParams) throws IOException {
+    FacetFields facetFields = new FacetFields(tw, iParams);
+    for (CategoryPath cp : CATEGORIES) {
+      Document doc = new Document();
+      facetFields.addFields(doc, Collections.singletonList(cp));
+      doc.add(new TextField("content", "alpha", Field.Store.YES));
+      iw.addDocument(doc);
+    }
   }
 
   private static void checkResult(FacetResultNode sub, String label, double value) {
-    assertEquals("Label of subresult " + sub.getLabel() + " was incorrect",
-        label, sub.getLabel().toString());
-    assertEquals(
-        "Value for " + sub.getLabel() + " subresult was incorrect",
-        value, sub.getValue(), 0.0);
+    assertEquals("Label of subresult " + sub.getLabel() + " was incorrect", label, sub.getLabel().toString());
+    assertEquals("Value for " + sub.getLabel() + " subresult was incorrect", value, sub.getValue(), 0.0);
   }
 
 }
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java Wed Jan 16 09:43:51 2013
@@ -93,6 +93,7 @@ public class TestStandardFacetsAccumulat
     indexTwoDocs(indexWriter, null, false);        // 4th segment, no content, or categories
     indexTwoDocs(indexWriter, null, true);         // 5th segment, with content, no categories
     indexTwoDocs(indexWriter, facetFields, true);  // 6th segment, with content, with categories
+    indexTwoDocs(indexWriter, null, true);         // 7th segment, with content, no categories
     IOUtils.close(indexWriter, taxoWriter);
 
     DirectoryReader indexReader = DirectoryReader.open(indexDir);

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java Wed Jan 16 09:43:51 2013
@@ -178,7 +178,7 @@ public class TestTopKInEachNodeResultHan
       }
       
       FacetResult fr = facetResults.get(0); // a, depth=3, K=2
-      boolean hasDoctor = "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      boolean hasDoctor = "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(9, fr.getNumValidDescendants());
       FacetResultNode parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -219,7 +219,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(1); // a, depth=2, K=2. same result as before
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(9, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -239,7 +239,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(2); // a, depth=1, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(4, fr.getNumValidDescendants(), 4);
       parentRes = fr.getFacetResultNode();
       assertEquals(16.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -257,7 +257,7 @@ public class TestTopKInEachNodeResultHan
       }
       
       fr = facetResults.get(3); // a/b, depth=3, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -272,7 +272,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(4); // a/b, depth=2, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -286,7 +286,7 @@ public class TestTopKInEachNodeResultHan
       }
 
       fr = facetResults.get(5); // a/b, depth=1, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(4, fr.getNumValidDescendants());
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
@@ -300,13 +300,13 @@ public class TestTopKInEachNodeResultHan
       }
       
       fr = facetResults.get(6); // a/b, depth=0, K=2
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
       assertEquals(0, fr.getNumValidDescendants()); // 0 descendants but rootnode
       parentRes = fr.getFacetResultNode();
       assertEquals(8.0, parentRes.getValue(), Double.MIN_VALUE);
       assertEquals(0.0, parentRes.getResidue(), Double.MIN_VALUE);
       assertEquals(0, parentRes.getNumSubResults());
-      hasDoctor |= "Doctor".equals(fr.getFacetRequest().getCategoryPath().components[0]);
+      hasDoctor |= "Doctor".equals(fr.getFacetRequest().categoryPath.components[0]);
 
       // doctor, depth=1, K=2
       assertFalse("Shouldn't have found anything for a FacetRequest " +

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCounts.java Wed Jan 16 09:43:51 2013
@@ -85,12 +85,12 @@ public class TestTotalFacetCounts extend
 
     TotalFacetCountsCache tfcc = TotalFacetCountsCache.getSingleton();
     File tmpFile = _TestUtil.createTempFile("test", "tmp", TEMP_DIR);
-    tfcc.store(tmpFile, readers[0].indexReader, readers[0].taxReader, iParams, null);
+    tfcc.store(tmpFile, readers[0].indexReader, readers[0].taxReader, iParams);
     tfcc.clear(); // not really required because TFCC overrides on load(), but in the test we need not rely on this.
     tfcc.load(tmpFile, readers[0].indexReader, readers[0].taxReader, iParams);
     
     // now retrieve the one just loaded
-    TotalFacetCounts totalCounts = tfcc.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    TotalFacetCounts totalCounts = tfcc.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
 
     int partition = 0;
     for (int i=0; i<expectedCounts.length; i+=partitionSize) {

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTotalFacetCountsCache.java Wed Jan 16 09:43:51 2013
@@ -78,7 +78,7 @@ public class TestTotalFacetCountsCache e
     @Override
     public void run() {
       try {
-        tfc = TFC.getTotalCounts(r, tr, iParams, null);
+        tfc = TFC.getTotalCounts(r, tr, iParams);
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -264,29 +264,29 @@ public class TestTotalFacetCountsCache e
     // As this is the first time we have invoked the TotalFacetCountsManager, 
     // we should expect to compute and not read from disk.
     TotalFacetCounts totalCounts = 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     int prevGen = assertRecomputed(totalCounts, 0, "after first attempt to get it!");
 
     // Repeating same operation should pull from the cache - not recomputed. 
     assertTrue("Should be obtained from cache at 2nd attempt",totalCounts == 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null));
+      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams));
 
     // Repeat the same operation as above. but clear first - now should recompute again
     initCache();
-    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts, prevGen, "after cache clear, 3rd attempt to get it!");
     
     //store to file
     File outputFile = _TestUtil.createTempFile("test", "tmp", TEMP_DIR);
     initCache();
-    TFC.store(outputFile, readers[0].indexReader, readers[0].taxReader, iParams, null);
-    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    TFC.store(outputFile, readers[0].indexReader, readers[0].taxReader, iParams);
+    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts, prevGen, "after cache clear, 4th attempt to get it!");
 
     //clear and load
     initCache();
     TFC.load(outputFile, readers[0].indexReader, readers[0].taxReader, iParams);
-    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertReadFromDisc(totalCounts, prevGen, "after 5th attempt to get it!");
 
     // Add a new facet to the index, commit and refresh readers
@@ -306,12 +306,12 @@ public class TestTotalFacetCountsCache e
     readers[0].indexReader = r2;
 
     // now use the new reader - should recompute
-    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts, prevGen, "after updating the index - 7th attempt!");
 
     // try again - should not recompute
     assertTrue("Should be obtained from cache at 8th attempt",totalCounts == 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null));
+      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams));
     
     readers[0].close();
     outputFile.delete();
@@ -361,7 +361,7 @@ public class TestTotalFacetCountsCache e
 
     // Create TFC and write cache to disk
     File outputFile = _TestUtil.createTempFile("test", "tmp", TEMP_DIR);
-    TFC.store(outputFile, readers[0].indexReader, readers[0].taxReader, iParams, null);
+    TFC.store(outputFile, readers[0].indexReader, readers[0].taxReader, iParams);
     
     // Make the taxonomy grow without touching the index
     for (int i = 0; i < 10; i++) {
@@ -377,8 +377,7 @@ public class TestTotalFacetCountsCache e
 
     // With the bug, this next call should result in an exception
     TFC.load(outputFile, readers[0].indexReader, readers[0].taxReader, iParams);
-    TotalFacetCounts totalCounts = TFC.getTotalCounts(
-        readers[0].indexReader, readers[0].taxReader, iParams, null);
+    TotalFacetCounts totalCounts = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     assertReadFromDisc(totalCounts, 0, "after reading from disk.");
     outputFile.delete();
     writers[0].close();
@@ -467,28 +466,25 @@ public class TestTotalFacetCountsCache e
     // As this is the first time we have invoked the TotalFacetCountsManager, we
     // should expect to compute.
     TotalFacetCounts totalCounts0 = 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     int prevGen = -1;
     prevGen = assertRecomputed(totalCounts0, prevGen, "after attempt 1");
     assertTrue("attempt 1b for same input [0] shout find it in cache",
-        totalCounts0 == TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null));
+        totalCounts0 == TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams));
     
     // 2nd Reader - As this is the first time we have invoked the
     // TotalFacetCountsManager, we should expect a state of NEW to be returned.
-    TotalFacetCounts totalCounts1 = 
-      TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams, null);
+    TotalFacetCounts totalCounts1 = TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts1, prevGen, "after attempt 2");
     assertTrue("attempt 2b for same input [1] shout find it in cache",
-        totalCounts1 == TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams, null));
+        totalCounts1 == TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams));
 
     // Right now cache size is one, so first TFC is gone and should be recomputed  
-    totalCounts0 = 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    totalCounts0 = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts0, prevGen, "after attempt 3");
     
     // Similarly will recompute the second result  
-    totalCounts1 = 
-      TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams, null);
+    totalCounts1 = TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts1, prevGen, "after attempt 4");
 
     // Now we set the cache size to two, meaning both should exist in the
@@ -496,17 +492,15 @@ public class TestTotalFacetCountsCache e
     TFC.setCacheSize(2);
 
     // Re-compute totalCounts0 (was evicted from the cache when the cache was smaller)
-    totalCounts0 = 
-      TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null);
+    totalCounts0 = TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams);
     prevGen = assertRecomputed(totalCounts0, prevGen, "after attempt 5");
 
     // now both are in the larger cache and should not be recomputed 
-    totalCounts1 = TFC.getTotalCounts(readers[1].indexReader,
-        readers[1].taxReader, iParams, null);
+    totalCounts1 = TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams);
     assertTrue("with cache of size 2 res no. 0 should come from cache",
-        totalCounts0 == TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams, null));
+        totalCounts0 == TFC.getTotalCounts(readers[0].indexReader, readers[0].taxReader, iParams));
     assertTrue("with cache of size 2 res no. 1 should come from cache",
-        totalCounts1 == TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams, null));
+        totalCounts1 == TFC.getTotalCounts(readers[1].indexReader, readers[1].taxReader, iParams));
     
     writers[0].close();
     writers[1].close();

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetRequestTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetRequestTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetRequestTest.java Wed Jan 16 09:43:51 2013
@@ -32,12 +32,12 @@ public class FacetRequestTest extends Lu
 
   @Test(expected=IllegalArgumentException.class)
   public void testIllegalNumResults() throws Exception {
-    new CountFacetRequest(new CategoryPath("a", "b"), 0);
+    assertNotNull(new CountFacetRequest(new CategoryPath("a", "b"), 0));
   }
   
   @Test(expected=IllegalArgumentException.class)
   public void testIllegalCategoryPath() throws Exception {
-    new CountFacetRequest(null, 1);
+    assertNotNull(new CountFacetRequest(null, 1));
   }
 
   @Test

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetSearchParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetSearchParamsTest.java?rev=1433869&r1=1433868&r2=1433869&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetSearchParamsTest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetSearchParamsTest.java Wed Jan 16 09:43:51 2013
@@ -1,13 +1,5 @@
 package org.apache.lucene.facet.search.params;
 
-import org.apache.lucene.facet.index.params.FacetIndexingParams;
-import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
-import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
-import org.apache.lucene.facet.util.PartitionsUtils;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.junit.Test;
 
@@ -31,28 +23,6 @@ import org.junit.Test;
 public class FacetSearchParamsTest extends LuceneTestCase {
 
   @Test
-  public void testAddFacetRequest() throws Exception {
-    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("a", "b"), 1));
-    assertEquals("expected 1 facet request", 1, fsp.getFacetRequests().size());
-  }
-  
-  @Test
-  public void testPartitionSizeWithCategories() throws Exception {
-    Directory dir = newDirectory();
-    TaxonomyWriter tw = new DirectoryTaxonomyWriter(dir);
-    tw.addCategory(new CategoryPath("a"));
-    tw.commit();
-    tw.close();
-    TaxonomyReader tr = new DirectoryTaxonomyReader(dir);
-    assertEquals("unexpected partition offset for 1 categories", 2,
-        PartitionsUtils.partitionOffset(FacetIndexingParams.ALL_PARENTS, 1, tr));
-    assertEquals("unexpected partition size for 1 categories", 2,
-        PartitionsUtils.partitionSize(FacetIndexingParams.ALL_PARENTS,tr));
-    tr.close();
-    dir.close();
-  }
-  
-  @Test
   public void testSearchParamsWithNullRequest() throws Exception {
     try {
       assertNull(new FacetSearchParams());