lucene-pylucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From va...@apache.org
Subject svn commit: r961790 - in /lucene/pylucene/trunk: ./ jcc/jcc/ samples/LuceneInAction/lia/advsearching/ samples/LuceneInAction/lia/analysis/ samples/LuceneInAction/lia/analysis/synonym/ samples/LuceneInAction/lia/extsearch/collector/ samples/LuceneInActi...
Date Thu, 08 Jul 2010 14:53:34 GMT
Author: vajda
Date: Thu Jul  8 14:53:33 2010
New Revision: 961790

URL: http://svn.apache.org/viewvc?rev=961790&view=rev
Log:
caught up with lucene trunk, migrated to new API

Modified:
    lucene/pylucene/trunk/   (props changed)
    lucene/pylucene/trunk/Makefile
    lucene/pylucene/trunk/jcc/jcc/cpp.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/CategorizerTest.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/SpanQueryTest.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/AnalyzerUtils.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymAnalyzerTest.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymFilter.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/collector/BookLinkCollector.py
    lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/filters/SpecialsFilter.py
    lucene/pylucene/trunk/test/BaseTokenStreamTestCase.py
    lucene/pylucene/trunk/test/test_PerFieldAnalyzerWrapper.py
    lucene/pylucene/trunk/test/test_PositionIncrement.py
    lucene/pylucene/trunk/test/test_PyLucene.py
    lucene/pylucene/trunk/test/test_StopAnalyzer.py
    lucene/pylucene/trunk/test/test_bug1842.py

Propchange: lucene/pylucene/trunk/
------------------------------------------------------------------------------
    svn:mergeinfo = /lucene/pylucene/branches/branch_3x:944953-944954

Modified: lucene/pylucene/trunk/Makefile
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/Makefile?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/Makefile (original)
+++ lucene/pylucene/trunk/Makefile Thu Jul  8 14:53:33 2010
@@ -14,12 +14,14 @@
 # site-packages directory.
 #
 
-VERSION=3.1-0
+VERSION=4.0-0
 LUCENE_SVN_VER=HEAD
-LUCENE_VER=3.1
+LUCENE_VER=4.0
 LUCENE_SVN=http://svn.apache.org/repos/asf/lucene/dev/trunk/lucene
+MODULES_SVN=http://svn.apache.org/repos/asf/lucene/dev/trunk/modules
 PYLUCENE:=$(shell pwd)
 LUCENE=lucene-java-$(LUCENE_VER)
+MODULES=lucene-modules-$(LUCENE_VER)
 
 # 
 # You need to uncomment and edit the variables below in the section
@@ -30,6 +32,8 @@ LUCENE=lucene-java-$(LUCENE_VER)
 # PREFIX: where programs are normally installed on your system (Unix).
 # PREFIX_PYTHON: where your version of python is installed.
 # JCC: how jcc is invoked, depending on the python version:
+#  - python 2.7:
+#      $(PYTHON) -m jcc
 #  - python 2.6:
 #      $(PYTHON) -m jcc.__main__
 #  - python 2.5:
@@ -48,11 +52,11 @@ LUCENE=lucene-java-$(LUCENE_VER)
 #JCC=$(PYTHON) -m jcc.__main__ --shared --arch x86_64
 #NUM_FILES=3
 
-# Mac OS X 10.6 (MacPorts 1.8.0 64-bit Python 2.6, Java 1.6)
+# Mac OS X 10.6 (MacPorts 1.8.0 64-bit Python 2.7, Java 1.6)
 #PREFIX_PYTHON=/opt/local
 #ANT=ant
 #PYTHON=$(PREFIX_PYTHON)/bin/python
-#JCC=$(PYTHON) -m jcc.__main__ --shared --arch x86_64
+#JCC=$(PYTHON) -m jcc --arch x86_64
 #NUM_FILES=3
 
 # Mac OS X 10.6 (64-bit and 32-bit Python 2.6 together, Java 1.6)
@@ -137,7 +141,7 @@ endif
 DEFINES=-DPYLUCENE_VER="\"$(VERSION)\"" -DLUCENE_VER="\"$(LUCENE_VER)\""
 
 LUCENE_JAR=$(LUCENE)/build/lucene-core-$(LUCENE_VER).jar
