You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/04/11 01:21:41 UTC
svn commit: r1312037 - in /lucene/dev/trunk: lucene/
lucene/contrib/memory/src/java/org/apache/lucene/index/memory/
lucene/core/src/java/org/apache/lucene/codecs/
lucene/core/src/java/org/apache/lucene/codecs/lucene3x/
lucene/core/src/java/org/apache/l...
Author: mikemccand
Date: Tue Apr 10 23:21:39 2012
New Revision: 1312037
URL: http://svn.apache.org/viewvc?rev=1312037&view=rev
Log:
LUCENE-3970: rename getUniqueTerm/FieldCount() to size()
Modified:
lucene/dev/trunk/lucene/CHANGES.txt
lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java
lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Apr 10 23:21:39 2012
@@ -259,6 +259,9 @@ Changes in backwards compatibility polic
* LUCENE-2000: clone() now returns covariant types where possible. (ryan)
+* LUCENE-3970: Rename Fields.getUniqueFieldCount -> .size() and
+ Terms.getUniqueTermCount -> .size(). (Iulius Curt via Mike McCandless)
+
Changes in Runtime Behavior
* LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
Modified: lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Tue Apr 10 23:21:39 2012
@@ -759,7 +759,7 @@ public class MemoryIndex {
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return info.sortedTerms.length;
}
@@ -785,7 +785,7 @@ public class MemoryIndex {
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return sortedFields.length;
}
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermsReader.java Tue Apr 10 23:21:39 2012
@@ -202,7 +202,7 @@ public class BlockTermsReader extends Fi
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return fields.size();
}
@@ -261,7 +261,7 @@ public class BlockTermsReader extends Fi
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return numTerms;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Tue Apr 10 23:21:39 2012
@@ -216,7 +216,7 @@ public class BlockTreeTermsReader extend
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return fields.size();
}
@@ -455,7 +455,7 @@ public class BlockTreeTermsReader extend
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return numTerms;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Tue Apr 10 23:21:39 2012
@@ -166,17 +166,17 @@ public abstract class TermVectorsWriter
/** Safe (but, slowish) default method to write every
* vector field in the document. This default
* implementation requires that the vectors implement
- * both Fields.getUniqueFieldCount and
- * Terms.getUniqueTermCount. */
+ * both Fields.size and
+ * Terms.size. */
protected final void addAllDocVectors(Fields vectors, FieldInfos fieldInfos) throws IOException {
if (vectors == null) {
startDocument(0);
return;
}
- final int numFields = vectors.getUniqueFieldCount();
+ final int numFields = vectors.size();
if (numFields == -1) {
- throw new IllegalStateException("vectors.getUniqueFieldCount() must be implemented (it returned -1)");
+ throw new IllegalStateException("vectors.size() must be implemented (it returned -1)");
}
startDocument(numFields);
@@ -195,9 +195,9 @@ public abstract class TermVectorsWriter
// FieldsEnum shouldn't lie...
continue;
}
- final int numTerms = (int) terms.getUniqueTermCount();
+ final int numTerms = (int) terms.size();
if (numTerms == -1) {
- throw new IllegalStateException("vector.getUniqueTermCount() must be implemented (it returned -1)");
+ throw new IllegalStateException("terms.size() must be implemented (it returned -1)");
}
final TermsEnum termsEnum = terms.iterator(null);
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xFields.java Tue Apr 10 23:21:39 2012
@@ -161,7 +161,7 @@ class Lucene3xFields extends FieldsProdu
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return preTerms.size();
}
@@ -246,7 +246,7 @@ class Lucene3xFields extends FieldsProdu
}
@Override
- public long getUniqueTermCount() throws IOException {
+ public long size() throws IOException {
return -1;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xTermVectorsReader.java Tue Apr 10 23:21:39 2012
@@ -270,7 +270,7 @@ class Lucene3xTermVectorsReader extends
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
if (fieldNumbers == null) {
return 0;
} else {
@@ -307,7 +307,7 @@ class Lucene3xTermVectorsReader extends
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return numTerms;
}
@@ -660,7 +660,7 @@ class Lucene3xTermVectorsReader extends
}
if (tvx != null) {
Fields fields = new TVFields(docID);
- if (fields.getUniqueFieldCount() == 0) {
+ if (fields.size() == 0) {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that
// this doc has no TVs
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Tue Apr 10 23:21:39 2012
@@ -300,7 +300,7 @@ public class Lucene40TermVectorsReader e
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
if (fieldNumbers == null) {
return 0;
} else {
@@ -335,7 +335,7 @@ public class Lucene40TermVectorsReader e
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return numTerms;
}
@@ -674,7 +674,7 @@ public class Lucene40TermVectorsReader e
}
if (tvx != null) {
Fields fields = new TVFields(docID);
- if (fields.getUniqueFieldCount() == 0) {
+ if (fields.size() == 0) {
// TODO: we can improve writer here, eg write 0 into
// tvx file, so we know on first read from tvx that
// this doc has no TVs
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Tue Apr 10 23:21:39 2012
@@ -824,7 +824,7 @@ public class MemoryPostingsFormat extend
}
@Override
- public long getUniqueTermCount() throws IOException {
+ public long size() throws IOException {
return termCount;
}
@@ -888,7 +888,7 @@ public class MemoryPostingsFormat extend
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return fields.size();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Tue Apr 10 23:21:39 2012
@@ -248,7 +248,7 @@ public abstract class PerFieldPostingsFo
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return fields.size();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Tue Apr 10 23:21:39 2012
@@ -596,7 +596,7 @@ class SimpleTextFieldsReader extends Fie
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return (long) termCount;
}
@@ -641,7 +641,7 @@ class SimpleTextFieldsReader extends Fie
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return -1;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Tue Apr 10 23:21:39 2012
@@ -257,7 +257,7 @@ public class SimpleTextTermVectorsReader
}
@Override
- public int getUniqueFieldCount() throws IOException {
+ public int size() throws IOException {
return fields.size();
}
}
@@ -281,7 +281,7 @@ public class SimpleTextTermVectorsReader
}
@Override
- public long getUniqueTermCount() throws IOException {
+ public long size() throws IOException {
return terms.size();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Apr 10 23:21:39 2012
@@ -1051,7 +1051,7 @@ public class CheckIndex {
long termCount = -1;
if (status.termCount-termCountStart > 0) {
- termCount = fields.terms(field).getUniqueTermCount();
+ termCount = fields.terms(field).size();
if (termCount != -1 && termCount != status.termCount - termCountStart) {
throw new RuntimeException("termCount mismatch " + termCount + " vs " + (status.termCount - termCountStart));
@@ -1104,7 +1104,7 @@ public class CheckIndex {
}
}
- int fieldCount = fields.getUniqueFieldCount();
+ int fieldCount = fields.size();
if (fieldCount != -1) {
if (fieldCount < 0) {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Fields.java Tue Apr 10 23:21:39 2012
@@ -36,7 +36,7 @@ public abstract class Fields {
* measure isn't stored by the codec. Note that, just like
* other term measures, this measure does not take deleted
* documents into account. */
- public abstract int getUniqueFieldCount() throws IOException;
+ public abstract int size() throws IOException;
/** Returns the number of terms for all fields, or -1 if this
* measure isn't stored by the codec. Note that, just like
@@ -53,7 +53,7 @@ public abstract class Fields {
}
Terms terms = terms(field);
if (terms != null) {
- final long termCount = terms.getUniqueTermCount();
+ final long termCount = terms.size();
if (termCount == -1) {
return -1;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Tue Apr 10 23:21:39 2012
@@ -56,8 +56,8 @@ public class FilterAtomicReader extends
}
@Override
- public int getUniqueFieldCount() throws IOException {
- return in.getUniqueFieldCount();
+ public int size() throws IOException {
+ return in.size();
}
@Override
@@ -86,8 +86,8 @@ public class FilterAtomicReader extends
}
@Override
- public long getUniqueTermCount() throws IOException {
- return in.getUniqueTermCount();
+ public long size() throws IOException {
+ return in.size();
}
@Override
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Tue Apr 10 23:21:39 2012
@@ -228,7 +228,7 @@ public final class MultiFields extends F
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return -1;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTerms.java Tue Apr 10 23:21:39 2012
@@ -96,7 +96,7 @@ public final class MultiTerms extends Te
}
@Override
- public long getUniqueTermCount() throws IOException {
+ public long size() throws IOException {
return -1;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Tue Apr 10 23:21:39 2012
@@ -197,7 +197,7 @@ public final class ParallelAtomicReader
}
@Override
- public int getUniqueFieldCount() throws IOException {
+ public int size() throws IOException {
return fields.size();
}
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java Tue Apr 10 23:21:39 2012
@@ -81,7 +81,7 @@ public abstract class Terms {
* measure isn't stored by the codec. Note that, just like
* other term measures, this measure does not take deleted
* documents into account. */
- public abstract long getUniqueTermCount() throws IOException;
+ public abstract long size() throws IOException;
/** Returns the sum of {@link TermsEnum#totalTermFreq} for
* all terms in this field, or -1 if this measure isn't
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Tue Apr 10 23:21:39 2012
@@ -1114,7 +1114,7 @@ class FieldCacheImpl implements FieldCac
// is fine -- GrowableWriter will reallocate as needed
long numUniqueTerms = 0;
try {
- numUniqueTerms = terms.getUniqueTermCount();
+ numUniqueTerms = terms.size();
} catch (UnsupportedOperationException uoe) {
numUniqueTerms = -1;
}
@@ -1165,7 +1165,7 @@ class FieldCacheImpl implements FieldCac
if (termOrd == termOrdToBytesOffset.size()) {
// NOTE: this code only runs if the incoming
// reader impl doesn't implement
- // getUniqueTermCount (which should be uncommon)
+ // size (which should be uncommon)
termOrdToBytesOffset = termOrdToBytesOffset.resize(ArrayUtil.oversize(1+termOrd, 1));
}
termOrdToBytesOffset.set(termOrd, bytes.copyUsingLengthPrefix(term));
@@ -1252,7 +1252,7 @@ class FieldCacheImpl implements FieldCac
// is fine -- GrowableWriter will reallocate as needed
long numUniqueTerms = 0;
try {
- numUniqueTerms = terms.getUniqueTermCount();
+ numUniqueTerms = terms.size();
} catch (UnsupportedOperationException uoe) {
numUniqueTerms = -1;
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/codecs/lucene40/TestReuseDocsEnum.java Tue Apr 10 23:21:39 2012
@@ -21,7 +21,6 @@ import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
@@ -63,7 +62,7 @@ public class TestReuseDocsEnum extends L
enums.put(docs, true);
}
- assertEquals(terms.getUniqueTermCount(), enums.size());
+ assertEquals(terms.size(), enums.size());
}
}.run();
IOUtils.close(writer, open, dir);
@@ -100,7 +99,7 @@ public class TestReuseDocsEnum extends L
docs = iterator.docs(new Bits.MatchNoBits(open.maxDoc()), docs, random.nextBoolean());
enums.put(docs, true);
}
- assertEquals(terms.getUniqueTermCount(), enums.size());
+ assertEquals(terms.size(), enums.size());
enums.clear();
iterator = terms.iterator(null);
@@ -141,7 +140,7 @@ public class TestReuseDocsEnum extends L
docs = iterator.docs(null, randomDocsEnum("body", term, sequentialSubReaders2, bits), random.nextBoolean());
enums.put(docs, true);
}
- assertEquals(terms.getUniqueTermCount(), enums.size());
+ assertEquals(terms.size(), enums.size());
iterator = terms.iterator(null);
enums.clear();
@@ -150,7 +149,7 @@ public class TestReuseDocsEnum extends L
docs = iterator.docs(bits, randomDocsEnum("body", term, sequentialSubReaders2, bits), random.nextBoolean());
enums.put(docs, true);
}
- assertEquals(terms.getUniqueTermCount(), enums.size());
+ assertEquals(terms.size(), enums.size());
}
IOUtils.close(writer, firstReader, secondReader, dir);
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/document/TestDocument.java Tue Apr 10 23:21:39 2012
@@ -21,7 +21,6 @@ import java.io.StringReader;
import org.apache.lucene.analysis.EmptyTokenizer;
import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.Fields;
@@ -345,7 +344,7 @@ public class TestDocument extends Lucene
Fields tvFields = r.getTermVectors(0);
Terms tvs = tvFields.terms(field);
assertNotNull(tvs);
- assertEquals(2, tvs.getUniqueTermCount());
+ assertEquals(2, tvs.size());
TermsEnum tvsEnum = tvs.iterator(null);
assertEquals(new BytesRef("abc"), tvsEnum.next());
final DocsAndPositionsEnum dpEnum = tvsEnum.docsAndPositions(null, null, false);
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Tue Apr 10 23:21:39 2012
@@ -290,11 +290,11 @@ public class TestDocumentWriter extends
// f1
Terms tfv1 = reader.getTermVectors(0).terms("f1");
assertNotNull(tfv1);
- assertEquals("the 'with_tv' setting should rule!",2,tfv1.getUniqueTermCount());
+ assertEquals("the 'with_tv' setting should rule!",2,tfv1.size());
// f2
Terms tfv2 = reader.getTermVectors(0).terms("f2");
assertNotNull(tfv2);
- assertEquals("the 'with_tv' setting should rule!",2,tfv2.getUniqueTermCount());
+ assertEquals("the 'with_tv' setting should rule!",2,tfv2.size());
reader.close();
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java Tue Apr 10 23:21:39 2012
@@ -184,8 +184,8 @@ public class TestDuelingCodecs extends L
* checks that top-level statistics on Fields are the same
*/
public void assertFieldStatistics(Fields leftFields, Fields rightFields) throws Exception {
- if (leftFields.getUniqueFieldCount() != -1 && rightFields.getUniqueFieldCount() != -1) {
- assertEquals(info, leftFields.getUniqueFieldCount(), rightFields.getUniqueFieldCount());
+ if (leftFields.size() != -1 && rightFields.size() != -1) {
+ assertEquals(info, leftFields.size(), rightFields.size());
}
if (leftFields.getUniqueTermCount() != -1 && rightFields.getUniqueTermCount() != -1) {
@@ -238,8 +238,8 @@ public class TestDuelingCodecs extends L
if (leftTerms.getSumTotalTermFreq() != -1 && rightTerms.getSumTotalTermFreq() != -1) {
assertEquals(info, leftTerms.getSumTotalTermFreq(), rightTerms.getSumTotalTermFreq());
}
- if (leftTerms.getUniqueTermCount() != -1 && rightTerms.getUniqueTermCount() != -1) {
- assertEquals(info, leftTerms.getUniqueTermCount(), rightTerms.getUniqueTermCount());
+ if (leftTerms.size() != -1 && rightTerms.size() != -1) {
+ assertEquals(info, leftTerms.size(), rightTerms.size());
}
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java Tue Apr 10 23:21:39 2012
@@ -19,10 +19,8 @@ package org.apache.lucene.index;
import java.io.IOException;
-import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
-import org.apache.lucene.document.TextField;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
@@ -120,7 +118,7 @@ public class TestSegmentMerger extends L
Terms vector = mergedReader.getTermVectors(0).terms(DocHelper.TEXT_FIELD_2_KEY);
assertNotNull(vector);
- assertEquals(3, vector.getUniqueTermCount());
+ assertEquals(3, vector.size());
TermsEnum termsEnum = vector.iterator(null);
int i = 0;
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java Tue Apr 10 23:21:39 2012
@@ -193,7 +193,7 @@ public class TestSegmentReader extends L
public void testTermVectors() throws IOException {
Terms result = reader.getTermVectors(0).terms(DocHelper.TEXT_FIELD_2_KEY);
assertNotNull(result);
- assertEquals(3, result.getUniqueTermCount());
+ assertEquals(3, result.size());
TermsEnum termsEnum = result.iterator(null);
while(termsEnum.next() != null) {
String term = termsEnum.term().utf8ToString();
@@ -204,6 +204,6 @@ public class TestSegmentReader extends L
Fields results = reader.getTermVectors(0);
assertTrue(results != null);
- assertEquals("We do not have 3 term freq vectors", 3, results.getUniqueFieldCount());
+ assertEquals("We do not have 3 term freq vectors", 3, results.size());
}
}
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java Tue Apr 10 23:21:39 2012
@@ -600,7 +600,7 @@ public class TestStressIndexing2 extends
public static void verifyEquals(Fields d1, Fields d2) throws IOException {
if (d1 == null) {
- assertTrue(d2 == null || d2.getUniqueFieldCount() == 0);
+ assertTrue(d2 == null || d2.size() == 0);
return;
}
assertTrue(d2 != null);
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java Tue Apr 10 23:21:39 2012
@@ -204,7 +204,7 @@ public class TestTermVectorsReader exten
for (int j = 0; j < 5; j++) {
Terms vector = reader.get(j).terms(testFields[0]);
assertNotNull(vector);
- assertEquals(testTerms.length, vector.getUniqueTermCount());
+ assertEquals(testTerms.length, vector.size());
TermsEnum termsEnum = vector.iterator(null);
for (int i = 0; i < testTerms.length; i++) {
final BytesRef text = termsEnum.next();
@@ -223,7 +223,7 @@ public class TestTermVectorsReader exten
for (int j = 0; j < 5; j++) {
Terms vector = reader.get(j).terms(testFields[0]);
assertNotNull(vector);
- assertEquals(testTerms.length, vector.getUniqueTermCount());
+ assertEquals(testTerms.length, vector.size());
TermsEnum termsEnum = vector.iterator(null);
DocsEnum docsEnum = null;
for (int i = 0; i < testTerms.length; i++) {
@@ -250,7 +250,7 @@ public class TestTermVectorsReader exten
BytesRef[] terms;
Terms vector = reader.get(0).terms(testFields[0]);
assertNotNull(vector);
- assertEquals(testTerms.length, vector.getUniqueTermCount());
+ assertEquals(testTerms.length, vector.size());
TermsEnum termsEnum = vector.iterator(null);
DocsAndPositionsEnum dpEnum = null;
for (int i = 0; i < testTerms.length; i++) {
@@ -287,7 +287,7 @@ public class TestTermVectorsReader exten
Terms freqVector = reader.get(0).terms(testFields[1]); //no pos, no offset
assertNotNull(freqVector);
- assertEquals(testTerms.length, freqVector.getUniqueTermCount());
+ assertEquals(testTerms.length, freqVector.size());
termsEnum = freqVector.iterator(null);
assertNotNull(termsEnum);
for (int i = 0; i < testTerms.length; i++) {
@@ -306,7 +306,7 @@ public class TestTermVectorsReader exten
assertNotNull(vector);
TermsEnum termsEnum = vector.iterator(null);
assertNotNull(termsEnum);
- assertEquals(testTerms.length, vector.getUniqueTermCount());
+ assertEquals(testTerms.length, vector.size());
DocsAndPositionsEnum dpEnum = null;
for (int i = 0; i < testTerms.length; i++) {
final BytesRef text = termsEnum.next();
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/FieldCacheRewriteMethod.java Tue Apr 10 23:21:39 2012
@@ -119,7 +119,7 @@ public final class FieldCacheRewriteMeth
}
@Override
- public long getUniqueTermCount() throws IOException {
+ public long size() throws IOException {
return -1;
}
});
Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermVectors.java Tue Apr 10 23:21:39 2012
@@ -98,7 +98,7 @@ public class TestTermVectors extends Luc
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
- assertEquals("doc=" + hits[i].doc + " tv=" + vectors, 1, vectors.getUniqueFieldCount());
+ assertEquals("doc=" + hits[i].doc + " tv=" + vectors, 1, vectors.size());
}
Terms vector;
vector = searcher.reader.getTermVectors(hits[0].doc).terms("noTV");
@@ -121,13 +121,13 @@ public class TestTermVectors extends Luc
IndexReader reader = writer.getReader();
writer.close();
Fields v = reader.getTermVectors(0);
- assertEquals(4, v.getUniqueFieldCount());
+ assertEquals(4, v.size());
String[] expectedFields = new String[]{"a", "b", "c", "x"};
int[] expectedPositions = new int[]{1, 2, 0};
FieldsEnum fieldsEnum = v.iterator();
for(int i=0;i<expectedFields.length;i++) {
assertEquals(expectedFields[i], fieldsEnum.next());
- assertEquals(3, v.terms(expectedFields[i]).getUniqueTermCount());
+ assertEquals(3, v.terms(expectedFields[i]).size());
DocsAndPositionsEnum dpEnum = null;
Terms terms = fieldsEnum.terms();
@@ -166,7 +166,7 @@ public class TestTermVectors extends Luc
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
- assertEquals(1, vectors.getUniqueFieldCount());
+ assertEquals(1, vectors.size());
TermsEnum termsEnum = vectors.terms("field").iterator(null);
assertNotNull(termsEnum.next());
@@ -205,7 +205,7 @@ public class TestTermVectors extends Luc
for (int i = 0; i < hits.length; i++) {
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
- assertEquals(1, vectors.getUniqueFieldCount());
+ assertEquals(1, vectors.size());
}
}
@@ -303,7 +303,7 @@ public class TestTermVectors extends Luc
Terms vector = knownSearcher.reader.getTermVectors(hits[1].doc).terms("field");
assertNotNull(vector);
//System.out.println("Vector: " + vector);
- assertEquals(10, vector.getUniqueTermCount());
+ assertEquals(10, vector.size());
TermsEnum termsEnum = vector.iterator(null);
while(termsEnum.next() != null) {
String term = termsEnum.term().utf8ToString();
@@ -371,7 +371,7 @@ public class TestTermVectors extends Luc
Fields vectors = searcher.reader.getTermVectors(hits[i].doc);
assertNotNull(vectors);
- assertEquals(1, vectors.getUniqueFieldCount());
+ assertEquals(1, vectors.size());
}
reader.close();
}
@@ -418,10 +418,10 @@ public class TestTermVectors extends Luc
Fields vectors = searcher.reader.getTermVectors(hits[0].doc);
assertNotNull(vectors);
- assertEquals(1, vectors.getUniqueFieldCount());
+ assertEquals(1, vectors.size());
Terms vector = vectors.terms("field");
assertNotNull(vector);
- assertEquals(1, vector.getUniqueTermCount());
+ assertEquals(1, vector.size());
TermsEnum termsEnum = vector.iterator(null);
assertNotNull(termsEnum.next());
assertEquals("one", termsEnum.term().utf8ToString());
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Tue Apr 10 23:21:39 2012
@@ -109,7 +109,7 @@ public class RAMOnlyPostingsFormat exten
}
@Override
- public int getUniqueFieldCount() {
+ public int size() {
return fieldToTerms.size();
}
@@ -135,7 +135,7 @@ public class RAMOnlyPostingsFormat exten
}
@Override
- public long getUniqueTermCount() {
+ public long size() {
return termToDocs.size();
}
Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/FieldFilterAtomicReader.java Tue Apr 10 23:21:39 2012
@@ -130,7 +130,7 @@ public final class FieldFilterAtomicRead
}
@Override
- public int getUniqueFieldCount() throws IOException {
+ public int size() throws IOException {
// TODO: add faster implementation!
int c = 0;
final FieldsEnum it = iterator();
Modified: lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java (original)
+++ lucene/dev/trunk/modules/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java Tue Apr 10 23:21:39 2012
@@ -30,7 +30,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Terms;
@@ -106,7 +105,7 @@ public class TestTeeSinkTokenFilter exte
IndexReader r = IndexReader.open(dir);
Terms vector = r.getTermVectors(0).terms("field");
- assertEquals(1, vector.getUniqueTermCount());
+ assertEquals(1, vector.size());
TermsEnum termsEnum = vector.iterator(null);
termsEnum.next();
assertEquals(2, termsEnum.totalTermFreq());
Modified: lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/dev/trunk/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Tue Apr 10 23:21:39 2012
@@ -752,7 +752,7 @@ public class TestPerfTasksLogic extends
IndexReader reader = IndexReader.open(dir);
Fields tfv = reader.getTermVectors(0);
assertNotNull(tfv);
- assertTrue(tfv.getUniqueFieldCount() > 0);
+ assertTrue(tfv.size() > 0);
reader.close();
}
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java?rev=1312037&r1=1312036&r2=1312037&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java Tue Apr 10 23:21:39 2012
@@ -595,7 +595,7 @@ public class LukeRequestHandler extends
if (freq > tiq.minFreq) {
UnicodeUtil.UTF8toUTF16(text, spare);
String t = spare.toString();
- tiq.distinctTerms = new Long(terms.getUniqueTermCount()).intValue();
+ tiq.distinctTerms = new Long(terms.size()).intValue();
tiq.add(new TopTermQueue.TermInfo(new Term(field, t), termsEnum.docFreq()));
if (tiq.size() > numTerms) { // if tiq full