-ANALYZERS_JAR=$(LUCENE)/build/contrib/analyzers/common/lucene-analyzers-$(LUCENE_VER).jar
+ANALYZERS_JAR=$(MODULES)/analysis/build/common/lucene-analyzers-common-$(LUCENE_VER).jar
 HIGHLIGHTER_JAR=$(LUCENE)/build/contrib/highlighter/lucene-highlighter-$(LUCENE_VER).jar
 MEMORY_JAR=$(LUCENE)/build/contrib/memory/lucene-memory-$(LUCENE_VER).jar
 QUERIES_JAR=$(LUCENE)/build/contrib/queries/lucene-queries-$(LUCENE_VER).jar
@@ -152,6 +156,7 @@ default: all
 
 $(LUCENE):
 	svn export -r $(LUCENE_SVN_VER) $(LUCENE_SVN) $(LUCENE)
+	svn export -r $(LUCENE_SVN_VER) $(MODULES_SVN) $(MODULES)
 
 sources: $(LUCENE)
 
@@ -171,7 +176,7 @@ $(LUCENE_JAR): $(LUCENE)
 	cd $(LUCENE); $(ANT) -Dversion=$(LUCENE_VER)
 
 $(ANALYZERS_JAR): $(LUCENE_JAR)
-	cd $(LUCENE)/contrib/analyzers/common; $(ANT) -Dversion=$(LUCENE_VER)
+	cd $(MODULES); $(ANT) -Dversion=$(LUCENE_VER) compile
 
 $(MEMORY_JAR): $(LUCENE_JAR)
 	cd $(LUCENE)/contrib/memory; $(ANT) -Dversion=$(LUCENE_VER)
@@ -189,18 +194,20 @@ JARS=$(LUCENE_JAR) $(ANALYZERS_JAR) \
      $(MEMORY_JAR) $(HIGHLIGHTER_JAR) $(QUERIES_JAR) \
      $(EXTENSIONS_JAR)
 
+JCCFLAGS?=
 
 jars: $(JARS)
 
 
 ifneq ($(ICUPKG),)
 
-RESOURCES=--resources $(LUCENE)/contrib/icu/src/resources
+ICURES= $(MODULES)/analysis/icu/src/resources
+RESOURCES=--resources $(ICURES)
 ENDIANNESS:=$(shell $(PYTHON) -c "import struct; print struct.pack('h', 1) == '\000\001'
and 'b' or 'l'")
 
-resources: $(LUCENE)/contrib/icu/src/resources/org/apache/lucene/analysis/icu/utr30.dat
+resources: $(ICURES)/org/apache/lucene/analysis/icu/utr30.dat
 
-$(LUCENE)/contrib/icu/src/resources/org/apache/lucene/analysis/icu/utr30.dat: $(LUCENE)/contrib/icu/src/resources/org/apache/lucene/analysis/icu/utr30.nrm
+$(ICURES)/org/apache/lucene/analysis/icu/utr30.dat: $(ICURES)/org/apache/lucene/analysis/icu/utr30.nrm
 	rm -f $@
 	cd $(dir $<); $(ICUPKG) --type $(ENDIANNESS) --add $(notdir $<) new $(notdir $@)
 
@@ -214,6 +221,7 @@ resources:
 endif
 
 GENERATE=$(JCC) $(foreach jar,$(JARS),--jar $(jar)) \
+           $(JCCFLAGS) \
            --package java.lang java.lang.System \
                                java.lang.Runtime \
            --package java.util \

Modified: lucene/pylucene/trunk/jcc/jcc/cpp.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/jcc/jcc/cpp.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/jcc/jcc/cpp.py (original)
+++ lucene/pylucene/trunk/jcc/jcc/cpp.py Thu Jul  8 14:53:33 2010
@@ -79,8 +79,8 @@ PRIMITIVES = { 'boolean': 'Z',
                'short': 'S',
                'void': 'V' }
 
-RESERVED = set(['delete', 'and', 'or', 'not', 'xor', 'union',
-                'register', 'const', 'bool', 'operator', 'typeof', 'asm',
+RESERVED = set(['delete', 'and', 'or', 'not', 'xor', 'union', 'register',
+                'const', 'bool', 'operator', 'typeof', 'asm',
                 'NULL', 'DOMAIN'])
 
 def cppname(name):

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/CategorizerTest.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/CategorizerTest.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/CategorizerTest.py (original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/CategorizerTest.py Thu Jul
 8 14:53:33 2010
@@ -63,6 +63,7 @@ class CategorizerTest(LiaTestCase):
 
         i = 0
         for term in terms:
+            term = term.utf8ToString()
             if term in vectorMap:
                 vectorMap[term] += freqs[i]
             else:

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/SpanQueryTest.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/SpanQueryTest.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/SpanQueryTest.py (original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/advsearching/SpanQueryTest.py Thu Jul
 8 14:53:33 2010
@@ -18,7 +18,7 @@ from cStringIO import StringIO
 from lucene import \
      WhitespaceAnalyzer, Document, Field, IndexReader, IndexWriter, Term, \
      IndexSearcher, PhraseQuery, SpanFirstQuery, SpanNearQuery, SpanNotQuery, \
-     SpanOrQuery, SpanTermQuery, RAMDirectory, TermAttribute, StringReader
+     SpanOrQuery, SpanTermQuery, RAMDirectory, CharTermAttribute, StringReader
 
 from lia.analysis.AnalyzerUtils import AnalyzerUtils
 
@@ -185,7 +185,7 @@ class SpanQueryTest(TestCase):
             # positions, starting from 0
             stream = self.analyzer.tokenStream("contents",
                                                StringReader(doc.get("f")))
-            term = stream.addAttribute(TermAttribute.class_)
+            term = stream.addAttribute(CharTermAttribute.class_)
       
             buffer = StringIO()
             buffer.write("   ")
@@ -195,7 +195,7 @@ class SpanQueryTest(TestCase):
                 if i == spans.start():
                     buffer.write("<")
 
-                buffer.write(term.term())
+                buffer.write(term.toString())
                 if i + 1 == spans.end():
                     buffer.write(">")
 

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/AnalyzerUtils.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/AnalyzerUtils.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/AnalyzerUtils.py (original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/AnalyzerUtils.py Thu Jul  8
14:53:33 2010
@@ -14,7 +14,8 @@
 
 from lucene import \
      SimpleAnalyzer, StandardAnalyzer, StringReader, Version, \
-     TermAttribute, PositionIncrementAttribute, TypeAttribute, OffsetAttribute
+     CharTermAttribute, PositionIncrementAttribute, TypeAttribute, \
+     OffsetAttribute
 
 
 class AnalyzerUtils(object):
@@ -38,12 +39,13 @@ class AnalyzerUtils(object):
         return attr.getPositionIncrement()
 
     def setTerm(cls, source, term):
-        attr = source.addAttribute(TermAttribute.class_)
-        attr.setTermBuffer(term)
+        attr = source.addAttribute(CharTermAttribute.class_)
+        attr.setEmpty()
+        attr.append(term)
 
     def getTerm(cls, source):
-        attr = source.addAttribute(TermAttribute.class_)
-        return attr.term()
+        attr = source.addAttribute(CharTermAttribute.class_)
+        return attr.toString()
 
     def setType(cls, source, type):
         attr = source.addAttribute(TypeAttribute.class_)
@@ -56,15 +58,15 @@ class AnalyzerUtils(object):
     def displayTokens(cls, analyzer, text):
 
         tokenStream = analyzer.tokenStream("contents", StringReader(text))
-        term = tokenStream.addAttribute(TermAttribute.class_)
+        term = tokenStream.addAttribute(CharTermAttribute.class_)
 
         while tokenStream.incrementToken():
-            print "[%s]" %(term.term()),
+            print "[%s]" %(term.toString()),
 
     def displayTokensWithPositions(cls, analyzer, text):
 
         stream = analyzer.tokenStream("contents", StringReader(text))
-        term = stream.addAttribute(TermAttribute.class_)
+        term = stream.addAttribute(CharTermAttribute.class_)
         posIncr = stream.addAttribute(PositionIncrementAttribute.class_)
 
         position = 0
@@ -74,14 +76,14 @@ class AnalyzerUtils(object):
                 position = position + increment
                 print "\n%d:" %(position),
 
-            print "[%s]" %(term.term()),
+            print "[%s]" %(term.toString()),
         print
 
     def displayTokensWithFullDetails(cls, analyzer, text):
 
         stream = analyzer.tokenStream("contents", StringReader(text))
 
-        term = stream.addAttribute(TermAttribute.class_)
+        term = stream.addAttribute(CharTermAttribute.class_)
         posIncr = stream.addAttribute(PositionIncrementAttribute.class_)
         offset = stream.addAttribute(OffsetAttribute.class_)
         type = stream.addAttribute(TypeAttribute.class_)
@@ -93,7 +95,7 @@ class AnalyzerUtils(object):
                 position = position + increment
                 print "\n%d:" %(position),
 
-            print "[%s:%d->%d:%s]" %(term.term(),
+            print "[%s:%d->%d:%s]" %(term.toString(),
                                      offset.startOffset(),
                                      offset.endOffset(),
                                      type.type()),
@@ -102,12 +104,12 @@ class AnalyzerUtils(object):
     def assertAnalyzesTo(cls, analyzer, input, outputs):
 
         stream = analyzer.tokenStream("field", StringReader(input))
-        termAttr = stream.addAttribute(TermAttribute.class_)
+        termAttr = stream.addAttribute(CharTermAttribute.class_)
         for output in outputs:
             if not stream.incrementToken():
                 raise AssertionError, 'stream.incremementToken()'
-            if output != termAttr.term():
-                raise AssertionError, 'output == termAttr.term())'
+            if output != termAttr.toString():
+                raise AssertionError, 'output == termAttr.toString())'
 
         if stream.incrementToken():
             raise AssertionError, 'not stream.incremementToken()'

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymAnalyzerTest.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymAnalyzerTest.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymAnalyzerTest.py
(original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymAnalyzerTest.py
Thu Jul  8 14:53:33 2010
@@ -17,7 +17,7 @@ from unittest import TestCase
 from lucene import \
      StandardAnalyzer, RAMDirectory, IndexWriter, Term, Document, Field, \
      IndexSearcher, TermQuery, PhraseQuery, QueryParser, StringReader, \
-     TermAttribute, PositionIncrementAttribute, Version
+     CharTermAttribute, PositionIncrementAttribute, Version
 
 from lia.analysis.AnalyzerUtils import AnalyzerUtils
 from lia.analysis.synonym.SynonymAnalyzer import SynonymAnalyzer
@@ -51,13 +51,13 @@ class SynonymAnalyzerTest(TestCase):
 
         stream = self.synonymAnalyzer.tokenStream("contents",
                                                   StringReader("jumps"))
-        term = stream.addAttribute(TermAttribute.class_)
+        term = stream.addAttribute(CharTermAttribute.class_)
         posIncr = stream.addAttribute(PositionIncrementAttribute.class_)
 
         i = 0
         expected = ["jumps", "hops", "leaps"]
         while stream.incrementToken():
-            self.assertEqual(expected[i], term.term())
+            self.assertEqual(expected[i], term.toString())
             if i == 0:
                 expectedPos = 1
             else:

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymFilter.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymFilter.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymFilter.py (original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/analysis/synonym/SynonymFilter.py Thu
Jul  8 14:53:33 2010
@@ -12,7 +12,7 @@
 #   limitations under the License.
 # ====================================================================
 
-from lucene import Token, PythonTokenFilter, TermAttribute
+from lucene import Token, PythonTokenFilter, CharTermAttribute
 from lia.analysis.AnalyzerUtils import AnalyzerUtils
 
 #
@@ -26,7 +26,7 @@ class SynonymFilter(PythonTokenFilter):
         super(SynonymFilter, self).__init__(inStream)
 
         self.synonymStack = []
-        self.termAttr = self.addAttribute(TermAttribute.class_)
+        self.termAttr = self.addAttribute(CharTermAttribute.class_)
         self.save = inStream.cloneAttributes()
         self.engine = engine
         self.inStream = inStream
@@ -47,7 +47,7 @@ class SynonymFilter(PythonTokenFilter):
 
     def addAliasesToStack(self):
 
-        synonyms = self.engine.getSynonyms(self.termAttr.term())
+        synonyms = self.engine.getSynonyms(self.termAttr.toString())
         if synonyms is None:
             return
 

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/collector/BookLinkCollector.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/collector/BookLinkCollector.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/collector/BookLinkCollector.py
(original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/collector/BookLinkCollector.py
Thu Jul  8 14:53:33 2010
@@ -12,7 +12,8 @@
 #   limitations under the License.
 # ====================================================================
 
-from lucene import Document, IndexSearcher, PythonCollector, FieldCache
+from lucene import Document, IndexSearcher, PythonCollector, FieldCache, \
+    BytesRef
 
 #
 # A Collector extension
@@ -33,13 +34,14 @@ class BookLinkCollector(PythonCollector)
     def setNextReader(self, reader, docBase):
 
         self.docBase = docBase
-        self.urls = FieldCache.DEFAULT.getStrings(reader, "url")
-        self.titles = FieldCache.DEFAULT.getStrings(reader, "title2")
+        self.urls = FieldCache.DEFAULT.getTerms(reader, "url")
+        self.titles = FieldCache.DEFAULT.getTerms(reader, "title2")
 
     def collect(self, docID, score):
-
-        url = self.urls[docID]
-        title = self.titles[docID]
+        
+        term = BytesRef()
+        url = self.urls.getTerm(docID, term).utf8ToString()
+        title = self.titles.getTerm(docID, term).utf8ToString()
         self.documents[url] = title
 
         print "%s: %s" %(title, score)

Modified: lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/filters/SpecialsFilter.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/filters/SpecialsFilter.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/filters/SpecialsFilter.py (original)
+++ lucene/pylucene/trunk/samples/LuceneInAction/lia/extsearch/filters/SpecialsFilter.py Thu
Jul  8 14:53:33 2010
@@ -13,7 +13,7 @@
 # ====================================================================
 
 from lucene import \
-    IndexReader, Term, BitSet, PythonFilter, JArray, OpenBitSet
+    IndexReader, Term, BitSet, PythonFilter, JArray, OpenBitSet, BytesRef
 
 #
 # A Filter extension, with a TermDocs wrapper working around the lack of
@@ -31,15 +31,14 @@ class SpecialsFilter(PythonFilter):
         bits = OpenBitSet(long(reader.maxDoc()))
         isbns = self.accessor.isbns()
 
-        docs = JArray(int)(1)
-        freqs = JArray(int)(1)
-
         for isbn in isbns:
             if isbn is not None:
-                termDocs = reader.termDocs(Term("isbn", isbn))
-                count = termDocs.read(docs, freqs)
+                docsEnum = reader.termDocsEnum(reader.getDeletedDocs(),
+                                               "isbn", BytesRef(isbn))
+                result = docsEnum.getBulkResult()
+                count = docsEnum.read()
                 if count == 1:
-                    bits.set(long(docs[0]))
+                    bits.set(long(result.docs.ints[0]))
 
         return bits
 

Modified: lucene/pylucene/trunk/test/BaseTokenStreamTestCase.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/BaseTokenStreamTestCase.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/BaseTokenStreamTestCase.py (original)
+++ lucene/pylucene/trunk/test/BaseTokenStreamTestCase.py Thu Jul  8 14:53:33 2010
@@ -25,10 +25,10 @@ class BaseTokenStreamTestCase(TestCase):
                                    types=None, posIncrements=None):
 
         self.assert_(output is not None)
-        self.assert_(ts.hasAttribute(TermAttribute.class_),
+        self.assert_(ts.hasAttribute(CharTermAttribute.class_),
                                      "has TermAttribute")
 
-        termAtt = ts.getAttribute(TermAttribute.class_)
+        termAtt = ts.getAttribute(CharTermAttribute.class_)
 
         offsetAtt = None
         if startOffsets is not None or endOffsets is not None:
@@ -53,7 +53,7 @@ class BaseTokenStreamTestCase(TestCase):
             # extra safety to enforce, that the state is not preserved and
             # also assign bogus values
             ts.clearAttributes()
-            termAtt.setTermBuffer("bogusTerm")
+            termAtt.append("bogusTerm")
             if offsetAtt is not None:
                 offsetAtt.setOffset(14584724, 24683243)
             if typeAtt is not None:
@@ -62,7 +62,7 @@ class BaseTokenStreamTestCase(TestCase):
                 posIncrAtt.setPositionIncrement(45987657)
       
             self.assert_(ts.incrementToken(), "token %d exists" %(i))
-            self.assertEqual(output[i], termAtt.term(), "term %d" %(i))
+            self.assertEqual(output[i], termAtt.toString(), "term %d" %(i))
             if startOffsets is not None:
                 self.assertEqual(startOffsets[i], offsetAtt.startOffset(),
                                  "startOffset %d" %(i))

Modified: lucene/pylucene/trunk/test/test_PerFieldAnalyzerWrapper.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_PerFieldAnalyzerWrapper.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_PerFieldAnalyzerWrapper.py (original)
+++ lucene/pylucene/trunk/test/test_PerFieldAnalyzerWrapper.py Thu Jul  8 14:53:33 2010
@@ -28,16 +28,16 @@ class PerFieldAnalyzerTestCase(TestCase)
         analyzer.addAnalyzer("special", SimpleAnalyzer())
 
         tokenStream = analyzer.tokenStream("field", StringReader(text))
-        termAtt = tokenStream.getAttribute(TermAttribute.class_)
+        termAtt = tokenStream.getAttribute(CharTermAttribute.class_)
 
         self.assert_(tokenStream.incrementToken())
-        self.assertEqual("Qwerty", termAtt.term(),
+        self.assertEqual("Qwerty", termAtt.toString(),
                          "WhitespaceAnalyzer does not lowercase")
 
         tokenStream = analyzer.tokenStream("special", StringReader(text))
-        termAtt = tokenStream.getAttribute(TermAttribute.class_)
+        termAtt = tokenStream.getAttribute(CharTermAttribute.class_)
         self.assert_(tokenStream.incrementToken())
-        self.assertEqual("qwerty", termAtt.term(),
+        self.assertEqual("qwerty", termAtt.toString(),
                          "SimpleAnalyzer lowercases")
 
 

Modified: lucene/pylucene/trunk/test/test_PositionIncrement.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_PositionIncrement.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_PositionIncrement.py (original)
+++ lucene/pylucene/trunk/test/test_PositionIncrement.py Thu Jul  8 14:53:33 2010
@@ -32,12 +32,13 @@ class PositionIncrementTestCase(TestCase
                         self_.INCREMENTS = [1, 2, 1, 0, 1]
                         self_.i = 0
                         self_.posIncrAtt = self_.addAttribute(PositionIncrementAttribute.class_)
-                        self_.termAtt = self_.addAttribute(TermAttribute.class_)
+                        self_.termAtt = self_.addAttribute(CharTermAttribute.class_)
                         self_.offsetAtt = self_.addAttribute(OffsetAttribute.class_)
                     def incrementToken(self_):
                         if self_.i == len(self_.TOKENS):
                             return False
-                        self_.termAtt.setTermBuffer(self_.TOKENS[self_.i])
+                        self_.termAtt.setEmpty()
+                        self_.termAtt.append(self_.TOKENS[self_.i])
                         self_.offsetAtt.setOffset(self_.i, self_.i)
                         self_.posIncrAtt.setPositionIncrement(self_.INCREMENTS[self_.i])
                         self_.i += 1
@@ -64,13 +65,17 @@ class PositionIncrementTestCase(TestCase
 
         searcher = IndexSearcher(store, True)
 
-        pos = searcher.getIndexReader().termPositions(Term("field", "1"))
-        pos.next()
+        pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(),
+                                               MultiFields.getDeletedDocs(searcher.getIndexReader()),
+                                               "field", BytesRef("1"))
+        pos.nextDoc()
         # first token should be at position 0
         self.assertEqual(0, pos.nextPosition())
     
-        pos = searcher.getIndexReader().termPositions(Term("field", "2"))
-        pos.next()
+        pos = MultiFields.getTermPositionsEnum(searcher.getIndexReader(),
+                                               MultiFields.getDeletedDocs(searcher.getIndexReader()),
+                                               "field", BytesRef("2"))
+        pos.nextDoc()
         # second token should be at position 2
         self.assertEqual(2, pos.nextPosition())
     
@@ -204,9 +209,12 @@ class PositionIncrementTestCase(TestCase
 
         r = writer.getReader()
 
-        tp = r.termPositions(Term("content", "a"))
+        tp = MultiFields.getTermPositionsEnum(r,
+                                              MultiFields.getDeletedDocs(r),
+                                              "content", BytesRef("a"))
+
         count = 0
-        self.assert_(tp.next())
+        self.assert_(tp.nextDoc() != tp.NO_MORE_DOCS)
         # "a" occurs 4 times
         self.assertEqual(4, tp.freq())
 
@@ -217,7 +225,7 @@ class PositionIncrementTestCase(TestCase
         self.assertEqual(6, tp.nextPosition())
 
         # only one doc has "a"
-        self.assert_(not tp.next())
+        self.assert_(tp.nextDoc() == tp.NO_MORE_DOCS)
 
         searcher = IndexSearcher(r)
     
@@ -305,7 +313,7 @@ class PayloadFilter(PythonTokenFilter):
         self.i = 0
         self.posIncrAttr = input.addAttribute(PositionIncrementAttribute.class_)
         self.payloadAttr = input.addAttribute(PayloadAttribute.class_)
-        self.termAttr = input.addAttribute(TermAttribute.class_)
+        self.termAttr = input.addAttribute(CharTermAttribute.class_)
 
     def incrementToken(self):
 

Modified: lucene/pylucene/trunk/test/test_PyLucene.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_PyLucene.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_PyLucene.py (original)
+++ lucene/pylucene/trunk/test/test_PyLucene.py Thu Jul  8 14:53:33 2010
@@ -249,12 +249,11 @@ class Test_PyLuceneBase(object):
         reader = None
         try:
             reader = IndexReader.open(store, True)
-            term_enum = reader.terms(Term("docid", ''))
+            term_enum = MultiFields.getTerms(reader, "docid").iterator()
             docids = []
 
-            while term_enum.term().field() == 'docid':
-                docids.append(term_enum.term().text())
-                term_enum.next()
+            for term in term_enum:
+                docids.append(term.utf8ToString())
             self.assertEqual(len(docids), 2)
         finally:
             pass

Modified: lucene/pylucene/trunk/test/test_StopAnalyzer.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_StopAnalyzer.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_StopAnalyzer.py (original)
+++ lucene/pylucene/trunk/test/test_StopAnalyzer.py Thu Jul  8 14:53:33 2010
@@ -33,10 +33,10 @@ class StopAnalyzerTestCase(TestCase):
         stream = self.stop.tokenStream("test", reader)
         self.assert_(stream is not None)
 
-        termAtt = stream.getAttribute(TermAttribute.class_)
+        termAtt = stream.getAttribute(CharTermAttribute.class_)
     
         while stream.incrementToken():
-            self.assert_(termAtt.term() not in self.invalidTokens)
+            self.assert_(termAtt.toString() not in self.invalidTokens)
 
     def testStopList(self):
 
@@ -50,11 +50,11 @@ class StopAnalyzerTestCase(TestCase):
         stream = newStop.tokenStream("test", reader)
         self.assert_(stream is not None)
 
-        termAtt = stream.getAttribute(TermAttribute.class_)
+        termAtt = stream.getAttribute(CharTermAttribute.class_)
         posIncrAtt = stream.addAttribute(PositionIncrementAttribute.class_)
     
         while stream.incrementToken():
-            text = termAtt.term()
+            text = termAtt.toString()
             self.assert_(text not in stopWordsSet)
             # by default stop tokenizer does not apply increments.
             self.assertEqual(1, posIncrAtt.getPositionIncrement())
@@ -73,11 +73,11 @@ class StopAnalyzerTestCase(TestCase):
         self.assert_(stream is not None)
 
         i = 0
-        termAtt = stream.getAttribute(TermAttribute.class_)
+        termAtt = stream.getAttribute(CharTermAttribute.class_)
         posIncrAtt = stream.addAttribute(PositionIncrementAttribute.class_)
 
         while stream.incrementToken():
-            text = termAtt.term()
+            text = termAtt.toString()
             self.assert_(text not in stopWordsSet)
             self.assertEqual(expectedIncr[i],
                              posIncrAtt.getPositionIncrement())

Modified: lucene/pylucene/trunk/test/test_bug1842.py
URL: http://svn.apache.org/viewvc/lucene/pylucene/trunk/test/test_bug1842.py?rev=961790&r1=961789&r2=961790&view=diff
==============================================================================
--- lucene/pylucene/trunk/test/test_bug1842.py (original)
+++ lucene/pylucene/trunk/test/test_bug1842.py Thu Jul  8 14:53:33 2010
@@ -44,7 +44,7 @@ class Test_Bug1842(unittest.TestCase):
         q = TermQuery(Term("id", '1'))
         topDocs = searcher.search(q, 50)
         freqvec = reader.getTermFreqVector(topDocs.scoreDocs[0].doc, "all")
-        terms = list(freqvec.getTerms())
+        terms = [term.utf8ToString() for term in freqvec.getTerms()]
         terms.sort()
         self.assert_(terms == ['blah', 'gesundheit'])
 



Mime
View raw message