LUCENE-3312: Merged revision(s) 1371132-1372428 from lucene/dev/trunk:

Add support to also reload HashFunctions when Solr boots (otherwise codecs using new hash functions may fail to load).
We may need a better "automatic" reloading with one method call, that reloads all NamedSPILoaders. I will think about it and open issue. Solr could then only call NamedSPILoader.reloadAll(), so all instantiated ones get reloaded automatically. Currently its to risky to add new SPIs without Solr support (because it cannot be tested).
........
LUCENE-4298: MultiFields.getTermDocsEnum(Reader,Bits,String,BytesRef) did not work at all
........
SOLR-3715: remove sync around tlog serialization
........
fix test msg
........
add recovery strat to commented out list of log configs for tests
........
change log format - core= rather than Core:
........
add some tests mixing up payload options
........
tests: specify a seed to Random()
........
tests: use random seed of doc base
........
set the SVN variables so this task can build the unsigned release
........
Remove smoke again, this is now a new ReleaseNightly job on ASF Jenkins (without scripts, only ANT). I will later remove the artifact building from the standard nightly scripts, too. Those will now only run tests and clover.
........
Smoketester should also run with OpenJDK to be able to run on FreeBSD
........
clean up BS2's Coordinator, use ctor instead of init, make coordFactors final
........
LUCENE-4300: BooleanQuery's rewrite was unsafe if coord(1,1) != 1
........
SOLR-3725: fix package-local-src-tgz to not bring in jars/binary content
........
LUCENE-4299: add Terms hasPositions/hasOffsets, so you know what features a docs TVs have
........
ignore silly bugs in sun's regex impl
........
LUCENE-4187: Die, shell scripts, die, die, die!
........
LUCENE-4187: Allow disabling validation tasks and javadocs-linting from jenkins
........
nuke useless scripts - part #1
........
LUCENE-4187: Merge clover reports of Lucene/Solr for easier and more useful Jenkins reporting
........
LUCENE-4187: Remove remaining clover reference
........
LUCENE-4187: Add jenkins clover task
........
LUCENE-4187: Nuke remaining shell scripts

........
LUCENE-4187: exclude backwards (when it comes)
........
LUCENE-4187: simplier pattern
........
LUCENE-4187: Fix title for clover
........
LUCENE-4187: Make clover configuration for jenkins fixed and faster
........
Don't allow negatives in the positions file
........
LUCENE-1888: added option to store payloads on the term vectors
........
throw IllegalArgumentException if you try to index term vector offsets/positions when term vectors are not indexed, or if you try to index term vectors when field is not index
........
fix test to actually compare index1 vs index2, not index1 against itself
........
LUCENE-3985: Upgrade to randomizedtesting 2.0.0. Added support for thread leak detection. Added support for suite timeouts. (Dawid Weiss)
........
fix test bug
........
re-enable test
........
Allow better debugging this test
........
fix clustering and DIH problems with conflicting servlet-api dependencies by making scope be provided in solr's test-framework POM
........
revisit payloads API in DocsAndPositionsEnum
........
maven build: reduce solr test logging output, to allow Jenkins mailer to succeed
........
disable test timeouts until nightly builds have been addressed
........
Fix test failure caused in Maven, because test-framework classes are packaged in JAR.
........


git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene3312@1372438 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2012-08-13 14:18:26 +00:00
commit ac38630855
173 changed files with 2842 additions and 2080 deletions

49
build-clover.xml Normal file
View File

@ -0,0 +1,49 @@
<?xml version="1.0"?>
<!--
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.
-->
<project name="clover" basedir=".">
<import file="lucene/common-build.xml"/>
<!--
Run after Junit tests.
This target is in a separate file, as it needs to include common-build.xml,
but must run from top-level!
-->
<target name="generate-clover-reports" depends="clover">
<fail unless="run.clover">Clover not enabled!</fail>
<mkdir dir="${clover.report.dir}"/>
<fileset dir="." id="clover.test.result.files">
<include name="*/build/**/test/TEST-*.xml"/>
<exclude name="lucene/build/backwards/**"/>
</fileset>
<clover-report>
<current outfile="${clover.report.dir}" title="${final.name}" numThreads="0">
<format type="html" filter="assert"/>
<testresults refid="clover.test.result.files"/>
</current>
<current outfile="${clover.report.dir}/clover.xml" title="${final.name}">
<format type="xml" filter="assert"/>
<testresults refid="clover.test.result.files"/>
</current>
</clover-report>
<echo>You can find the merged Lucene/Solr Clover report in '${clover.report.dir}'.</echo>
</target>
</project>

View File

@ -51,11 +51,28 @@
</sequential>
</target>
<target name="validate" description="Validate dependencies, licenses, etc.">
<sequential><subant target="validate" inheritall="false" failonerror="true">
<fileset dir="lucene" includes="build.xml" />
<fileset dir="solr" includes="build.xml" />
</subant></sequential>
<target name="validate" description="Validate dependencies, licenses, etc." depends="-validate-source-patterns">
<subant target="validate" inheritall="false" failonerror="true">
<fileset dir="lucene" includes="build.xml" />
<fileset dir="solr" includes="build.xml" />
</subant>
</target>
<target name="-validate-source-patterns" unless="disable.source-patterns">
<!-- check that there are no nocommits or @author javadoc tags: -->
<property name="validate.currDir" location="."/>
<pathconvert pathsep="${line.separator}" dirsep="/" property="validate.patternsFound" setonempty="false">
<fileset dir="${validate.currDir}">
<include name="**/*.java"/>
<exclude name="**/backwards/**"/>
<or>
<containsregexp expression="@author\b" casesensitive="yes"/>
<containsregexp expression="\bno(n|)commit\b" casesensitive="no"/>
</or>
</fileset>
<map from="${validate.currDir}${file.separator}" to="* "/>
</pathconvert>
<fail if="validate.patternsFound">The following files contain @author tags or nocommits:${line.separator}${validate.patternsFound}</fail>
</target>
<target name="rat-sources" description="Runs rat across all sources and tests">
@ -185,15 +202,16 @@
</sequential>
</target>
<!-- define here, as common-build is not included! -->
<property name="python32.exe" value="python3.2" />
<property name="JAVA6_HOME" value="/usr/local/jdk1.6.0_27"/>
<property name="JAVA7_HOME" value="/usr/local/jdk1.7.0_01"/>
<property name="fakeRelease" value="lucene/build/fakeRelease"/>
<property name="fakeReleaseTmp" value="lucene/build/fakeReleaseTmp"/>
<property name="fakeReleaseVersion" value="5.0"/> <!-- *not* -SNAPSHOT, the real version -->
<target name="nightly-smoke" description="Builds an unsigned release and smoke tests it." depends="clean">
<sequential>
<fail unless="JAVA6_HOME">JAVA6_HOME property is not defined.</fail>
<fail unless="JAVA7_HOME">JAVA7_HOME property is not defined.</fail>
<subant target="prepare-release-no-sign" inheritall="false" failonerror="true">
<fileset dir="lucene" includes="build.xml" />
<fileset dir="solr" includes="build.xml" />
@ -229,4 +247,65 @@
<delete dir="${fakeReleaseTmp}"/>
</sequential>
</target>
<!-- Calls only generate-clover-reports on Lucene, as Solr's is just a clone with other target; the database itsself is fixed -->
<target name="generate-clover-reports">
<subant target="generate-clover-reports" inheritall="false" failonerror="true">
<fileset dir="." includes="build-clover.xml" />
</subant>
</target>
<!-- Jenkins tasks -->
<target name="jenkins-hourly" depends="clean,test,validate,-jenkins-javadocs-lint,-svn-status"/>
<target name="jenkins-clover">
<antcall target="-jenkins-clover">
<param name="run.clover" value="true"/>
<!-- must be 1, as clover does not like parallel test runs: -->
<param name="tests.jvms" value="1"/>
<!-- Also override some other props to be fast, ignoring what's set on command line: -->
<param name="tests.multiplier" value="1"/>
<param name="tests.slow" value="false"/>
<param name="tests.nightly" value="false"/>
<param name="tests.weekly" value="false"/>
<param name="tests.multiplier" value="1"/>
</antcall>
</target>
<target name="-jenkins-clover" depends="clean,test,generate-clover-reports"/>
<!-- we need this extra condition, as we want to match only on "true", not solely if property is set: -->
<property name="disable.javadocs-lint" value="false" />
<condition property="-disable.javadocs-lint">
<equals arg1="${disable.javadocs-lint}" arg2="true"/>
</condition>
<target name="-jenkins-javadocs-lint" unless="-disable.javadocs-lint">
<antcall target="javadocs-lint"/>
</target>
<!-- define here, as common-build is not included! -->
<property name="svn.exe" value="svn" />
<target name="-svn-status">
<exec executable="${svn.exe}" dir="." failonerror="true">
<arg value="status"/>
<redirector outputproperty="svn.status.output">
<outputfilterchain>
<linecontainsregexp>
<regexp pattern="^\?" />
</linecontainsregexp>
<tokenfilter>
<replaceregex pattern="^........" replace="* " />
<replacestring from="${file.separator}" to="/" />
</tokenfilter>
</outputfilterchain>
</redirector>
</exec>
<fail message="Source checkout is dirty after running tests!!! Offending files:${line.separator}${svn.status.output}">
<condition>
<not>
<equals arg1="${svn.status.output}" arg2=""/>
</not>
</condition>
</fail>
</target>
</project>

View File

@ -174,6 +174,6 @@
<classpathentry kind="lib" path="solr/contrib/velocity/lib/commons-beanutils-1.7.0.jar"/>
<classpathentry kind="lib" path="solr/contrib/velocity/lib/commons-collections-3.2.1.jar"/>
<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER"/>
<classpathentry kind="lib" path="lucene/test-framework/lib/randomizedtesting-runner-1.6.0.jar"/>
<classpathentry kind="lib" path="lucene/test-framework/lib/randomizedtesting-runner-2.0.0.rc5.jar"/>
<classpathentry kind="output" path="bin/other"/>
</classpath>

View File

@ -2,7 +2,7 @@
<library name="JUnit">
<CLASSES>
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/junit-4.10.jar!/" />
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/randomizedtesting-runner-1.6.0.jar!/" />
<root url="jar://$PROJECT_DIR$/lucene/test-framework/lib/randomizedtesting-runner-2.0.0.rc5.jar!/" />
</CLASSES>
<JAVADOC />
<SOURCES />

View File

@ -385,7 +385,7 @@
<dependency>
<groupId>com.carrotsearch.randomizedtesting</groupId>
<artifactId>randomizedtesting-runner</artifactId>
<version>1.6.0</version>
<version>2.0.0.rc5</version>
</dependency>
</dependencies>
</dependencyManagement>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/analysis-extras</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -95,17 +94,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/clustering</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -100,17 +99,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/dataimporthandler-extras</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -98,17 +97,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/dataimporthandler</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -84,6 +83,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
@ -97,15 +102,6 @@
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -38,7 +38,6 @@
<module-directory>solr/contrib/extraction</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -96,17 +95,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -39,7 +39,6 @@
<module-directory>solr/contrib/langid</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -101,17 +100,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/uima</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -115,17 +114,12 @@
<testResource>
<directory>${module-path}/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/contrib/velocity</module-directory>
<top-level>../../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -136,17 +135,12 @@
<testResource>
<directory>${top-level}/solr/core/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -35,7 +35,6 @@
<module-directory>solr/core</module-directory>
<top-level>../../..</top-level>
<module-path>${top-level}/${module-directory}</module-path>
<surefire-top-level>${top-level}/../..</surefire-top-level>
</properties>
<scm>
<connection>scm:svn:${vc-anonymous-base-url}/${module-directory}</connection>
@ -237,17 +236,14 @@
<testResource>
<directory>${top-level}/solr/solrj/src/test-files</directory>
</testResource>
<testResource>
<directory>${top-level}/dev-tools/maven/solr</directory>
<includes>
<include>maven.testlogging.properties</include>
</includes>
</testResource>
</testResources>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>${surefire-top-level}/solr/testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>

View File

@ -0,0 +1,2 @@
handlers=java.util.logging.ConsoleHandler
.level=SEVERE

View File

@ -99,6 +99,15 @@
<doctitle>${project.name} ${project.version} API (${now.version})</doctitle>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<java.util.logging.config.file>../test-classes/maven.testlogging.properties</java.util.logging.config.file>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</pluginManagement>
</build>

View File

@ -57,6 +57,8 @@
<dependency>
<groupId>javax.servlet</groupId>
<artifactId>servlet-api</artifactId>
<!-- SOLR-3263: Provided scope is required to avoid jar signing conflicts -->
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>

View File

@ -58,7 +58,7 @@ def javaExe(version):
def verifyJavaVersion(version):
s = os.popen('%s; java -version 2>&1' % javaExe(version)).read()
if s.find('java version "%s.' % version) == -1:
if s.find(' version "%s.' % version) == -1:
raise RuntimeError('got wrong version for java %s:\n%s' % (version, s))
# http://s.apache.org/lusolr32rc2

View File

@ -8,6 +8,29 @@ http://s.apache.org/luceneversions
======================= Lucene 4.0.0 =======================
New Features
* LUCENE-1888: Added the option to store payloads in the term
vectors (IndexableFieldType.storeTermVectorPayloads()). Note
that you must store term vector positions to store payloads.
(Robert Muir)
API Changes
* LUCENE-4299: Added Terms.hasPositions() and Terms.hasOffsets().
Previously you had no real way to know that a term vector field
had positions or offsets, since this can be configured on a
per-field-per-document basis. (Robert Muir)
* Removed DocsAndPositionsEnum.hasPayload() and simplified the
contract of getPayload(). It returns null if there is no payload,
otherwise returns the current payload. You can now call it multiple
times per position if you want. (Robert Muir)
* Removed FieldsEnum. Fields API instead implements Iterable<String>
and exposes Iterator, so you can iterate over field names with
for (String field : fields) instead. (Robert Muir)
Bug Fixes
* LUCENE-4297: BooleanScorer2 would multiply the coord() factor
@ -16,6 +39,24 @@ Bug Fixes
than 1 when overlap == maxOverlap (always the case for conjunctions),
then the score would be incorrect. (Pascal Chollet, Robert Muir)
* LUCENE-4298: MultiFields.getTermDocsEnum(IndexReader, Bits, String, BytesRef)
did not work at all, it would infinitely recurse.
(Alberto Paro via Robert Muir)
* LUCENE-4300: BooleanQuery's rewrite was not always safe: if you
had a custom Similarity where coord(1,1) != 1F, then the rewritten
query would be scored differently. (Robert Muir)
* Don't allow negatives in the positions file. If you have an index
from 2.4.0 or earlier with such negative positions, and you already
upgraded to 3.x, then to Lucene 4.0-ALPHA or -BETA, you should run
CheckIndex. If it fails, then you need to upgrade again to 4.0 (Robert Muir)
Build
* LUCENE-3985: Upgrade to randomizedtesting 2.0.0. Added support for
thread leak detection. Added support for suite timeouts. (Dawid Weiss)
======================= Lucene 4.0.0-BETA =======================
New features

View File

@ -9,7 +9,7 @@ enumeration APIs. Here are the major changes:
by the BytesRef class (which provides an offset + length "slice"
into an existing byte[]).
* Fields are separately enumerated (FieldsEnum) from the terms
* Fields are separately enumerated (Fields.iterator()) from the terms
within each field (TermEnum). So instead of this:
TermEnum termsEnum = ...;
@ -20,10 +20,8 @@ enumeration APIs. Here are the major changes:
Do this:
FieldsEnum fieldsEnum = ...;
String field;
while((field = fieldsEnum.next()) != null) {
TermsEnum termsEnum = fieldsEnum.terms();
for(String field : fields) {
TermsEnum termsEnum = fields.terms(field);
BytesRef text;
while((text = termsEnum.next()) != null) {
System.out.println("field=" + field + "; text=" + text.utf8ToString());

View File

@ -100,8 +100,7 @@ public class TestAllAnalyzersHaveFactories extends LuceneTestCase {
private static final ResourceLoader loader = new StringMockResourceLoader("");
public void test() throws Exception {
List<Class<?>> analysisClasses = new ArrayList<Class<?>>();
TestRandomChains.getClassesForPackage("org.apache.lucene.analysis", analysisClasses);
List<Class<?>> analysisClasses = TestRandomChains.getClassesForPackage("org.apache.lucene.analysis");
for (final Class<?> c : analysisClasses) {
final int modifiers = c.getModifiers();

View File

@ -25,6 +25,7 @@ import java.io.StringReader;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Modifier;
import java.net.URI;
import java.net.URL;
import java.nio.CharBuffer;
import java.util.ArrayList;
@ -165,8 +166,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
@BeforeClass
public static void beforeClass() throws Exception {
List<Class<?>> analysisClasses = new ArrayList<Class<?>>();
getClassesForPackage("org.apache.lucene.analysis", analysisClasses);
List<Class<?>> analysisClasses = getClassesForPackage("org.apache.lucene.analysis");
tokenizers = new ArrayList<Constructor<? extends Tokenizer>>();
tokenfilters = new ArrayList<Constructor<? extends TokenFilter>>();
charfilters = new ArrayList<Constructor<? extends CharFilter>>();
@ -235,19 +235,30 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
private static <T> Constructor<T> castConstructor(Class<T> instanceClazz, Constructor<?> ctor) {
return (Constructor<T>) ctor;
}
static void getClassesForPackage(String pckgname, List<Class<?>> classes) throws Exception {
public static List<Class<?>> getClassesForPackage(String pckgname) throws Exception {
final List<Class<?>> classes = new ArrayList<Class<?>>();
collectClassesForPackage(pckgname, classes);
assertFalse("No classes found in package '"+pckgname+"'; maybe your test classes are packaged as JAR file?", classes.isEmpty());
return classes;
}
private static void collectClassesForPackage(String pckgname, List<Class<?>> classes) throws Exception {
final ClassLoader cld = TestRandomChains.class.getClassLoader();
final String path = pckgname.replace('.', '/');
final Enumeration<URL> resources = cld.getResources(path);
while (resources.hasMoreElements()) {
final File directory = new File(resources.nextElement().toURI());
final URI uri = resources.nextElement().toURI();
if (!"file".equalsIgnoreCase(uri.getScheme()))
continue;
final File directory = new File(uri);
if (directory.exists()) {
String[] files = directory.list();
for (String file : files) {
if (new File(directory, file).isDirectory()) {
// recurse
String subPackage = pckgname + "." + file;
getClassesForPackage(subPackage, classes);
collectClassesForPackage(subPackage, classes);
}
if (file.endsWith(".class")) {
String clazzName = file.substring(0, file.length() - 6);

View File

@ -43,7 +43,6 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@ -206,6 +205,7 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
// 1. alg definition (required in every "logic" test)
String algLines[] = {
"doc.stored=true",//doc storage is required in order to have text to highlight
"doc.term.vector=true",
"doc.term.vector.offsets=true",
"content.source=org.apache.lucene.benchmark.byTask.feeds.LineDocSource",
"docs.file=" + getReuters20LinesFile(),
@ -492,13 +492,13 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
int totalTokenCount2 = 0;
FieldsEnum fields = MultiFields.getFields(reader).iterator();
String fieldName = null;
while((fieldName = fields.next()) != null) {
Fields fields = MultiFields.getFields(reader);
for (String fieldName : fields) {
if (fieldName.equals(DocMaker.ID_FIELD) || fieldName.equals(DocMaker.DATE_MSEC_FIELD) || fieldName.equals(DocMaker.TIME_SEC_FIELD)) {
continue;
}
Terms terms = fields.terms();
Terms terms = fields.terms(fieldName);
if (terms == null) {
continue;
}

View File

@ -139,29 +139,6 @@
<target name="compile-core" depends="compile-lucene-core"/>
<!--
Run after Junit tests.
-->
<target name="generate-clover-reports" depends="clover">
<fail unless="run.clover">Clover not enabled!</fail>
<mkdir dir="${clover.report.dir}"/>
<fileset dir="build" id="clover.test.result.files">
<include name="**/test/TEST-*.xml"/>
<!-- do not include BW tests -->
<exclude name="backwards/**"/>
</fileset>
<clover-report>
<current outfile="${clover.report.dir}" title="${final.name}" numThreads="0">
<format type="html" filter="assert"/>
<testresults refid="clover.test.result.files"/>
</current>
<current outfile="${clover.report.dir}/clover.xml" title="${final.name}">
<format type="xml" filter="assert"/>
<testresults refid="clover.test.result.files"/>
</current>
</clover-report>
</target>
<!-- Validation (license/notice/api checks). -->
<target name="validate" depends="check-licenses,rat-sources,check-forbidden-apis" description="Validate stuff." />
@ -176,6 +153,7 @@
<apiFileSet dir="${custom-tasks.dir}/forbiddenApis">
<include name="jdk.txt" />
<include name="jdk-deprecated.txt" />
<include name="executors.txt" />
</apiFileSet>
<fileset dir="${basedir}/build" includes="**/*.class" />
</forbidden-apis>

View File

@ -88,7 +88,7 @@
<property name="tests.timezone" value="random" />
<property name="tests.directory" value="random" />
<property name="tests.linedocsfile" value="europarl.lines.txt.gz" />
<property name="tests.loggingfile" value="/dev/null"/>
<property name="tests.loggingfile" value="${common.dir}/tools/junit4/logging.properties"/>
<property name="tests.nightly" value="false" />
<property name="tests.weekly" value="false" />
<property name="tests.slow" value="true" />
@ -700,15 +700,22 @@
<condition property="tests.method" value="${testmethod}*">
<isset property="testmethod" />
</condition>
<condition property="tests.showSuccess" value="true">
<or>
<isset property="tests.class" />
<isset property="tests.method" />
</or>
</condition>
<!-- default -->
<property name="tests.showSuccess" value="false"/>
<condition property="tests.showOutput" value="always">
<or>
<isset property="tests.class" />
<isset property="tests.method" />
</or>
</condition>
<property name="tests.showOutput" value="onerror"/>
<!-- Test macro using junit4. -->
<macrodef name="test-macro" description="Executes junit tests.">
@ -854,6 +861,7 @@
<syspropertyset>
<propertyref prefix="tests.maxfailures" />
<propertyref prefix="tests.failfast" />
<propertyref prefix="tests.badapples" />
</syspropertyset>
<!-- Pass randomized settings to the forked JVM. -->
@ -875,8 +883,7 @@
<junit4:report-text
showThrowable="true"
showStackTraces="true"
showOutputStream="true"
showErrorStream="true"
showOutput="${tests.showOutput}"
showStatusOk="${tests.showSuccess}"
showStatusError="${tests.showError}"
@ -896,8 +903,7 @@
file="@{junit.output.dir}/tests-report.txt"
showThrowable="true"
showStackTraces="true"
showOutputStream="true"
showErrorStream="true"
showOutput="always"
showStatusOk="true"
showStatusError="true"
@ -913,8 +919,7 @@
file="@{junit.output.dir}/tests-failures.txt"
showThrowable="true"
showStackTraces="true"
showOutputStream="true"
showErrorStream="true"
showOutput="onerror"
showStatusOk="false"
showStatusError="true"
@ -929,8 +934,13 @@
the slowest tests or for reuse in balancing). -->
<junit4:report-execution-times file="@{junit.output.dir}/tests-timehints.txt" historyLength="5" />
<junit4:report-ant-xml dir="@{junit.output.dir}" />
<junit4:report-json file="@{junit.output.dir}/tests-report-${ant.project.name}/index.html" />
<!-- ANT-compatible XMLs for jenkins records etc. -->
<junit4:report-ant-xml dir="@{junit.output.dir}" outputStreams="no" />
<!--
Enable if you wish to have a nice HTML5 report.
<junit4:report-json file="@{junit.output.dir}/tests-report-${ant.project.name}/index.html" outputStreams="no" />
-->
</listeners>
<!-- Input test classes. -->

View File

@ -27,7 +27,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
@ -40,6 +39,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.DoubleBarrelLRUCache;
import org.apache.lucene.util.UnmodifiableIterator;
/** Handles a terms dict, but decouples all details of
* doc/freqs/positions reading to an instance of {@link
@ -184,8 +184,8 @@ public class BlockTermsReader extends FieldsProducer {
}
@Override
public FieldsEnum iterator() {
return new TermFieldsEnum();
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override
@ -199,32 +199,6 @@ public class BlockTermsReader extends FieldsProducer {
return fields.size();
}
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
FieldReader current;
TermFieldsEnum() {
it = fields.values().iterator();
}
@Override
public String next() {
if (it.hasNext()) {
current = it.next();
return current.fieldInfo.name;
} else {
current = null;
return null;
}
}
@Override
public Terms terms() throws IOException {
return current;
}
}
private class FieldReader extends Terms {
final long numTerms;
final FieldInfo fieldInfo;
@ -253,6 +227,21 @@ public class BlockTermsReader extends FieldsProducer {
return new SegmentTermsEnum();
}
@Override
public boolean hasOffsets() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldInfo.hasPayloads();
}
@Override
public long size() {
return numTerms;

View File

@ -31,7 +31,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
@ -46,6 +45,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnmodifiableIterator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
@ -199,8 +199,8 @@ public class BlockTreeTermsReader extends FieldsProducer {
}
@Override
public FieldsEnum iterator() {
return new TermFieldsEnum();
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override
@ -214,32 +214,6 @@ public class BlockTreeTermsReader extends FieldsProducer {
return fields.size();
}
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
FieldReader current;
TermFieldsEnum() {
it = fields.values().iterator();
}
@Override
public String next() {
if (it.hasNext()) {
current = it.next();
return current.fieldInfo.name;
} else {
current = null;
return null;
}
}
@Override
public Terms terms() throws IOException {
return current;
}
}
// for debugging
String brToString(BytesRef b) {
if (b == null) {
@ -456,6 +430,21 @@ public class BlockTreeTermsReader extends FieldsProducer {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public boolean hasOffsets() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldInfo.hasPayloads();
}
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
return new SegmentTermsEnum();

View File

@ -22,7 +22,6 @@ import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentWriteState; // javadocs
import org.apache.lucene.index.Terms;
@ -53,13 +52,10 @@ public abstract class FieldsConsumer implements Closeable {
public abstract void close() throws IOException;
public void merge(MergeState mergeState, Fields fields) throws IOException {
FieldsEnum fieldsEnum = fields.iterator();
assert fieldsEnum != null;
String field;
while((field = fieldsEnum.next()) != null) {
for (String field : fields) {
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field;
Terms terms = fieldsEnum.terms();
Terms terms = fields.terms(field);
if (terms != null) {
final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
termsConsumer.merge(mergeState, terms.iterator(null));

View File

@ -124,15 +124,17 @@ public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum
@Override
public BytesRef getPayload() throws IOException {
BytesRef payload = current.getPayload();
if (mergeState.currentPayloadProcessor[upto] != null) {
if (mergeState.currentPayloadProcessor[upto] != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
// TODO: reuse a BytesRef if there is a PPP
payload = BytesRef.deepCopyOf(payload);
mergeState.currentPayloadProcessor[upto].processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
return null;
}
}
return payload;
}
@Override
public boolean hasPayload() {
return current.hasPayload();
}
}

View File

@ -112,12 +112,7 @@ public abstract class PostingsConsumer {
totTF += freq;
for(int i=0;i<freq;i++) {
final int position = postingsEnum.nextPosition();
final BytesRef payload;
if (postingsEnum.hasPayload()) {
payload = postingsEnum.getPayload();
} else {
payload = null;
}
final BytesRef payload = postingsEnum.getPayload();
this.addPosition(position, payload, -1, -1);
}
this.finishDoc();
@ -137,12 +132,7 @@ public abstract class PostingsConsumer {
totTF += freq;
for(int i=0;i<freq;i++) {
final int position = postingsEnum.nextPosition();
final BytesRef payload;
if (postingsEnum.hasPayload()) {
payload = postingsEnum.getPayload();
} else {
payload = null;
}
final BytesRef payload = postingsEnum.getPayload();
this.addPosition(position, payload, postingsEnum.startOffset(), postingsEnum.endOffset());
}
this.finishDoc();

View File

@ -26,8 +26,9 @@ import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
@ -41,14 +42,14 @@ import org.apache.lucene.util.BytesRef;
* <ol>
* <li>For every document, {@link #startDocument(int)} is called,
* informing the Codec how many fields will be written.
* <li>{@link #startField(FieldInfo, int, boolean, boolean)} is called for
* <li>{@link #startField(FieldInfo, int, boolean, boolean, boolean)} is called for
* each field in the document, informing the codec how many terms
* will be written for that field, and whether or not positions
* or offsets are enabled.
* will be written for that field, and whether or not positions,
* offsets, or payloads are enabled.
* <li>Within each field, {@link #startTerm(BytesRef, int)} is called
* for each term.
* <li>If offsets and/or positions are enabled, then
* {@link #addPosition(int, int, int)} will be called for each term
* {@link #addPosition(int, int, int, BytesRef)} will be called for each term
* occurrence.
* <li>After all documents have been written, {@link #finish(FieldInfos, int)}
* is called for verification/sanity-checks.
@ -60,7 +61,7 @@ import org.apache.lucene.util.BytesRef;
public abstract class TermVectorsWriter implements Closeable {
/** Called before writing the term vectors of the document.
* {@link #startField(FieldInfo, int, boolean, boolean)} will
* {@link #startField(FieldInfo, int, boolean, boolean, boolean)} will
* be called <code>numVectorFields</code> times. Note that if term
* vectors are enabled, this is called even if the document
* has no vector fields, in this case <code>numVectorFields</code>
@ -69,17 +70,17 @@ public abstract class TermVectorsWriter implements Closeable {
/** Called before writing the terms of the field.
* {@link #startTerm(BytesRef, int)} will be called <code>numTerms</code> times. */
public abstract void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException;
public abstract void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException;
/** Adds a term and its term frequency <code>freq</code>.
* If this field has positions and/or offsets enabled, then
* {@link #addPosition(int, int, int)} will be called
* {@link #addPosition(int, int, int, BytesRef)} will be called
* <code>freq</code> times respectively.
*/
public abstract void startTerm(BytesRef term, int freq) throws IOException;
/** Adds a term position and offsets */
public abstract void addPosition(int position, int startOffset, int endOffset) throws IOException;
public abstract void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException;
/** Aborts writing entirely, implementation should remove
* any partially-written files, etc. */
@ -99,7 +100,7 @@ public abstract class TermVectorsWriter implements Closeable {
* This is an expert API that allows the codec to consume
* positions and offsets directly from the indexer.
* <p>
* The default implementation calls {@link #addPosition(int, int, int)},
* The default implementation calls {@link #addPosition(int, int, int, BytesRef)},
* but subclasses can override this if they want to efficiently write
* all the positions, then all the offsets, for example.
* <p>
@ -111,15 +112,36 @@ public abstract class TermVectorsWriter implements Closeable {
public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
int position = 0;
int lastOffset = 0;
BytesRef payload = null;
for (int i = 0; i < numProx; i++) {
final int startOffset;
final int endOffset;
final BytesRef thisPayload;
if (positions == null) {
position = -1;
thisPayload = null;
} else {
position += positions.readVInt();
int code = positions.readVInt();
position += code >>> 1;
if ((code & 1) != 0) {
// This position has a payload
final int payloadLength = positions.readVInt();
if (payload == null) {
payload = new BytesRef();
payload.bytes = new byte[payloadLength];
} else if (payload.bytes.length < payloadLength) {
payload.grow(payloadLength);
}
positions.readBytes(payload.bytes, 0, payloadLength);
payload.length = payloadLength;
thisPayload = payload;
} else {
thisPayload = null;
}
}
if (offsets == null) {
@ -129,24 +151,31 @@ public abstract class TermVectorsWriter implements Closeable {
endOffset = startOffset + offsets.readVInt();
lastOffset = endOffset;
}
addPosition(position, startOffset, endOffset);
addPosition(position, startOffset, endOffset, thisPayload);
}
}
/** Merges in the term vectors from the readers in
* <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)},
* {@link #startField(FieldInfo, int, boolean, boolean)},
* {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int)},
* {@link #startField(FieldInfo, int, boolean, boolean, boolean)},
* {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int, BytesRef)},
* and {@link #finish(FieldInfos, int)},
* returning the number of documents that were written.
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
public int merge(MergeState mergeState) throws IOException {
int docCount = 0;
for (AtomicReader reader : mergeState.readers) {
for (int i = 0; i < mergeState.readers.size(); i++) {
final AtomicReader reader = mergeState.readers.get(i);
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
for (int docID = 0; docID < maxDoc; docID++) {
if (liveDocs != null && !liveDocs.get(docID)) {
// skip deleted docs
@ -155,7 +184,7 @@ public abstract class TermVectorsWriter implements Closeable {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.getTermVectors(docID);
addAllDocVectors(vectors, mergeState.fieldInfos);
addAllDocVectors(vectors, mergeState);
docCount++;
mergeState.checkAbort.work(300);
}
@ -169,7 +198,7 @@ public abstract class TermVectorsWriter implements Closeable {
* implementation requires that the vectors implement
* both Fields.size and
* Terms.size. */
protected final void addAllDocVectors(Fields vectors, FieldInfos fieldInfos) throws IOException {
protected final void addAllDocVectors(Fields vectors, MergeState mergeState) throws IOException {
if (vectors == null) {
startDocument(0);
return;
@ -181,54 +210,55 @@ public abstract class TermVectorsWriter implements Closeable {
}
startDocument(numFields);
final FieldsEnum fieldsEnum = vectors.iterator();
String fieldName;
String lastFieldName = null;
TermsEnum termsEnum = null;
DocsAndPositionsEnum docsAndPositionsEnum = null;
final ReaderPayloadProcessor readerPayloadProcessor = mergeState.currentReaderPayloadProcessor;
PayloadProcessor payloadProcessor = null;
while((fieldName = fieldsEnum.next()) != null) {
final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldName);
for(String fieldName : vectors) {
final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
lastFieldName = fieldName;
final Terms terms = fieldsEnum.terms();
final Terms terms = vectors.terms(fieldName);
if (terms == null) {
// FieldsEnum shouldn't lie...
continue;
}
final boolean hasPositions = terms.hasPositions();
final boolean hasOffsets = terms.hasOffsets();
final boolean hasPayloads = terms.hasPayloads();
assert !hasPayloads || hasPositions;
final int numTerms = (int) terms.size();
if (numTerms == -1) {
throw new IllegalStateException("terms.size() must be implemented (it returned -1)");
}
final TermsEnum termsEnum = terms.iterator(null);
DocsAndPositionsEnum docsAndPositionsEnum = null;
boolean startedField = false;
// NOTE: this is tricky, because TermVectors allow
// indexing offsets but NOT positions. So we must
// lazily init the field by checking whether first
// position we see is -1 or not.
startField(fieldInfo, numTerms, hasPositions, hasOffsets, hasPayloads);
termsEnum = terms.iterator(termsEnum);
int termCount = 0;
while(termsEnum.next() != null) {
termCount++;
final int freq = (int) termsEnum.totalTermFreq();
if (startedField) {
startTerm(termsEnum.term(), freq);
startTerm(termsEnum.term(), freq);
if (hasPayloads && readerPayloadProcessor != null) {
payloadProcessor = readerPayloadProcessor.getProcessor(fieldName, termsEnum.term());
}
// TODO: we need a "query" API where we can ask (via
// flex API) what this term was indexed with...
// Both positions & offsets:
docsAndPositionsEnum = termsEnum.docsAndPositions(null, null);
boolean hasOffsets = false;
boolean hasPositions = false;
if (docsAndPositionsEnum != null) {
if (hasPositions || hasOffsets) {
docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
assert docsAndPositionsEnum != null;
final int docID = docsAndPositionsEnum.nextDoc();
assert docID != DocIdSetIterator.NO_MORE_DOCS;
assert docsAndPositionsEnum.freq() == freq;
@ -237,27 +267,21 @@ public abstract class TermVectorsWriter implements Closeable {
final int pos = docsAndPositionsEnum.nextPosition();
final int startOffset = docsAndPositionsEnum.startOffset();
final int endOffset = docsAndPositionsEnum.endOffset();
if (!startedField) {
assert numTerms > 0;
hasPositions = pos != -1;
hasOffsets = startOffset != -1;
startField(fieldInfo, numTerms, hasPositions, hasOffsets);
startTerm(termsEnum.term(), freq);
startedField = true;
}
if (hasOffsets) {
assert startOffset != -1;
assert endOffset != -1;
BytesRef payload = docsAndPositionsEnum.getPayload();
if (payloadProcessor != null && payload != null) {
// to not violate the D&P api, we must give the processor a private copy
payload = BytesRef.deepCopyOf(payload);
payloadProcessor.processPayload(payload);
if (payload.length == 0) {
// don't let PayloadProcessors corrumpt the index
payload = null;
}
}
assert !hasPositions || pos >= 0;
addPosition(pos, startOffset, endOffset);
}
} else {
if (!startedField) {
assert numTerms > 0;
startField(fieldInfo, numTerms, hasPositions, hasOffsets);
startTerm(termsEnum.term(), freq);
startedField = true;
addPosition(pos, startOffset, endOffset, payload);
}
}
}

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -35,7 +36,6 @@ import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -44,7 +44,6 @@ import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.AttributeSource;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FuzzySet;
@ -187,9 +186,8 @@ public class BloomFilteringPostingsFormat extends PostingsFormat {
}
public FieldsEnum iterator() throws IOException {
return new BloomFilteredFieldsEnum(delegateFieldsProducer.iterator(),
bloomsByFieldName);
public Iterator<String> iterator() {
return delegateFieldsProducer.iterator();
}
public void close() throws IOException {
@ -217,44 +215,6 @@ public class BloomFilteringPostingsFormat extends PostingsFormat {
return delegateFieldsProducer.getUniqueTermCount();
}
// Not all fields in a segment may be subject to a bloom filter. This class
// wraps Terms objects appropriately if a filtering request is present
class BloomFilteredFieldsEnum extends FieldsEnum {
private FieldsEnum delegateFieldsEnum;
private HashMap<String,FuzzySet> bloomsByFieldName;
private String currentFieldName;
public BloomFilteredFieldsEnum(FieldsEnum iterator,
HashMap<String,FuzzySet> bloomsByFieldName) {
this.delegateFieldsEnum = iterator;
this.bloomsByFieldName = bloomsByFieldName;
}
public AttributeSource attributes() {
return delegateFieldsEnum.attributes();
}
public String next() throws IOException {
currentFieldName = delegateFieldsEnum.next();
return currentFieldName;
}
public Terms terms() throws IOException {
FuzzySet filter = bloomsByFieldName.get(currentFieldName);
if (filter == null) {
return delegateFieldsEnum.terms();
} else {
Terms result = delegateFieldsEnum.terms();
if (result == null) {
return null;
}
// wrap the terms object with a bloom filter
return new BloomFilteredTerms(result, filter);
}
}
}
class BloomFilteredTerms extends Terms {
private Terms delegateTerms;
private FuzzySet filter;
@ -314,6 +274,21 @@ public class BloomFilteringPostingsFormat extends PostingsFormat {
public int getDocCount() throws IOException {
return delegateTerms.getDocCount();
}
@Override
public boolean hasOffsets() {
return delegateTerms.hasOffsets();
}
@Override
public boolean hasPositions() {
return delegateTerms.hasPositions();
}
@Override
public boolean hasPayloads() {
return delegateTerms.hasPayloads();
}
}
class BloomFilteredTermsEnum extends TermsEnum {

View File

@ -873,12 +873,7 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
* payload was indexed. */
@Override
public BytesRef getPayload() throws IOException {
throw new IOException("No payloads exist for this field!");
}
@Override
public boolean hasPayload() {
return false;
return null;
}
}
@ -1152,28 +1147,26 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
@Override
public BytesRef getPayload() throws IOException {
if (storePayloads) {
if (payloadLength <= 0) {
return null;
}
assert lazyProxPointer == -1;
assert posPendingCount < freq;
if (!payloadPending) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
}
if (payloadLength > payload.bytes.length) {
payload.grow(payloadLength);
}
if (payloadPending) {
if (payloadLength > payload.bytes.length) {
payload.grow(payloadLength);
}
proxIn.readBytes(payload.bytes, 0, payloadLength);
payload.length = payloadLength;
payloadPending = false;
proxIn.readBytes(payload.bytes, 0, payloadLength);
payload.length = payloadLength;
payloadPending = false;
}
return payload;
} else {
throw new IOException("No payloads exist for this field!");
return null;
}
}
@Override
public boolean hasPayload() {
return payloadPending && payloadLength > 0;
}
}
}

View File

@ -67,33 +67,46 @@ import org.apache.lucene.store.IOContext;
* <li><a name="tvf" id="tvf"></a>
* <p>The Field or .tvf file.</p>
* <p>This file contains, for each field that has a term vector stored, a list of
* the terms, their frequencies and, optionally, position and offset
* the terms, their frequencies and, optionally, position, offset, and payload
* information.</p>
* <p>Field (.tvf) --&gt; Header,&lt;NumTerms, Position/Offset, TermFreqs&gt;
* <p>Field (.tvf) --&gt; Header,&lt;NumTerms, Flags, TermFreqs&gt;
* <sup>NumFields</sup></p>
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>NumTerms --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>Position/Offset --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>TermFreqs --&gt; &lt;TermText, TermFreq, Positions?, Offsets?&gt;
* <li>Flags --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>TermFreqs --&gt; &lt;TermText, TermFreq, Positions?, PayloadData?, Offsets?&gt;
* <sup>NumTerms</sup></li>
* <li>TermText --&gt; &lt;PrefixLength, Suffix&gt;</li>
* <li>PrefixLength --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>Suffix --&gt; {@link DataOutput#writeString String}</li>
* <li>TermFreq --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>Positions --&gt; &lt;{@link DataOutput#writeVInt VInt}&gt;<sup>TermFreq</sup></li>
* <li>Positions --&gt; &lt;PositionDelta PayloadLength?&gt;<sup>TermFreq</sup></li>
* <li>PositionDelta --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>PayloadLength --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>PayloadData --&gt; {@link DataOutput#writeByte Byte}<sup>NumPayloadBytes</sup></li>
* <li>Offsets --&gt; &lt;{@link DataOutput#writeVInt VInt}, {@link DataOutput#writeVInt VInt}&gt;<sup>TermFreq</sup></li>
* </ul>
* <p>Notes:</p>
* <ul>
* <li>Position/Offset byte stores whether this term vector has position or offset
* <li>Flags byte stores whether this term vector has position, offset, payload.
* information stored.</li>
* <li>Term byte prefixes are shared. The PrefixLength is the number of initial
* bytes from the previous term which must be pre-pended to a term's suffix
* in order to form the term's bytes. Thus, if the previous term's text was "bone"
* and the term is "boy", the PrefixLength is two and the suffix is "y".</li>
* <li>Positions are stored as delta encoded VInts. This means we only store the
* difference of the current position from the last position</li>
* <li>PositionDelta is, if payloads are disabled for the term's field, the
* difference between the position of the current occurrence in the document and
* the previous occurrence (or zero, if this is the first occurrence in this
* document). If payloads are enabled for the term's field, then PositionDelta/2
* is the difference between the current and the previous position. If payloads
* are enabled and PositionDelta is odd, then PayloadLength is stored, indicating
* the length of the payload at the current term position.</li>
* <li>PayloadData is metadata associated with a term position. If
* PayloadLength is stored at the current position, then it indicates the length
* of this payload. If PayloadLength is not stored, then this payload has the same
* length as the payload at the previous position. PayloadData encodes the
* concatenated bytes for all of a terms occurrences.</li>
* <li>Offsets are stored as delta encoded VInts. The first VInt is the
* startOffset, the second is the endOffset.</li>
* </ul>

View File

@ -21,7 +21,9 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
@ -30,7 +32,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.Terms;
@ -55,6 +56,8 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
static final byte STORE_PAYLOAD_WITH_TERMVECTOR = 0x4;
/** Extension of vectors fields file */
static final String VECTORS_FIELDS_EXTENSION = "tvf";
@ -68,8 +71,10 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
static final String CODEC_NAME_DOCS = "Lucene40TermVectorsDocs";
static final String CODEC_NAME_INDEX = "Lucene40TermVectorsIndex";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final int VERSION_NO_PAYLOADS = 0;
static final int VERSION_PAYLOADS = 1;
static final int VERSION_START = VERSION_NO_PAYLOADS;
static final int VERSION_CURRENT = VERSION_PAYLOADS;
static final long HEADER_LENGTH_FIELDS = CodecUtil.headerLength(CODEC_NAME_FIELDS);
static final long HEADER_LENGTH_DOCS = CodecUtil.headerLength(CODEC_NAME_DOCS);
@ -245,9 +250,8 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
}
@Override
public FieldsEnum iterator() throws IOException {
return new FieldsEnum() {
public Iterator<String> iterator() {
return new Iterator<String>() {
private int fieldUpto;
@Override
@ -255,13 +259,18 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
if (fieldNumbers != null && fieldUpto < fieldNumbers.length) {
return fieldInfos.fieldInfo(fieldNumbers[fieldUpto++]).name;
} else {
return null;
throw new NoSuchElementException();
}
}
@Override
public Terms terms() throws IOException {
return TVFields.this.terms(fieldInfos.fieldInfo(fieldNumbers[fieldUpto-1]).name);
public boolean hasNext() {
return fieldNumbers != null && fieldUpto < fieldNumbers.length;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
};
}
@ -296,10 +305,17 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
private class TVTerms extends Terms {
private final int numTerms;
private final long tvfFPStart;
private final boolean storePositions;
private final boolean storeOffsets;
private final boolean storePayloads;
public TVTerms(long tvfFP) throws IOException {
tvf.seek(tvfFP);
numTerms = tvf.readVInt();
final byte bits = tvf.readByte();
storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
storePayloads = (bits & STORE_PAYLOAD_WITH_TERMVECTOR) != 0;
tvfFPStart = tvf.getFilePointer();
}
@ -314,7 +330,7 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
} else {
termsEnum = new TVTermsEnum();
}
termsEnum.reset(numTerms, tvfFPStart);
termsEnum.reset(numTerms, tvfFPStart, storePositions, storeOffsets, storePayloads);
return termsEnum;
}
@ -345,6 +361,21 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
// this...? I guess codec could buffer and re-sort...
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public boolean hasOffsets() {
return storeOffsets;
}
@Override
public boolean hasPositions() {
return storePositions;
}
@Override
public boolean hasPayloads() {
return storePayloads;
}
}
private class TVTermsEnum extends TermsEnum {
@ -357,11 +388,17 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
private BytesRef term = new BytesRef();
private boolean storePositions;
private boolean storeOffsets;
private boolean storePayloads;
private long tvfFP;
private int[] positions;
private int[] startOffsets;
private int[] endOffsets;
// one shared byte[] for any term's payloads
private int[] payloadOffsets;
private int lastPayloadLength;
private byte[] payloadData;
// NOTE: tvf is pre-positioned by caller
public TVTermsEnum() {
@ -373,17 +410,20 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
return tvf == origTVF;
}
public void reset(int numTerms, long tvfFPStart) throws IOException {
public void reset(int numTerms, long tvfFPStart, boolean storePositions, boolean storeOffsets, boolean storePayloads) throws IOException {
this.numTerms = numTerms;
this.storePositions = storePositions;
this.storeOffsets = storeOffsets;
this.storePayloads = storePayloads;
nextTerm = 0;
tvf.seek(tvfFPStart);
final byte bits = tvf.readByte();
storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
tvfFP = 1+tvfFPStart;
positions = null;
startOffsets = null;
endOffsets = null;
payloadOffsets = null;
payloadData = null;
lastPayloadLength = -1;
}
// NOTE: slow! (linear scan)
@ -430,7 +470,26 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
tvf.readBytes(term.bytes, start, deltaLen);
freq = tvf.readVInt();
if (storePositions) {
if (storePayloads) {
positions = new int[freq];
payloadOffsets = new int[freq];
int totalPayloadLength = 0;
int pos = 0;
for(int posUpto=0;posUpto<freq;posUpto++) {
int code = tvf.readVInt();
pos += code >>> 1;
positions[posUpto] = pos;
if ((code & 1) != 0) {
// length change
lastPayloadLength = tvf.readVInt();
}
payloadOffsets[posUpto] = totalPayloadLength;
totalPayloadLength += lastPayloadLength;
assert totalPayloadLength >= 0;
}
payloadData = new byte[totalPayloadLength];
tvf.readBytes(payloadData, 0, payloadData.length);
} else if (storePositions /* no payloads */) {
// TODO: we could maybe reuse last array, if we can
// somehow be careful about consumer never using two
// D&PEnums at once...
@ -502,14 +561,12 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
} else {
docsAndPositionsEnum = new TVDocsAndPositionsEnum();
}
docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets);
docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets, payloadOffsets, payloadData);
return docsAndPositionsEnum;
}
@Override
public Comparator<BytesRef> getComparator() {
// TODO: really indexer hardwires
// this...? I guess codec could buffer and re-sort...
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
}
@ -567,6 +624,9 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
private int[] positions;
private int[] startOffsets;
private int[] endOffsets;
private int[] payloadOffsets;
private BytesRef payload = new BytesRef();
private byte[] payloadBytes;
@Override
public int freq() throws IOException {
@ -602,11 +662,13 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
}
}
public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets) {
public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, int[] payloadLengths, byte[] payloadBytes) {
this.liveDocs = liveDocs;
this.positions = positions;
this.startOffsets = startOffsets;
this.endOffsets = endOffsets;
this.payloadOffsets = payloadLengths;
this.payloadBytes = payloadBytes;
this.doc = -1;
didNext = false;
nextPos = 0;
@ -614,12 +676,19 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
@Override
public BytesRef getPayload() {
return null;
}
@Override
public boolean hasPayload() {
return false;
if (payloadOffsets == null) {
return null;
} else {
int off = payloadOffsets[nextPos-1];
int end = nextPos == payloadOffsets.length ? payloadBytes.length : payloadOffsets[nextPos];
if (end - off == 0) {
return null;
}
payload.bytes = payloadBytes;
payload.offset = off;
payload.length = end - off;
return payload;
}
}
@Override

View File

@ -106,12 +106,14 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
private String lastFieldName;
@Override
public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException {
public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException {
assert lastFieldName == null || info.name.compareTo(lastFieldName) > 0: "fieldName=" + info.name + " lastFieldName=" + lastFieldName;
lastFieldName = info.name;
this.positions = positions;
this.offsets = offsets;
this.payloads = payloads;
lastTerm.length = 0;
lastPayloadLength = -1; // force first payload to write its length
fps[fieldCount++] = tvf.getFilePointer();
tvd.writeVInt(info.number);
tvf.writeVInt(numTerms);
@ -120,6 +122,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
bits |= Lucene40TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
if (offsets)
bits |= Lucene40TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
if (payloads)
bits |= Lucene40TermVectorsReader.STORE_PAYLOAD_WITH_TERMVECTOR;
tvf.writeByte(bits);
assert fieldCount <= numVectorFields;
@ -138,10 +142,12 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
// we also don't buffer during bulk merges.
private int offsetStartBuffer[] = new int[10];
private int offsetEndBuffer[] = new int[10];
private int offsetIndex = 0;
private int offsetFreq = 0;
private BytesRef payloadData = new BytesRef(10);
private int bufferedIndex = 0;
private int bufferedFreq = 0;
private boolean positions = false;
private boolean offsets = false;
private boolean payloads = false;
@Override
public void startTerm(BytesRef term, int freq) throws IOException {
@ -158,20 +164,40 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
// we might need to buffer if its a non-bulk merge
offsetStartBuffer = ArrayUtil.grow(offsetStartBuffer, freq);
offsetEndBuffer = ArrayUtil.grow(offsetEndBuffer, freq);
offsetIndex = 0;
offsetFreq = freq;
}
bufferedIndex = 0;
bufferedFreq = freq;
payloadData.length = 0;
}
int lastPosition = 0;
int lastOffset = 0;
int lastPayloadLength = -1; // force first payload to write its length
BytesRef scratch = new BytesRef(); // used only by this optimized flush below
@Override
public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
// TODO: technically we could just copy bytes and not re-encode if we knew the length...
if (positions != null) {
if (payloads) {
// TODO, maybe overkill and just call super.addProx() in this case?
// we do avoid buffering the offsets in RAM though.
for (int i = 0; i < numProx; i++) {
tvf.writeVInt(positions.readVInt());
int code = positions.readVInt();
if ((code & 1) == 1) {
int length = positions.readVInt();
scratch.grow(length);
scratch.length = length;
positions.readBytes(scratch.bytes, scratch.offset, scratch.length);
writePosition(code >>> 1, scratch);
} else {
writePosition(code >>> 1, null);
}
}
tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
} else if (positions != null) {
// pure positions, no payloads
for (int i = 0; i < numProx; i++) {
tvf.writeVInt(positions.readVInt() >>> 1);
}
}
@ -184,28 +210,36 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
}
@Override
public void addPosition(int position, int startOffset, int endOffset) throws IOException {
if (positions && offsets) {
public void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException {
if (positions && (offsets || payloads)) {
// write position delta
tvf.writeVInt(position - lastPosition);
writePosition(position - lastPosition, payload);
lastPosition = position;
// buffer offsets
offsetStartBuffer[offsetIndex] = startOffset;
offsetEndBuffer[offsetIndex] = endOffset;
offsetIndex++;
if (offsets) {
offsetStartBuffer[bufferedIndex] = startOffset;
offsetEndBuffer[bufferedIndex] = endOffset;
}
bufferedIndex++;
// dump buffer if we are done
if (offsetIndex == offsetFreq) {
for (int i = 0; i < offsetIndex; i++) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
if (bufferedIndex == bufferedFreq) {
if (payloads) {
tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
}
for (int i = 0; i < bufferedIndex; i++) {
if (offsets) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
}
}
}
} else if (positions) {
// write position delta
tvf.writeVInt(position - lastPosition);
writePosition(position - lastPosition, payload);
lastPosition = position;
} else if (offsets) {
// write offset deltas
@ -214,6 +248,30 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
lastOffset = endOffset;
}
}
private void writePosition(int delta, BytesRef payload) throws IOException {
if (payloads) {
int payloadLength = payload == null ? 0 : payload.length;
if (payloadLength != lastPayloadLength) {
lastPayloadLength = payloadLength;
tvf.writeVInt((delta<<1)|1);
tvf.writeVInt(payloadLength);
} else {
tvf.writeVInt(delta << 1);
}
if (payloadLength > 0) {
if (payloadLength + payloadData.length < 0) {
// we overflowed the payload buffer, just throw UOE
// having > Integer.MAX_VALUE bytes of payload for a single term in a single doc is nuts.
throw new UnsupportedOperationException("A term cannot have more than Integer.MAX_VALUE bytes of payload data in a single document");
}
payloadData.append(payload);
}
} else {
tvf.writeVInt(delta);
}
}
@Override
public void abort() {
@ -255,7 +313,14 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int idx = 0;
int numDocs = 0;
for (final AtomicReader reader : mergeState.readers) {
for (int i = 0; i < mergeState.readers.size(); i++) {
final AtomicReader reader = mergeState.readers.get(i);
// set PayloadProcessor
if (mergeState.payloadProcessorProvider != null) {
mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
} else {
mergeState.currentReaderPayloadProcessor = null;
}
final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
Lucene40TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
@ -288,8 +353,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
final int maxDoc = reader.maxDoc();
final Bits liveDocs = reader.getLiveDocs();
int totalNumDocs = 0;
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
for (int docNum = 0; docNum < maxDoc;) {
if (!liveDocs.get(docNum)) {
// skip deleted docs
@ -324,7 +389,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState.fieldInfos);
addAllDocVectors(vectors, mergeState);
totalNumDocs++;
mergeState.checkAbort.work(300);
}
@ -339,8 +404,8 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
int rawDocLengths2[])
throws IOException {
final int maxDoc = reader.maxDoc();
if (matchingVectorsReader != null) {
// We can bulk-copy because the fieldInfos are "congruent"
if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
// We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
int docCount = 0;
while (docCount < maxDoc) {
int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
@ -354,7 +419,7 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Fields vectors = reader.getTermVectors(docNum);
addAllDocVectors(vectors, mergeState.fieldInfos);
addAllDocVectors(vectors, mergeState);
mergeState.checkAbort.work(300);
}
}

View File

@ -366,7 +366,7 @@ the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
factors need no longer be a single byte, they can be any DocValues
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
strings, they can be any byte sequence. Term offsets can optionally be indexed
into the postings lists.</li>
into the postings lists. Payloads can be stored in the term vectors.</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>

View File

@ -32,7 +32,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -44,6 +43,7 @@ import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnmodifiableIterator;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RunAutomaton;
import org.apache.lucene.util.automaton.Transition;
@ -124,36 +124,14 @@ public class DirectPostingsFormat extends PostingsFormat {
private final Map<String,DirectField> fields = new TreeMap<String,DirectField>();
public DirectFields(SegmentReadState state, Fields fields, int minSkipCount, int lowFreqCutoff) throws IOException {
FieldsEnum fieldsEnum = fields.iterator();
String field;
while ((field = fieldsEnum.next()) != null) {
this.fields.put(field, new DirectField(state, field, fieldsEnum.terms(), minSkipCount, lowFreqCutoff));
for (String field : fields) {
this.fields.put(field, new DirectField(state, field, fields.terms(field), minSkipCount, lowFreqCutoff));
}
}
@Override
public FieldsEnum iterator() {
final Iterator<Map.Entry<String,DirectField>> iter = fields.entrySet().iterator();
return new FieldsEnum() {
Map.Entry<String,DirectField> current;
@Override
public String next() {
if (iter.hasNext()) {
current = iter.next();
return current.getKey();
} else {
return null;
}
}
@Override
public Terms terms() {
return current.getValue();
}
};
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override
@ -348,9 +326,8 @@ public class DirectPostingsFormat extends PostingsFormat {
scratch.add(docsAndPositionsEnum.endOffset());
}
if (hasPayloads) {
final BytesRef payload;
if (docsAndPositionsEnum.hasPayload()) {
payload = docsAndPositionsEnum.getPayload();
final BytesRef payload = docsAndPositionsEnum.getPayload();
if (payload != null) {
scratch.add(payload.length);
ros.writeBytes(payload.bytes, payload.offset, payload.length);
} else {
@ -421,9 +398,8 @@ public class DirectPostingsFormat extends PostingsFormat {
for(int pos=0;pos<freq;pos++) {
positions[upto][posUpto] = docsAndPositionsEnum.nextPosition();
if (hasPayloads) {
if (docsAndPositionsEnum.hasPayload()) {
BytesRef payload = docsAndPositionsEnum.getPayload();
assert payload != null;
BytesRef payload = docsAndPositionsEnum.getPayload();
if (payload != null) {
byte[] payloadBytes = new byte[payload.length];
System.arraycopy(payload.bytes, payload.offset, payloadBytes, 0, payload.length);
payloads[upto][pos] = payloadBytes;
@ -635,6 +611,21 @@ public class DirectPostingsFormat extends PostingsFormat {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public boolean hasOffsets() {
return hasOffsets;
}
@Override
public boolean hasPositions() {
return hasPos;
}
@Override
public boolean hasPayloads() {
return hasPayloads;
}
private final class DirectTermsEnum extends TermsEnum {
private final BytesRef scratch = new BytesRef();
@ -1791,18 +1782,12 @@ public class DirectPostingsFormat extends PostingsFormat {
return docID;
}
@Override
public boolean hasPayload() {
return payloadLength > 0;
}
@Override
public BytesRef getPayload() {
if (payloadLength > 0) {
payload.bytes = payloadBytes;
payload.offset = lastPayloadOffset;
payload.length = payloadLength;
payloadLength = 0;
return payload;
} else {
return null;
@ -1995,7 +1980,6 @@ public class DirectPostingsFormat extends PostingsFormat {
private int upto;
private int docID = -1;
private int posUpto;
private boolean gotPayload;
private int[] curPositions;
public HighFreqDocsAndPositionsEnum(Bits liveDocs, boolean hasOffsets) {
@ -2065,7 +2049,6 @@ public class DirectPostingsFormat extends PostingsFormat {
@Override
public int nextPosition() {
posUpto += posJump;
gotPayload = false;
return curPositions[posUpto];
}
@ -2199,21 +2182,22 @@ public class DirectPostingsFormat extends PostingsFormat {
}
}
@Override
public boolean hasPayload() {
return !gotPayload && payloads != null && payloads[upto][posUpto/(hasOffsets ? 3 : 1)] != null;
}
private final BytesRef payload = new BytesRef();
@Override
public BytesRef getPayload() {
final byte[] payloadBytes = payloads[upto][posUpto/(hasOffsets ? 3:1)];
payload.bytes = payloadBytes;
payload.length = payloadBytes.length;
payload.offset = 0;
gotPayload = true;
return payload;
if (payloads == null) {
return null;
} else {
final byte[] payloadBytes = payloads[upto][posUpto/(hasOffsets ? 3:1)];
if (payloadBytes == null) {
return null;
}
payload.bytes = payloadBytes;
payload.length = payloadBytes.length;
payload.offset = 0;
return payload;
}
}
}
}

View File

@ -34,7 +34,6 @@ import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -49,6 +48,7 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.UnmodifiableIterator;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
@ -446,7 +446,6 @@ public class MemoryPostingsFormat extends PostingsFormat {
private int numDocs;
private int posPending;
private int payloadLength;
private boolean payloadRetrieved;
final boolean storeOffsets;
int offsetLength;
int startOffset;
@ -484,7 +483,6 @@ public class MemoryPostingsFormat extends PostingsFormat {
payloadLength = 0;
this.numDocs = numDocs;
posPending = 0;
payloadRetrieved = false;
startOffset = storeOffsets ? 0 : -1; // always return -1 if no offsets are stored
offsetLength = 0;
return this;
@ -577,10 +575,6 @@ public class MemoryPostingsFormat extends PostingsFormat {
payload.offset = in.getPosition();
in.skipBytes(payloadLength);
payload.length = payloadLength;
// Necessary, in case caller changed the
// payload.bytes from prior call:
payload.bytes = buffer;
payloadRetrieved = false;
}
//System.out.println(" pos=" + pos + " payload=" + payload + " fp=" + in.getPosition());
@ -599,13 +593,7 @@ public class MemoryPostingsFormat extends PostingsFormat {
@Override
public BytesRef getPayload() {
payloadRetrieved = true;
return payload;
}
@Override
public boolean hasPayload() {
return !payloadRetrieved && payload.length > 0;
return payload.length > 0 ? payload : null;
}
@Override
@ -834,6 +822,21 @@ public class MemoryPostingsFormat extends PostingsFormat {
public Comparator<BytesRef> getComparator() {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
public boolean hasOffsets() {
return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return field.hasPayloads();
}
}
@Override
@ -859,24 +862,8 @@ public class MemoryPostingsFormat extends PostingsFormat {
return new FieldsProducer() {
@Override
public FieldsEnum iterator() {
final Iterator<TermsReader> iter = fields.values().iterator();
return new FieldsEnum() {
private TermsReader current;
@Override
public String next() {
current = iter.next();
return current.field.name;
}
@Override
public Terms terms() {
return current;
}
};
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override

View File

@ -30,11 +30,11 @@ import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.UnmodifiableIterator;
/**
* Enables per field format support.
@ -197,34 +197,9 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
}
}
private final class FieldsIterator extends FieldsEnum {
private final Iterator<String> it;
private String current;
public FieldsIterator() {
it = fields.keySet().iterator();
}
@Override
public String next() {
if (it.hasNext()) {
current = it.next();
} else {
current = null;
}
return current;
}
@Override
public Terms terms() throws IOException {
return fields.get(current).terms(current);
}
}
@Override
public FieldsEnum iterator() throws IOException {
return new FieldsIterator();
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override

View File

@ -532,19 +532,13 @@ public class PulsingPostingsReader extends PostingsReaderBase {
}
}
@Override
public boolean hasPayload() {
return storePayloads && !payloadRetrieved && payloadLength > 0;
}
@Override
public BytesRef getPayload() throws IOException {
//System.out.println("PR getPayload payloadLength=" + payloadLength + " this=" + this);
if (payloadRetrieved) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
}
payloadRetrieved = true;
if (payloadLength > 0) {
return payload;
} else if (storePayloads && payloadLength > 0) {
payloadRetrieved = true;
if (payload == null) {
payload = new BytesRef(payloadLength);
} else {

View File

@ -714,7 +714,11 @@ public class SepPostingsReader extends PostingsReaderBase {
@Override
public BytesRef getPayload() throws IOException {
if (!payloadPending) {
throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
return null;
}
if (pendingPayloadBytes == 0) {
return payload;
}
assert pendingPayloadBytes >= payloadLength;
@ -731,15 +735,9 @@ public class SepPostingsReader extends PostingsReaderBase {
}
payloadIn.readBytes(payload.bytes, 0, payloadLength);
payloadPending = false;
payload.length = payloadLength;
pendingPayloadBytes = 0;
return payload;
}
@Override
public boolean hasPayload() {
return payloadPending && payloadLength > 0;
}
}
}

View File

@ -20,14 +20,17 @@ package org.apache.lucene.codecs.simpletext;
import java.io.IOException;
import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
@ -40,6 +43,7 @@ import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.OpenBitSet;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.UnmodifiableIterator;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
@ -48,7 +52,7 @@ import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
class SimpleTextFieldsReader extends FieldsProducer {
private final TreeMap<String,Long> fields;
private final IndexInput in;
private final FieldInfos fieldInfos;
@ -66,35 +70,22 @@ class SimpleTextFieldsReader extends FieldsProducer {
in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
fieldInfos = state.fieldInfos;
fields = readFields((IndexInput)in.clone());
}
private class SimpleTextFieldsEnum extends FieldsEnum {
private final IndexInput in;
private final BytesRef scratch = new BytesRef(10);
private String current;
public SimpleTextFieldsEnum() {
this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
}
@Override
public String next() throws IOException {
while(true) {
SimpleTextUtil.readLine(in, scratch);
if (scratch.equals(END)) {
current = null;
return null;
}
if (StringHelper.startsWith(scratch, FIELD)) {
return current = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
}
private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
BytesRef scratch = new BytesRef(10);
TreeMap<String,Long> fields = new TreeMap<String,Long>();
while (true) {
SimpleTextUtil.readLine(in, scratch);
if (scratch.equals(END)) {
return fields;
} else if (StringHelper.startsWith(scratch, FIELD)) {
String fieldName = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
fields.put(fieldName, in.getFilePointer());
}
}
@Override
public Terms terms() throws IOException {
return SimpleTextFieldsReader.this.terms(current);
}
}
private class SimpleTextTermsEnum extends TermsEnum {
@ -471,18 +462,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
@Override
public BytesRef getPayload() {
// Some tests rely on only being able to retrieve the
// payload once
try {
return payload;
} finally {
payload = null;
}
}
@Override
public boolean hasPayload() {
return payload != null;
return payload;
}
}
@ -498,7 +478,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
private class SimpleTextTerms extends Terms {
private final long termsStart;
private final IndexOptions indexOptions;
private final FieldInfo fieldInfo;
private long sumTotalTermFreq;
private long sumDocFreq;
private int docCount;
@ -509,7 +489,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
public SimpleTextTerms(String field, long termsStart) throws IOException {
this.termsStart = termsStart;
indexOptions = fieldInfos.fieldInfo(field).getIndexOptions();
fieldInfo = fieldInfos.fieldInfo(field);
loadTerms();
}
@ -579,7 +559,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
@Override
public TermsEnum iterator(TermsEnum reuse) throws IOException {
if (fst != null) {
return new SimpleTextTermsEnum(fst, indexOptions);
return new SimpleTextTermsEnum(fst, fieldInfo.getIndexOptions());
} else {
return TermsEnum.EMPTY;
}
@ -597,7 +577,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
@Override
public long getSumTotalTermFreq() {
return indexOptions == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq;
return fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq;
}
@Override
@ -609,11 +589,26 @@ class SimpleTextFieldsReader extends FieldsProducer {
public int getDocCount() throws IOException {
return docCount;
}
@Override
public boolean hasOffsets() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
}
@Override
public boolean hasPositions() {
return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
}
@Override
public boolean hasPayloads() {
return fieldInfo.hasPayloads();
}
}
@Override
public FieldsEnum iterator() throws IOException {
return new SimpleTextFieldsEnum();
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
private final Map<String,Terms> termsCache = new HashMap<String,Terms>();
@ -622,15 +617,13 @@ class SimpleTextFieldsReader extends FieldsProducer {
synchronized public Terms terms(String field) throws IOException {
Terms terms = termsCache.get(field);
if (terms == null) {
SimpleTextFieldsEnum fe = (SimpleTextFieldsEnum) iterator();
String fieldUpto;
while((fieldUpto = fe.next()) != null) {
if (fieldUpto.equals(field)) {
terms = new SimpleTextTerms(field, fe.in.getFilePointer());
break;
}
Long fp = fields.get(field);
if (fp == null) {
return null;
} else {
terms = new SimpleTextTerms(field, fp);
termsCache.put(field, terms);
}
termsCache.put(field, terms);
}
return terms;
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.Terms;
@ -45,6 +44,7 @@ import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.UnmodifiableIterator;
import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
@ -126,11 +126,15 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
assert StringHelper.startsWith(scratch, FIELDOFFSETS);
boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
readLine();
assert StringHelper.startsWith(scratch, FIELDPAYLOADS);
boolean payloads = Boolean.parseBoolean(readString(FIELDPAYLOADS.length, scratch));
readLine();
assert StringHelper.startsWith(scratch, FIELDTERMCOUNT);
int termCount = parseIntAt(FIELDTERMCOUNT.length);
SimpleTVTerms terms = new SimpleTVTerms();
SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
fields.put(fieldName, terms);
for (int j = 0; j < termCount; j++) {
@ -152,6 +156,9 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
if (positions || offsets) {
if (positions) {
postings.positions = new int[postings.freq];
if (payloads) {
postings.payloads = new BytesRef[postings.freq];
}
}
if (offsets) {
@ -164,6 +171,17 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
readLine();
assert StringHelper.startsWith(scratch, POSITION);
postings.positions[k] = parseIntAt(POSITION.length);
if (payloads) {
readLine();
assert StringHelper.startsWith(scratch, PAYLOAD);
if (scratch.length - PAYLOAD.length == 0) {
postings.payloads[k] = null;
} else {
byte payloadBytes[] = new byte[scratch.length - PAYLOAD.length];
System.arraycopy(scratch.bytes, scratch.offset+PAYLOAD.length, payloadBytes, 0, payloadBytes.length);
postings.payloads[k] = new BytesRef(payloadBytes);
}
}
}
if (offsets) {
@ -222,26 +240,8 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
@Override
public FieldsEnum iterator() throws IOException {
return new FieldsEnum() {
private Iterator<Map.Entry<String,SimpleTVTerms>> iterator = fields.entrySet().iterator();
private Map.Entry<String,SimpleTVTerms> current = null;
@Override
public String next() {
if (!iterator.hasNext()) {
return null;
} else {
current = iterator.next();
return current.getKey();
}
}
@Override
public Terms terms() {
return current.getValue();
}
};
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override
@ -257,8 +257,14 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
private static class SimpleTVTerms extends Terms {
final SortedMap<BytesRef,SimpleTVPostings> terms;
final boolean hasOffsets;
final boolean hasPositions;
final boolean hasPayloads;
SimpleTVTerms() {
SimpleTVTerms(boolean hasOffsets, boolean hasPositions, boolean hasPayloads) {
this.hasOffsets = hasOffsets;
this.hasPositions = hasPositions;
this.hasPayloads = hasPayloads;
terms = new TreeMap<BytesRef,SimpleTVPostings>();
}
@ -292,6 +298,21 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
public int getDocCount() throws IOException {
return 1;
}
@Override
public boolean hasOffsets() {
return hasOffsets;
}
@Override
public boolean hasPositions() {
return hasPositions;
}
@Override
public boolean hasPayloads() {
return hasPayloads;
}
}
private static class SimpleTVPostings {
@ -299,6 +320,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
private int positions[];
private int startOffsets[];
private int endOffsets[];
private BytesRef payloads[];
}
private static class SimpleTVTermsEnum extends TermsEnum {
@ -372,7 +394,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
// TODO: reuse
SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets);
e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
return e;
}
@ -433,6 +455,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
private int nextPos;
private Bits liveDocs;
private int[] positions;
private BytesRef[] payloads;
private int[] startOffsets;
private int[] endOffsets;
@ -470,11 +493,12 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
}
}
public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets) {
public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, BytesRef payloads[]) {
this.liveDocs = liveDocs;
this.positions = positions;
this.startOffsets = startOffsets;
this.endOffsets = endOffsets;
this.payloads = payloads;
this.doc = -1;
didNext = false;
nextPos = 0;
@ -482,12 +506,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
@Override
public BytesRef getPayload() {
return null;
}
@Override
public boolean hasPayload() {
return false;
return payloads == null ? null : payloads[nextPos-1];
}
@Override

View File

@ -45,10 +45,12 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
static final BytesRef FIELDNAME = new BytesRef(" name ");
static final BytesRef FIELDPOSITIONS = new BytesRef(" positions ");
static final BytesRef FIELDOFFSETS = new BytesRef(" offsets ");
static final BytesRef FIELDPAYLOADS = new BytesRef(" payloads ");
static final BytesRef FIELDTERMCOUNT = new BytesRef(" numterms ");
static final BytesRef TERMTEXT = new BytesRef(" term ");
static final BytesRef TERMFREQ = new BytesRef(" freq ");
static final BytesRef POSITION = new BytesRef(" position ");
static final BytesRef PAYLOAD = new BytesRef(" payload ");
static final BytesRef STARTOFFSET = new BytesRef(" startoffset ");
static final BytesRef ENDOFFSET = new BytesRef(" endoffset ");
@ -61,6 +63,7 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
private final BytesRef scratch = new BytesRef();
private boolean offsets;
private boolean positions;
private boolean payloads;
public SimpleTextTermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
this.directory = directory;
@ -89,7 +92,7 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
}
@Override
public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException {
public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException {
write(FIELD);
write(Integer.toString(info.number));
newLine();
@ -106,12 +109,17 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
write(Boolean.toString(offsets));
newLine();
write(FIELDPAYLOADS);
write(Boolean.toString(payloads));
newLine();
write(FIELDTERMCOUNT);
write(Integer.toString(numTerms));
newLine();
this.positions = positions;
this.offsets = offsets;
this.payloads = payloads;
}
@Override
@ -126,13 +134,22 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
}
@Override
public void addPosition(int position, int startOffset, int endOffset) throws IOException {
public void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException {
assert positions || offsets;
if (positions) {
write(POSITION);
write(Integer.toString(position));
newLine();
if (payloads) {
write(PAYLOAD);
if (payload != null) {
assert payload.length > 0;
write(payload);
}
newLine();
}
}
if (offsets) {

View File

@ -39,6 +39,7 @@ public class FieldType implements IndexableFieldType {
private boolean storeTermVectors;
private boolean storeTermVectorOffsets;
private boolean storeTermVectorPositions;
private boolean storeTermVectorPayloads;
private boolean omitNorms;
private IndexOptions indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
private NumericType numericType;
@ -53,6 +54,7 @@ public class FieldType implements IndexableFieldType {
this.storeTermVectors = ref.storeTermVectors();
this.storeTermVectorOffsets = ref.storeTermVectorOffsets();
this.storeTermVectorPositions = ref.storeTermVectorPositions();
this.storeTermVectorPayloads = ref.storeTermVectorPayloads();
this.omitNorms = ref.omitNorms();
this.indexOptions = ref.indexOptions();
this.docValueType = ref.docValueType();
@ -132,6 +134,15 @@ public class FieldType implements IndexableFieldType {
this.storeTermVectorPositions = value;
}
public boolean storeTermVectorPayloads() {
return this.storeTermVectorPayloads;
}
public void setStoreTermVectorPayloads(boolean value) {
checkIfFrozen();
this.storeTermVectorPayloads = value;
}
public boolean omitNorms() {
return this.omitNorms;
}
@ -188,24 +199,19 @@ public class FieldType implements IndexableFieldType {
result.append(",");
result.append("indexed");
if (tokenized()) {
if (result.length() > 0)
result.append(",");
result.append("tokenized");
result.append(",tokenized");
}
if (storeTermVectors()) {
if (result.length() > 0)
result.append(",");
result.append("termVector");
result.append(",termVector");
}
if (storeTermVectorOffsets()) {
if (result.length() > 0)
result.append(",");
result.append("termVectorOffsets");
result.append(",termVectorOffsets");
}
if (storeTermVectorPositions()) {
if (result.length() > 0)
result.append(",");
result.append("termVectorPosition");
result.append(",termVectorPosition");
if (storeTermVectorPayloads()) {
result.append(",termVectorPayloads");
}
}
if (omitNorms()) {
result.append(",omitNorms");
@ -222,7 +228,9 @@ public class FieldType implements IndexableFieldType {
}
}
if (docValueType != null) {
result.append(",docValueType=");
if (result.length() > 0)
result.append(",");
result.append("docValueType=");
result.append(docValueType);
}

View File

@ -685,12 +685,7 @@ public class CheckIndex {
DocsAndPositionsEnum postings = null;
String lastField = null;
final FieldsEnum fieldsEnum = fields.iterator();
while(true) {
final String field = fieldsEnum.next();
if (field == null) {
break;
}
for (String field : fields) {
// MultiFieldsEnum relies upon this order...
if (lastField != null && field.compareTo(lastField) <= 0) {
throw new RuntimeException("fields out of order: lastField=" + lastField + " field=" + field);
@ -713,11 +708,16 @@ public class CheckIndex {
// assert fields.terms(field) != null;
computedFieldCount++;
final Terms terms = fieldsEnum.terms();
final Terms terms = fields.terms(field);
if (terms == null) {
continue;
}
final boolean hasPositions = terms.hasPositions();
final boolean hasOffsets = terms.hasOffsets();
// term vectors cannot omit TF
final boolean hasFreqs = isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
final TermsEnum termsEnum = terms.iterator(null);
boolean hasOrd = true;
@ -777,17 +777,10 @@ public class CheckIndex {
status.termCount++;
final DocsEnum docs2;
final boolean hasPositions;
// if we are checking vectors, we have freqs implicitly
final boolean hasFreqs = isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
// if we are checking vectors, offsets are a free-for-all anyway
final boolean hasOffsets = isVectors || fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
if (postings != null) {
docs2 = postings;
hasPositions = true;
} else {
docs2 = docs;
hasPositions = false;
}
int lastDoc = -1;
@ -824,22 +817,17 @@ public class CheckIndex {
if (hasPositions) {
for(int j=0;j<freq;j++) {
final int pos = postings.nextPosition();
// NOTE: pos=-1 is allowed because of ancient bug
// (LUCENE-1542) whereby IndexWriter could
// write pos=-1 when first token's posInc is 0
// (separately: analyzers should not give
// posInc=0 to first token); also, term
// vectors are allowed to return pos=-1 if
// they indexed offset but not positions:
if (pos < -1) {
if (pos < 0) {
throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " is out of bounds");
}
if (pos < lastPos) {
throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " < lastPos " + lastPos);
}
lastPos = pos;
if (postings.hasPayload()) {
postings.getPayload();
BytesRef payload = postings.getPayload();
if (payload != null && payload.length < 1) {
throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " payload length is out of bounds " + payload.length);
}
if (hasOffsets) {
int startOffset = postings.startOffset();
@ -924,14 +912,8 @@ public class CheckIndex {
int lastOffset = 0;
for(int posUpto=0;posUpto<freq;posUpto++) {
final int pos = postings.nextPosition();
// NOTE: pos=-1 is allowed because of ancient bug
// (LUCENE-1542) whereby IndexWriter could
// write pos=-1 when first token's posInc is 0
// (separately: analyzers should not give
// posInc=0 to first token); also, term
// vectors are allowed to return pos=-1 if
// they indexed offset but not positions:
if (pos < -1) {
if (pos < 0) {
throw new RuntimeException("position " + pos + " is out of bounds");
}
if (pos < lastPosition) {
@ -1000,11 +982,7 @@ public class CheckIndex {
// only happen if it's a ghost field (field with
// no terms, eg there used to be terms but all
// docs got deleted and then merged away):
// make sure TermsEnum is empty:
final Terms fieldTerms2 = fieldsEnum.terms();
if (fieldTerms2 != null && fieldTerms2.iterator(null).next() != null) {
throw new RuntimeException("Fields.terms(field=" + field + ") returned null yet the field appears to have terms");
}
} else {
if (fieldTerms instanceof BlockTreeTermsReader.FieldReader) {
final BlockTreeTermsReader.Stats stats = ((BlockTreeTermsReader.FieldReader) fieldTerms).computeStats();
@ -1415,9 +1393,7 @@ public class CheckIndex {
status.docCount++;
}
FieldsEnum fieldsEnum = tfv.iterator();
String field = null;
while((field = fieldsEnum.next()) != null) {
for(String field : tfv) {
if (doStats) {
status.totVectors++;
}
@ -1432,6 +1408,8 @@ public class CheckIndex {
Terms terms = tfv.terms(field);
termsEnum = terms.iterator(termsEnum);
final boolean postingsHasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
final boolean postingsHasPayload = fieldInfo.hasPayloads();
final boolean vectorsHasPayload = terms.hasPayloads();
Terms postingsTerms = postingsFields.terms(field);
if (postingsTerms == null) {
@ -1439,19 +1417,18 @@ public class CheckIndex {
}
postingsTermsEnum = postingsTerms.iterator(postingsTermsEnum);
final boolean hasProx = terms.hasOffsets() || terms.hasPositions();
BytesRef term = null;
while ((term = termsEnum.next()) != null) {
final boolean hasProx;
// Try positions:
postings = termsEnum.docsAndPositions(null, postings);
if (postings == null) {
hasProx = false;
// Try docIDs & freqs:
docs = termsEnum.docs(null, docs);
if (hasProx) {
postings = termsEnum.docsAndPositions(null, postings);
assert postings != null;
docs = null;
} else {
hasProx = true;
docs = termsEnum.docs(null, docs);
assert docs != null;
postings = null;
}
final DocsEnum docs2;
@ -1504,7 +1481,7 @@ public class CheckIndex {
int pos = postings.nextPosition();
if (postingsPostings != null) {
int postingsPos = postingsPostings.nextPosition();
if (pos != -1 && postingsPos != -1 && pos != postingsPos) {
if (terms.hasPositions() && pos != postingsPos) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": pos=" + pos + " differs from postings pos=" + postingsPos);
}
}
@ -1535,6 +1512,34 @@ public class CheckIndex {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + ": endOffset=" + endOffset + " differs from postings endOffset=" + postingsEndOffset);
}
}
BytesRef payload = postings.getPayload();
if (payload != null) {
assert vectorsHasPayload;
}
if (postingsHasPayload && vectorsHasPayload) {
assert postingsPostings != null;
if (payload == null) {
// we have payloads, but not at this position.
// postings has payloads too, it should not have one at this position
if (postingsPostings.getPayload() != null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has no payload but postings does: " + postingsPostings.getPayload());
}
} else {
// we have payloads, and one at this position
// postings should also have one at this position, with the same bytes.
if (postingsPostings.getPayload() == null) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but postings does not.");
}
BytesRef postingsPayload = postingsPostings.getPayload();
if (!payload.equals(postingsPayload)) {
throw new RuntimeException("vector term=" + term + " field=" + field + " doc=" + j + " has payload=" + payload + " but differs from postings payload=" + postingsPayload);
}
}
}
}
}
}

View File

@ -24,7 +24,7 @@ import java.util.List;
import java.util.Map;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.index.BufferedDeletesStream.QueryAndLimit;
class CoalescedDeletes {
@ -48,13 +48,14 @@ class CoalescedDeletes {
public Iterable<Term> termsIterable() {
return new Iterable<Term>() {
@SuppressWarnings("unchecked")
@Override
public Iterator<Term> iterator() {
ArrayList<Iterator<Term>> subs = new ArrayList<Iterator<Term>>(iterables.size());
for (Iterable<Term> iterable : iterables) {
subs.add(iterable.iterator());
Iterator<Term> subs[] = new Iterator[iterables.size()];
for (int i = 0; i < iterables.size(); i++) {
subs[i] = iterables.get(i).iterator();
}
return mergedIterator(subs);
return new MergedIterator<Term>(subs);
}
};
}
@ -86,106 +87,4 @@ class CoalescedDeletes {
}
};
}
/** provides a merged view across multiple iterators */
static Iterator<Term> mergedIterator(final List<Iterator<Term>> iterators) {
return new Iterator<Term>() {
Term current;
TermMergeQueue queue = new TermMergeQueue(iterators.size());
SubIterator[] top = new SubIterator[iterators.size()];
int numTop;
{
int index = 0;
for (Iterator<Term> iterator : iterators) {
if (iterator.hasNext()) {
SubIterator sub = new SubIterator();
sub.current = iterator.next();
sub.iterator = iterator;
sub.index = index++;
queue.add(sub);
}
}
}
public boolean hasNext() {
if (queue.size() > 0) {
return true;
}
for (int i = 0; i < numTop; i++) {
if (top[i].iterator.hasNext()) {
return true;
}
}
return false;
}
public Term next() {
// restore queue
pushTop();
// gather equal top fields
if (queue.size() > 0) {
pullTop();
} else {
current = null;
}
return current;
}
public void remove() {
throw new UnsupportedOperationException();
}
private void pullTop() {
// extract all subs from the queue that have the same top term
assert numTop == 0;
while (true) {
top[numTop++] = queue.pop();
if (queue.size() == 0
|| !(queue.top()).current.equals(top[0].current)) {
break;
}
}
current = top[0].current;
}
private void pushTop() {
// call next() on each top, and put back into queue
for (int i = 0; i < numTop; i++) {
if (top[i].iterator.hasNext()) {
top[i].current = top[i].iterator.next();
queue.add(top[i]);
} else {
// no more terms
top[i].current = null;
}
}
numTop = 0;
}
};
}
private static class SubIterator {
Iterator<Term> iterator;
Term current;
int index;
}
private static class TermMergeQueue extends PriorityQueue<SubIterator> {
TermMergeQueue(int size) {
super(size);
}
@Override
protected boolean lessThan(SubIterator a, SubIterator b) {
final int cmp = a.current.compareTo(b.current);
if (cmp != 0) {
return cmp < 0;
} else {
return a.index < b.index;
}
}
}
}

View File

@ -105,7 +105,7 @@ public abstract class DocValues implements Closeable {
* <p>
* {@link Source} instances obtained from this method are closed / released
* from the cache once this {@link DocValues} instance is closed by the
* {@link IndexReader}, {@link Fields} or {@link FieldsEnum} the
* {@link IndexReader}, {@link Fields} or the
* {@link DocValues} was created from.
*/
public Source getSource() throws IOException {

View File

@ -48,11 +48,8 @@ public abstract class DocsAndPositionsEnum extends DocsEnum {
public abstract int endOffset() throws IOException;
/** Returns the payload at this position, or null if no
* payload was indexed. Only call this once per
* position. You should not modify anything (neither
* members of the returned BytesRef nor bytes in the
* byte[]). */
* payload was indexed. You should not modify anything
* (neither members of the returned BytesRef nor bytes
* in the byte[]). */
public abstract BytesRef getPayload() throws IOException;
public abstract boolean hasPayload();
}

View File

@ -18,15 +18,16 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Iterator;
/** Flex API for access to fields and terms
* @lucene.experimental */
public abstract class Fields {
public abstract class Fields implements Iterable<String> {
/** Returns an iterator that will step through all fields
* names. This will not return null. */
public abstract FieldsEnum iterator() throws IOException;
public abstract Iterator<String> iterator();
/** Get the {@link Terms} for this field. This will return
* null if the field does not exist. */
@ -45,12 +46,7 @@ public abstract class Fields {
// TODO: deprecate?
public long getUniqueTermCount() throws IOException {
long numTerms = 0;
FieldsEnum it = iterator();
while(true) {
String field = it.next();
if (field == null) {
break;
}
for (String field : this) {
Terms terms = terms(field);
if (terms != null) {
final long termCount = terms.size();

View File

@ -1,79 +0,0 @@
package org.apache.lucene.index;
/*
* 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.
*/
import java.io.IOException;
import org.apache.lucene.util.AttributeSource;
/** Enumerates indexed fields. You must first call {@link
* #next} before calling {@link #terms}.
*
* @lucene.experimental */
public abstract class FieldsEnum {
// TODO: maybe allow retrieving FieldInfo for current
// field, as optional method?
private AttributeSource atts = null;
/**
* Returns the related attributes.
*/
public AttributeSource attributes() {
if (atts == null) {
atts = new AttributeSource();
}
return atts;
}
/** Increments the enumeration to the next field. Returns
* null when there are no more fields.*/
public abstract String next() throws IOException;
// TODO: would be nice to require/fix all impls so they
// never return null here... we have to fix the writers to
// never write 0-terms fields... or maybe allow a non-null
// Terms instance in just this case
/** Get {@link Terms} for the current field. After {@link #next} returns
* null this method should not be called. This method may
* return null in some cases, which means the provided
* field does not have any terms. */
public abstract Terms terms() throws IOException;
// TODO: should we allow pulling Terms as well? not just
// the iterator?
public final static FieldsEnum[] EMPTY_ARRAY = new FieldsEnum[0];
/** Provides zero fields */
public final static FieldsEnum EMPTY = new FieldsEnum() {
@Override
public String next() {
return null;
}
@Override
public Terms terms() {
throw new IllegalStateException("this method should never be called");
}
};
}

View File

@ -24,6 +24,7 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
import java.io.IOException;
import java.util.Comparator;
import java.util.Iterator;
/** A <code>FilterAtomicReader</code> contains another AtomicReader, which it
* uses as its basic source of data, possibly transforming the data along the
@ -46,7 +47,7 @@ public class FilterAtomicReader extends AtomicReader {
}
@Override
public FieldsEnum iterator() throws IOException {
public Iterator<String> iterator() {
return in.iterator();
}
@ -109,28 +110,20 @@ public class FilterAtomicReader extends AtomicReader {
public TermsEnum intersect(CompiledAutomaton automaton, BytesRef bytes) throws java.io.IOException {
return in.intersect(automaton, bytes);
}
}
/** Base class for filtering {@link TermsEnum} implementations. */
public static class FilterFieldsEnum extends FieldsEnum {
protected final FieldsEnum in;
public FilterFieldsEnum(FieldsEnum in) {
this.in = in;
@Override
public boolean hasOffsets() {
return in.hasOffsets();
}
@Override
public String next() throws IOException {
return in.next();
}
@Override
public Terms terms() throws IOException {
return in.terms();
public boolean hasPositions() {
return in.hasPositions();
}
@Override
public AttributeSource attributes() {
return in.attributes();
public boolean hasPayloads() {
return in.hasPayloads();
}
}
@ -292,11 +285,6 @@ public class FilterAtomicReader extends AtomicReader {
public BytesRef getPayload() throws IOException {
return in.getPayload();
}
@Override
public boolean hasPayload() {
return in.hasPayload();
}
@Override
public AttributeSource attributes() {

View File

@ -42,6 +42,9 @@ public interface IndexableFieldType {
/** True if term vector positions should be indexed */
public boolean storeTermVectorPositions();
/** True if term vector payloads should be indexed */
public boolean storeTermVectorPayloads();
/** True if norms should not be indexed */
public boolean omitNorms();

View File

@ -199,6 +199,7 @@ public class MergeState {
// and we could make a codec(wrapper) to do all of this privately so IW is uninvolved
public PayloadProcessorProvider payloadProcessorProvider;
public ReaderPayloadProcessor[] readerPayloadProcessor;
public ReaderPayloadProcessor currentReaderPayloadProcessor;
public PayloadProcessor[] currentPayloadProcessor;
// TODO: get rid of this? it tells you which segments are 'aligned' (e.g. for bulk merging)

View File

@ -137,11 +137,6 @@ public final class MultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
return current.endOffset();
}
@Override
public boolean hasPayload() {
return current.hasPayload();
}
@Override
public BytesRef getPayload() throws IOException {
return current.getPayload();

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Collection;
@ -27,6 +28,7 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.MergedIterator;
/**
* Exposes flex API, merged from flex API of sub-segments.
@ -125,7 +127,7 @@ public final class MultiFields extends Fields {
* term. This will return null if the field or term does
* not exist. */
public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
return getTermDocsEnum(r, liveDocs, field, term);
return getTermDocsEnum(r, liveDocs, field, term, DocsEnum.FLAG_FREQS);
}
/** Returns {@link DocsEnum} for the specified field &
@ -180,22 +182,14 @@ public final class MultiFields extends Fields {
this.subSlices = subSlices;
}
@SuppressWarnings("unchecked")
@Override
public FieldsEnum iterator() throws IOException {
final List<FieldsEnum> fieldsEnums = new ArrayList<FieldsEnum>();
final List<ReaderSlice> fieldsSlices = new ArrayList<ReaderSlice>();
public Iterator<String> iterator() {
Iterator<String> subIterators[] = new Iterator[subs.length];
for(int i=0;i<subs.length;i++) {
fieldsEnums.add(subs[i].iterator());
fieldsSlices.add(subSlices[i]);
}
if (fieldsEnums.size() == 0) {
return FieldsEnum.EMPTY;
} else {
return new MultiFieldsEnum(this,
fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
fieldsSlices.toArray(ReaderSlice.EMPTY_ARRAY));
subIterators[i] = subs[i].iterator();
}
return new MergedIterator<String>(subIterators);
}
@Override

View File

@ -1,133 +0,0 @@
package org.apache.lucene.index;
/*
* 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.
*/
import org.apache.lucene.util.PriorityQueue;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
/**
* Exposes flex API, merged from flex API of sub-segments.
* This does a merge sort, by field name, of the
* sub-readers.
*
* @lucene.experimental
*/
public final class MultiFieldsEnum extends FieldsEnum {
private final FieldMergeQueue queue;
// Holds sub-readers containing field we are currently
// on, popped from queue.
private final FieldsEnumWithSlice[] top;
private int numTop;
private final Fields fields;
private String currentField;
/** The subs array must be newly initialized FieldsEnum
* (ie, {@link FieldsEnum#next} has not been called. */
public MultiFieldsEnum(MultiFields fields, FieldsEnum[] subs, ReaderSlice[] subSlices) throws IOException {
this.fields = fields;
queue = new FieldMergeQueue(subs.length);
top = new FieldsEnumWithSlice[subs.length];
List<FieldsEnumWithSlice> enumWithSlices = new ArrayList<FieldsEnumWithSlice>();
// Init q
for(int i=0;i<subs.length;i++) {
assert subs[i] != null;
final String field = subs[i].next();
if (field != null) {
// this FieldsEnum has at least one field
final FieldsEnumWithSlice sub = new FieldsEnumWithSlice(subs[i], subSlices[i], i);
enumWithSlices.add(sub);
sub.current = field;
queue.add(sub);
}
}
}
@Override
public String next() throws IOException {
// restore queue
for(int i=0;i<numTop;i++) {
top[i].current = top[i].fields.next();
if (top[i].current != null) {
queue.add(top[i]);
} else {
// no more fields in this sub-reader
}
}
numTop = 0;
// gather equal top fields
if (queue.size() > 0) {
while(true) {
top[numTop++] = queue.pop();
if (queue.size() == 0 || !(queue.top()).current.equals(top[0].current)) {
break;
}
}
currentField = top[0].current;
} else {
currentField = null;
}
return currentField;
}
@Override
public Terms terms() throws IOException {
// Ask our parent MultiFields:
return fields.terms(currentField);
}
public final static class FieldsEnumWithSlice {
public static final FieldsEnumWithSlice[] EMPTY_ARRAY = new FieldsEnumWithSlice[0];
final FieldsEnum fields;
final ReaderSlice slice;
final int index;
String current;
public FieldsEnumWithSlice(FieldsEnum fields, ReaderSlice slice, int index) {
this.slice = slice;
this.index = index;
assert slice.length >= 0: "length=" + slice.length;
this.fields = fields;
}
}
private final static class FieldMergeQueue extends PriorityQueue<FieldsEnumWithSlice> {
FieldMergeQueue(int size) {
super(size);
}
@Override
protected final boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
// No need to break ties by field name: TermsEnum handles that
return fieldsA.current.compareTo(fieldsB.current) < 0;
}
}
}

View File

@ -37,12 +37,19 @@ public final class MultiTerms extends Terms {
private final Terms[] subs;
private final ReaderSlice[] subSlices;
private final Comparator<BytesRef> termComp;
private final boolean hasOffsets;
private final boolean hasPositions;
private final boolean hasPayloads;
public MultiTerms(Terms[] subs, ReaderSlice[] subSlices) throws IOException {
this.subs = subs;
this.subSlices = subSlices;
Comparator<BytesRef> _termComp = null;
assert subs.length > 0 : "inefficient: don't use MultiTerms over one sub";
boolean _hasOffsets = true;
boolean _hasPositions = true;
boolean _hasPayloads = false;
for(int i=0;i<subs.length;i++) {
if (_termComp == null) {
_termComp = subs[i].getComparator();
@ -54,9 +61,15 @@ public final class MultiTerms extends Terms {
throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
}
}
_hasOffsets &= subs[i].hasOffsets();
_hasPositions &= subs[i].hasPositions();
_hasPayloads |= subs[i].hasPayloads();
}
termComp = _termComp;
hasOffsets = _hasOffsets;
hasPositions = _hasPositions;
hasPayloads = hasPositions && _hasPayloads; // if all subs have pos, and at least one has payloads.
}
@Override
@ -142,5 +155,20 @@ public final class MultiTerms extends Terms {
public Comparator<BytesRef> getComparator() {
return termComp;
}
@Override
public boolean hasOffsets() {
return hasOffsets;
}
@Override
public boolean hasPositions() {
return hasPositions;
}
@Override
public boolean hasPayloads() {
return hasPayloads;
}
}

View File

@ -27,6 +27,7 @@ import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.UnmodifiableIterator;
/** An {@link AtomicReader} which reads multiple, parallel indexes. Each index
@ -121,12 +122,10 @@ public final class ParallelAtomicReader extends AtomicReader {
for (final AtomicReader reader : this.parallelReaders) {
final Fields readerFields = reader.fields();
if (readerFields != null) {
final FieldsEnum it = readerFields.iterator();
String name;
while ((name = it.next()) != null) {
for (String field : readerFields) {
// only add if the reader responsible for that field name is the current:
if (fieldToReader.get(name) == reader) {
this.fields.addField(name, it.terms());
if (fieldToReader.get(field) == reader) {
this.fields.addField(field, readerFields.terms(field));
}
}
}
@ -151,33 +150,6 @@ public final class ParallelAtomicReader extends AtomicReader {
return buffer.append(')').toString();
}
private final class ParallelFieldsEnum extends FieldsEnum {
private String currentField;
private final Iterator<String> keys;
private final ParallelFields fields;
ParallelFieldsEnum(ParallelFields fields) {
this.fields = fields;
keys = fields.fields.keySet().iterator();
}
@Override
public String next() {
if (keys.hasNext()) {
currentField = keys.next();
} else {
currentField = null;
}
return currentField;
}
@Override
public Terms terms() {
return fields.terms(currentField);
}
}
// Single instance of this, per ParallelReader instance
private final class ParallelFields extends Fields {
final Map<String,Terms> fields = new TreeMap<String,Terms>();
@ -190,8 +162,8 @@ public final class ParallelAtomicReader extends AtomicReader {
}
@Override
public FieldsEnum iterator() {
return new ParallelFieldsEnum(this);
public Iterator<String> iterator() {
return new UnmodifiableIterator<String>(fields.keySet().iterator());
}
@Override

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
@ -36,9 +37,12 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
boolean doVectors;
boolean doVectorPositions;
boolean doVectorOffsets;
boolean doVectorPayloads;
int maxNumPostings;
OffsetAttribute offsetAttribute;
PayloadAttribute payloadAttribute;
boolean hasPayloads; // if enabled, and we actually saw any for this field
public TermVectorsConsumerPerField(TermsHashPerField termsHashPerField, TermVectorsConsumer termsWriter, FieldInfo fieldInfo) {
this.termsHashPerField = termsHashPerField;
@ -58,13 +62,46 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
doVectors = false;
doVectorPositions = false;
doVectorOffsets = false;
doVectorPayloads = false;
hasPayloads = false;
for(int i=0;i<count;i++) {
IndexableField field = fields[i];
if (field.fieldType().storeTermVectors()) {
doVectors = true;
doVectorPositions |= field.fieldType().storeTermVectorPositions();
doVectorOffsets |= field.fieldType().storeTermVectorOffsets();
if (field.fieldType().indexed()) {
if (field.fieldType().storeTermVectors()) {
doVectors = true;
doVectorPositions |= field.fieldType().storeTermVectorPositions();
doVectorOffsets |= field.fieldType().storeTermVectorOffsets();
if (doVectorPositions) {
doVectorPayloads |= field.fieldType().storeTermVectorPayloads();
} else if (field.fieldType().storeTermVectorPayloads()) {
// TODO: move this check somewhere else, and impl the other missing ones
throw new IllegalArgumentException("cannot index term vector payloads for field: " + field + " without term vector positions");
}
} else {
if (field.fieldType().storeTermVectorOffsets()) {
throw new IllegalArgumentException("cannot index term vector offsets when term vectors are not indexed (field=\"" + field.name());
}
if (field.fieldType().storeTermVectorPositions()) {
throw new IllegalArgumentException("cannot index term vector positions when term vectors are not indexed (field=\"" + field.name());
}
if (field.fieldType().storeTermVectorPayloads()) {
throw new IllegalArgumentException("cannot index term vector payloads when term vectors are not indexed (field=\"" + field.name());
}
}
} else {
if (field.fieldType().storeTermVectors()) {
throw new IllegalArgumentException("cannot index term vectors when field is not indexed (field=\"" + field.name());
}
if (field.fieldType().storeTermVectorOffsets()) {
throw new IllegalArgumentException("cannot index term vector offsets when field is not indexed (field=\"" + field.name());
}
if (field.fieldType().storeTermVectorPositions()) {
throw new IllegalArgumentException("cannot index term vector positions when field is not indexed (field=\"" + field.name());
}
if (field.fieldType().storeTermVectorPayloads()) {
throw new IllegalArgumentException("cannot index term vector payloads when field is not indexed (field=\"" + field.name());
}
}
}
@ -121,7 +158,7 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
final int[] termIDs = termsHashPerField.sortPostings(tv.getComparator());
tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets);
tv.startField(fieldInfo, numPostings, doVectorPositions, doVectorOffsets, hasPayloads);
final ByteSliceReader posReader = doVectorPositions ? termsWriter.vectorSliceReaderPos : null;
final ByteSliceReader offReader = doVectorOffsets ? termsWriter.vectorSliceReaderOff : null;
@ -165,39 +202,14 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
} else {
offsetAttribute = null;
}
}
@Override
void newTerm(final int termID) {
assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
postings.freqs[termID] = 1;
if (doVectorOffsets) {
int startOffset = fieldState.offset + offsetAttribute.startOffset();
int endOffset = fieldState.offset + offsetAttribute.endOffset();
termsHashPerField.writeVInt(1, startOffset);
termsHashPerField.writeVInt(1, endOffset - startOffset);
postings.lastOffsets[termID] = endOffset;
}
if (doVectorPositions) {
termsHashPerField.writeVInt(0, fieldState.position);
postings.lastPositions[termID] = fieldState.position;
if (doVectorPayloads && fieldState.attributeSource.hasAttribute(PayloadAttribute.class)) {
payloadAttribute = fieldState.attributeSource.getAttribute(PayloadAttribute.class);
} else {
payloadAttribute = null;
}
}
@Override
void addTerm(final int termID) {
assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
postings.freqs[termID]++;
void writeProx(TermVectorsPostingsArray postings, int termID) {
if (doVectorOffsets) {
int startOffset = fieldState.offset + offsetAttribute.startOffset();
int endOffset = fieldState.offset + offsetAttribute.endOffset();
@ -208,11 +220,48 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
}
if (doVectorPositions) {
termsHashPerField.writeVInt(0, fieldState.position - postings.lastPositions[termID]);
final BytesRef payload;
if (payloadAttribute == null) {
payload = null;
} else {
payload = payloadAttribute.getPayload();
}
final int pos = fieldState.position - postings.lastPositions[termID];
if (payload != null && payload.length > 0) {
termsHashPerField.writeVInt(0, (pos<<1)|1);
termsHashPerField.writeVInt(0, payload.length);
termsHashPerField.writeBytes(0, payload.bytes, payload.offset, payload.length);
hasPayloads = true;
} else {
termsHashPerField.writeVInt(0, pos<<1);
}
postings.lastPositions[termID] = fieldState.position;
}
}
@Override
void newTerm(final int termID) {
assert docState.testPoint("TermVectorsTermsWriterPerField.newTerm start");
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
postings.freqs[termID] = 1;
postings.lastOffsets[termID] = 0;
postings.lastPositions[termID] = 0;
writeProx(postings, termID);
}
@Override
void addTerm(final int termID) {
assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
postings.freqs[termID]++;
writeProx(postings, termID);
}
@Override
void skippingLongTerm() {}

View File

@ -104,6 +104,15 @@ public abstract class Terms {
* measures, this measure does not take deleted documents
* into account. */
public abstract int getDocCount() throws IOException;
/** Returns true if documents in this field store offsets. */
public abstract boolean hasOffsets();
/** Returns true if documents in this field store positions. */
public abstract boolean hasPositions();
/** Returns true if documents in this field store payloads. */
public abstract boolean hasPayloads();
public final static Terms[] EMPTY_ARRAY = new Terms[0];
}

View File

@ -213,7 +213,11 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
public float coord(int overlap, int maxOverlap) {
return similarity.coord(overlap, maxOverlap);
// LUCENE-4300: in most cases of maxOverlap=1, BQ rewrites itself away,
// so coord() is not applied. But when BQ cannot optimize itself away
// for a single clause (minNrShouldMatch, prohibited clauses, etc), its
// important not to apply coord(1,1) for consistency, it might not be 1.0F
return maxOverlap == 1 ? 1F : similarity.coord(overlap, maxOverlap);
}
@Override

View File

@ -41,16 +41,16 @@ class BooleanScorer2 extends Scorer {
private final List<Scorer> prohibitedScorers;
private class Coordinator {
float[] coordFactors = null;
int maxCoord = 0; // to be increased for each non prohibited scorer
int nrMatchers; // to be increased by score() of match counting scorers.
void init(boolean disableCoord) { // use after all scorers have been added.
final float coordFactors[];
Coordinator(int maxCoord, boolean disableCoord) {
coordFactors = new float[optionalScorers.size() + requiredScorers.size() + 1];
for (int i = 0; i < coordFactors.length; i++) {
coordFactors[i] = disableCoord ? 1.0f : ((BooleanWeight)weight).coord(i, maxCoord);
}
}
int nrMatchers; // to be increased by score() of match counting scorers.
}
private final Coordinator coordinator;
@ -92,15 +92,13 @@ class BooleanScorer2 extends Scorer {
if (minNrShouldMatch < 0) {
throw new IllegalArgumentException("Minimum number of optional scorers should not be negative");
}
coordinator = new Coordinator();
this.minNrShouldMatch = minNrShouldMatch;
coordinator.maxCoord = maxCoord;
optionalScorers = optional;
requiredScorers = required;
prohibitedScorers = prohibited;
coordinator = new Coordinator(maxCoord, disableCoord);
coordinator.init(disableCoord);
countingSumScorer = makeCountingSumScorer(disableCoord);
}

View File

@ -548,12 +548,7 @@ class UnionDocsAndPositionsEnum extends DocsAndPositionsEnum {
@Override
public BytesRef getPayload() {
throw new UnsupportedOperationException();
}
@Override
public boolean hasPayload() {
throw new UnsupportedOperationException();
return null;
}
@Override

View File

@ -118,8 +118,8 @@ public class PayloadTermQuery extends SpanTermQuery {
}
protected void processPayload(Similarity similarity) throws IOException {
final DocsAndPositionsEnum postings = termSpans.getPostings();
if (postings.hasPayload()) {
if (termSpans.isPayloadAvailable()) {
final DocsAndPositionsEnum postings = termSpans.getPostings();
payload = postings.getPayload();
if (payload != null) {
payloadScore = function.currentScore(doc, term.field(),

View File

@ -126,7 +126,7 @@ public class NearSpansUnordered extends Spans {
// TODO: Remove warning after API has been finalized
@Override
public boolean isPayloadAvailable() {
public boolean isPayloadAvailable() throws IOException {
return spans.isPayloadAvailable();
}
@ -256,7 +256,7 @@ public class NearSpansUnordered extends Spans {
// TODO: Remove warning after API has been finalized
@Override
public boolean isPayloadAvailable() {
public boolean isPayloadAvailable() throws IOException {
SpansCell pointer = min();
while (pointer != null) {
if (pointer.isPayloadAvailable()) {

View File

@ -157,7 +157,7 @@ public class SpanNotQuery extends SpanQuery implements Cloneable {
// TODO: Remove warning after API has been finalized
@Override
public boolean isPayloadAvailable() {
public boolean isPayloadAvailable() throws IOException {
return includeSpans.isPayloadAvailable();
}

View File

@ -247,7 +247,7 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
}
@Override
public boolean isPayloadAvailable() {
public boolean isPayloadAvailable() throws IOException {
Spans top = top();
return top != null && top.isPayloadAvailable();
}

View File

@ -176,7 +176,7 @@ public abstract class SpanPositionCheckQuery extends SpanQuery implements Clonea
// TODO: Remove warning after API has been finalized
@Override
public boolean isPayloadAvailable() {
public boolean isPayloadAvailable() throws IOException {
return spans.isPayloadAvailable();
}

View File

@ -82,6 +82,6 @@ public abstract class Spans {
*
* @return true if there is a payload available at this position that can be loaded
*/
public abstract boolean isPayloadAvailable();
public abstract boolean isPayloadAvailable() throws IOException;
}

View File

@ -36,6 +36,7 @@ public class TermSpans extends Spans {
protected int freq;
protected int count;
protected int position;
protected boolean readPayload;
public TermSpans(DocsAndPositionsEnum postings, Term term) {
this.postings = postings;
@ -64,6 +65,7 @@ public class TermSpans extends Spans {
}
position = postings.nextPosition();
count++;
readPayload = false;
return true;
}
@ -78,7 +80,7 @@ public class TermSpans extends Spans {
count = 0;
position = postings.nextPosition();
count++;
readPayload = false;
return true;
}
@ -101,6 +103,7 @@ public class TermSpans extends Spans {
@Override
public Collection<byte[]> getPayload() throws IOException {
final BytesRef payload = postings.getPayload();
readPayload = true;
final byte[] bytes;
if (payload != null) {
bytes = new byte[payload.length];
@ -113,8 +116,8 @@ public class TermSpans extends Spans {
// TODO: Remove warning after API has been finalized
@Override
public boolean isPayloadAvailable() {
return postings.hasPayload();
public boolean isPayloadAvailable() throws IOException {
return readPayload == false && postings.getPayload() != null;
}
@Override

View File

@ -0,0 +1,148 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.util.Iterator;
import java.util.NoSuchElementException;
/**
* Provides a merged sorted view from several sorted iterators, each
* iterating over a unique set of elements.
* <p>
* If an element appears in multiple iterators, it is deduplicated,
* that is this iterator returns the sorted union of elements.
* <p>
* Caveats:
* <ul>
* <li>The behavior is undefined if the iterators are not actually
* sorted according to their comparator, or if a single iterator
* contains duplicates.
* <li>Null elements are unsupported.
* <li>When an element E is a duplicate across multiple iterators,
* only one is returned, but it is undefined which one: not
* guaranteed to be a stable sort.
* </ul>
* @lucene.internal
*/
public class MergedIterator<T extends Comparable<T>> implements Iterator<T> {
private T current;
private final TermMergeQueue<T> queue;
private final SubIterator<T>[] top;
private int numTop;
@SuppressWarnings("unchecked")
public MergedIterator(Iterator<T>... iterators) {
queue = new TermMergeQueue<T>(iterators.length);
top = new SubIterator[iterators.length];
int index = 0;
for (Iterator<T> iterator : iterators) {
if (iterator.hasNext()) {
SubIterator<T> sub = new SubIterator<T>();
sub.current = iterator.next();
sub.iterator = iterator;
sub.index = index++;
queue.add(sub);
}
}
}
@Override
public boolean hasNext() {
if (queue.size() > 0) {
return true;
}
for (int i = 0; i < numTop; i++) {
if (top[i].iterator.hasNext()) {
return true;
}
}
return false;
}
@Override
public T next() {
// restore queue
pushTop();
// gather equal top elements
if (queue.size() > 0) {
pullTop();
} else {
current = null;
}
if (current == null) {
throw new NoSuchElementException();
}
return current;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
private void pullTop() {
// extract all subs from the queue that have the same top element
assert numTop == 0;
while (true) {
top[numTop++] = queue.pop();
if (queue.size() == 0
|| !(queue.top()).current.equals(top[0].current)) {
break;
}
}
current = top[0].current;
}
private void pushTop() {
// call next() on each top, and put back into queue
for (int i = 0; i < numTop; i++) {
if (top[i].iterator.hasNext()) {
top[i].current = top[i].iterator.next();
queue.add(top[i]);
} else {
// no more elements
top[i].current = null;
}
}
numTop = 0;
}
private static class SubIterator<I extends Comparable<I>> {
Iterator<I> iterator;
I current;
int index;
}
private static class TermMergeQueue<C extends Comparable<C>> extends PriorityQueue<SubIterator<C>> {
TermMergeQueue(int size) {
super(size);
}
@Override
protected boolean lessThan(SubIterator<C> a, SubIterator<C> b) {
final int cmp = a.current.compareTo(b.current);
if (cmp != 0) {
return cmp < 0;
} else {
return a.index < b.index;
}
}
}
}

View File

@ -0,0 +1,46 @@
package org.apache.lucene.util;
/*
* 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.
*/
import java.util.Iterator;
/**
* Wraps an iterator to ensure its unmodifiable
*/
public class UnmodifiableIterator<T> implements Iterator<T> {
private final Iterator<T> in;
public UnmodifiableIterator(Iterator<T> in) {
this.in = in;
}
@Override
public boolean hasNext() {
return in.hasNext();
}
@Override
public T next() {
return in.next();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}

View File

@ -62,7 +62,21 @@ public abstract class HashFunction implements NamedSPILoader.NamedSPI {
return loader.availableServices();
}
/**
* Reloads the hash function list from the given {@link ClassLoader}.
* Changes to the function list are visible after the method ends, all
* iterators ({@link #availableHashFunctionNames()},...) stay consistent.
*
* <p><b>NOTE:</b> Only new functions are added, existing ones are
* never removed or replaced.
*
* <p><em>This method is expensive and should only be called for discovery
* of new functions on the given classpath/classloader!</em>
*/
public static void reloadHashFunctions(ClassLoader classloader) {
loader.reload(classloader);
}
@Override
public String toString() {
return name;

View File

@ -0,0 +1,98 @@
package org.apache.lucene;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Ignore;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.carrotsearch.randomizedtesting.annotations.Timeout;
/*
* 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.
*/
public class TestWorstCaseTestBehavior extends LuceneTestCase {
@Ignore
public void testThreadLeak() {
Thread t = new Thread() {
@Override
public void run() {
try {
Thread.sleep(10000);
} catch (InterruptedException e) {
// Ignore.
}
}
};
t.start();
while (!t.isAlive()) {
Thread.yield();
}
// once alive, leave it to run outside of the test scope.
}
@Ignore
public void testLaaaaaargeOutput() throws Exception {
String message = "I will not OOM on large output";
int howMuch = 250 * 1024 * 1024;
for (int i = 0; i < howMuch; i++) {
if (i > 0) System.out.print(",\n");
System.out.print(message);
howMuch -= message.length(); // approximately.
}
System.out.println(".");
}
@Ignore
public void testProgressiveOutput() throws Exception {
for (int i = 0; i < 20; i++) {
System.out.println("Emitting sysout line: " + i);
System.err.println("Emitting syserr line: " + i);
System.out.flush();
System.err.flush();
RandomizedTest.sleep(1000);
}
}
@Ignore
public void testUncaughtException() throws Exception {
Thread t = new Thread() {
@Override
public void run() {
throw new RuntimeException("foobar");
}
};
t.start();
t.join();
}
@Ignore
@Timeout(millis = 500)
public void testTimeout() throws Exception {
Thread.sleep(5000);
}
@Ignore
@Timeout(millis = 1000)
public void testZombie() throws Exception {
while (true) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {}
}
}
}

View File

@ -28,14 +28,18 @@ import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.BaseDirectoryWrapper;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TimeUnits;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
/**
* Test indexes ~82M docs with 26 terms each, so you get > Integer.MAX_VALUE terms/docs pairs
* @lucene.experimental
*/
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@TimeoutSuite(millis = 4 * TimeUnits.HOUR)
public class Test2BPostings extends LuceneTestCase {
@Nightly

View File

@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
@ -258,9 +259,10 @@ public class TestCodecs extends LuceneTestCase {
final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR));
final FieldsEnum fieldsEnum = reader.iterator();
assertNotNull(fieldsEnum.next());
final Terms terms2 = fieldsEnum.terms();
final Iterator<String> fieldsEnum = reader.iterator();
String fieldName = fieldsEnum.next();
assertNotNull(fieldName);
final Terms terms2 = reader.terms(fieldName);
assertNotNull(terms2);
final TermsEnum termsEnum = terms2.iterator(null);
@ -286,7 +288,7 @@ public class TestCodecs extends LuceneTestCase {
assertEquals(termsEnum.seekCeil(new BytesRef(terms[i].text2)), TermsEnum.SeekStatus.FOUND);
}
assertNull(fieldsEnum.next());
assertFalse(fieldsEnum.hasNext());
reader.close();
dir.close();
}
@ -436,14 +438,14 @@ public class TestCodecs extends LuceneTestCase {
final int pos = posEnum.nextPosition();
assertEquals(positions[i].pos, pos);
if (positions[i].payload != null) {
assertTrue(posEnum.hasPayload());
assertNotNull(posEnum.getPayload());
if (random().nextInt(3) < 2) {
// Verify the payload bytes
final BytesRef otherPayload = posEnum.getPayload();
assertTrue("expected=" + positions[i].payload.toString() + " got=" + otherPayload.toString(), positions[i].payload.equals(otherPayload));
}
} else {
assertFalse(posEnum.hasPayload());
assertNull(posEnum.getPayload());
}
}
}

View File

@ -612,20 +612,20 @@ public void testFilesOpenClose() throws IOException {
}
// check dictionary and posting lists
FieldsEnum fenum1 = MultiFields.getFields(index1).iterator();
FieldsEnum fenum2 = MultiFields.getFields(index1).iterator();
String field1 = null;
Fields fields1 = MultiFields.getFields(index1);
Fields fields2 = MultiFields.getFields(index2);
Iterator<String> fenum2 = fields2.iterator();
Bits liveDocs = MultiFields.getLiveDocs(index1);
while((field1=fenum1.next()) != null) {
for (String field1 : fields1) {
assertEquals("Different fields", field1, fenum2.next());
Terms terms1 = fenum1.terms();
Terms terms1 = fields1.terms(field1);
if (terms1 == null) {
assertNull(fenum2.terms());
assertNull(fields2.terms(field1));
continue;
}
TermsEnum enum1 = terms1.iterator(null);
Terms terms2 = fenum2.terms();
Terms terms2 = fields2.terms(field1);
assertNotNull(terms2);
TermsEnum enum2 = terms2.iterator(null);
@ -644,6 +644,7 @@ public void testFilesOpenClose() throws IOException {
}
}
}
assertFalse(fenum2.hasNext());
}
public void testGetIndexCommit() throws IOException {

View File

@ -32,7 +32,6 @@ import java.util.LinkedList;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.search.DocIdSetIterator;
@ -236,10 +235,9 @@ public class TestDoc extends LuceneTestCase {
for (int i = 0; i < reader.numDocs(); i++)
out.println(reader.document(i));
FieldsEnum fis = reader.fields().iterator();
String field = fis.next();
while(field != null) {
Terms terms = fis.terms();
Fields fields = reader.fields();
for (String field : fields) {
Terms terms = fields.terms(field);
assertNotNull(terms);
TermsEnum tis = terms.iterator(null);
while(tis.next() != null) {
@ -259,7 +257,6 @@ public class TestDoc extends LuceneTestCase {
out.println("");
}
}
field = fis.next();
}
reader.close();
}

View File

@ -61,9 +61,7 @@ public class TestDocCount extends LuceneTestCase {
if (fields == null) {
return;
}
FieldsEnum e = fields.iterator();
String field;
while ((field = e.next()) != null) {
for (String field : fields) {
Terms terms = fields.terms(field);
if (terms == null) {
continue;

View File

@ -205,11 +205,11 @@ public class TestDocumentWriter extends LuceneTestCase {
int freq = termPositions.freq();
assertEquals(3, freq);
assertEquals(0, termPositions.nextPosition());
assertEquals(true, termPositions.hasPayload());
assertNotNull(termPositions.getPayload());
assertEquals(6, termPositions.nextPosition());
assertEquals(false, termPositions.hasPayload());
assertNull(termPositions.getPayload());
assertEquals(7, termPositions.nextPosition());
assertEquals(false, termPositions.hasPayload());
assertNull(termPositions.getPayload());
reader.close();
}

View File

@ -27,12 +27,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.ThreadInterruptedException;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeaks;
/**
* Tests for {@link DocumentsWriterStallControl}
*/
@ThreadLeaks(failTestIfLeaking = true)
public class TestDocumentsWriterStallControl extends LuceneTestCase {
public void testSimpleStall() throws InterruptedException {

View File

@ -173,15 +173,15 @@ public class TestDuelingCodecs extends LuceneTestCase {
}
assertFieldStatistics(leftFields, rightFields);
FieldsEnum leftEnum = leftFields.iterator();
FieldsEnum rightEnum = rightFields.iterator();
Iterator<String> leftEnum = leftFields.iterator();
Iterator<String> rightEnum = rightFields.iterator();
String field;
while ((field = leftEnum.next()) != null) {
while (leftEnum.hasNext()) {
String field = leftEnum.next();
assertEquals(info, field, rightEnum.next());
assertTerms(leftEnum.terms(), rightEnum.terms(), deep);
assertTerms(leftFields.terms(field), rightFields.terms(field), deep);
}
assertNull(rightEnum.next());
assertFalse(rightEnum.hasNext());
}
/**
@ -207,6 +207,9 @@ public class TestDuelingCodecs extends LuceneTestCase {
return;
}
assertTermsStatistics(leftTerms, rightTerms);
assertEquals(leftTerms.hasOffsets(), rightTerms.hasOffsets());
assertEquals(leftTerms.hasPositions(), rightTerms.hasPositions());
assertEquals(leftTerms.hasPayloads(), rightTerms.hasPayloads());
TermsEnum leftTermsEnum = leftTerms.iterator(null);
TermsEnum rightTermsEnum = rightTerms.iterator(null);
@ -409,12 +412,9 @@ public class TestDuelingCodecs extends LuceneTestCase {
assertEquals(info, freq, rightDocs.freq());
for (int i = 0; i < freq; i++) {
assertEquals(info, leftDocs.nextPosition(), rightDocs.nextPosition());
assertEquals(info, leftDocs.hasPayload(), rightDocs.hasPayload());
assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
assertEquals(info, leftDocs.startOffset(), rightDocs.startOffset());
assertEquals(info, leftDocs.endOffset(), rightDocs.endOffset());
if (leftDocs.hasPayload()) {
assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
}
}
}
assertEquals(info, DocIdSetIterator.NO_MORE_DOCS, rightDocs.nextDoc());
@ -506,10 +506,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
assertEquals(info, freq, rightDocs.freq());
for (int i = 0; i < freq; i++) {
assertEquals(info, leftDocs.nextPosition(), rightDocs.nextPosition());
assertEquals(info, leftDocs.hasPayload(), rightDocs.hasPayload());
if (leftDocs.hasPayload()) {
assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
}
assertEquals(info, leftDocs.getPayload(), rightDocs.getPayload());
}
}
}
@ -528,9 +525,7 @@ public class TestDuelingCodecs extends LuceneTestCase {
return;
}
FieldsEnum fieldsEnum = leftFields.iterator();
String field;
while ((field = fieldsEnum.next()) != null) {
for (String field : leftFields) {
DocValues leftNorms = MultiDocValues.getNormDocValues(leftReader, field);
DocValues rightNorms = MultiDocValues.getNormDocValues(rightReader, field);
if (leftNorms != null && rightNorms != null) {

View File

@ -40,10 +40,7 @@ public class TestFilterAtomicReader extends LuceneTestCase {
TestFields(Fields in) {
super(in);
}
@Override
public FieldsEnum iterator() throws IOException {
return new TestFieldsEnum(super.iterator());
}
@Override
public Terms terms(String field) throws IOException {
return new TestTerms(super.terms(field));
@ -61,17 +58,6 @@ public class TestFilterAtomicReader extends LuceneTestCase {
}
}
private static class TestFieldsEnum extends FilterFieldsEnum {
TestFieldsEnum(FieldsEnum in) {
super(in);
}
@Override
public Terms terms() throws IOException {
return new TestTerms(super.terms());
}
}
private static class TestTermsEnum extends FilterTermsEnum {
public TestTermsEnum(TermsEnum in) {
super(in);
@ -194,7 +180,6 @@ public class TestFilterAtomicReader extends LuceneTestCase {
checkOverrideMethods(FilterAtomicReader.class);
checkOverrideMethods(FilterAtomicReader.FilterFields.class);
checkOverrideMethods(FilterAtomicReader.FilterTerms.class);
checkOverrideMethods(FilterAtomicReader.FilterFieldsEnum.class);
checkOverrideMethods(FilterAtomicReader.FilterTermsEnum.class);
checkOverrideMethods(FilterAtomicReader.FilterDocsEnum.class);
checkOverrideMethods(FilterAtomicReader.FilterDocsAndPositionsEnum.class);

View File

@ -526,8 +526,6 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
dir.setMaxSizeInBytes(Math.max(1, dir.getRecomputedActualSizeInBytes()));
final Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_STORED);
customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorOffsets(true);
doc.add(newField("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", customType));
try {
writer.addDocument(doc);

View File

@ -60,17 +60,22 @@ public class TestIndexableField extends LuceneTestCase {
@Override
public boolean storeTermVectors() {
return counter % 2 == 1 && counter % 10 != 9;
return indexed() && counter % 2 == 1 && counter % 10 != 9;
}
@Override
public boolean storeTermVectorOffsets() {
return counter % 2 == 1 && counter % 10 != 9;
return storeTermVectors() && counter % 10 != 9;
}
@Override
public boolean storeTermVectorPositions() {
return counter % 2 == 1 && counter % 10 != 9;
return storeTermVectors() && counter % 10 != 9;
}
@Override
public boolean storeTermVectorPayloads() {
return storeTermVectors() && counter % 10 != 9;
}
@Override

View File

@ -205,8 +205,11 @@ public class TestLongPostings extends LuceneTestCase {
assertTrue(freq >=1 && freq <= 4);
for(int pos=0;pos<freq;pos++) {
assertEquals(pos, postings.nextPosition());
if (random().nextBoolean() && postings.hasPayload()) {
if (random().nextBoolean()) {
postings.getPayload();
if (random().nextBoolean()) {
postings.getPayload(); // get it again
}
}
}
}
@ -247,8 +250,11 @@ public class TestLongPostings extends LuceneTestCase {
assertTrue(freq >=1 && freq <= 4);
for(int pos=0;pos<freq;pos++) {
assertEquals(pos, postings.nextPosition());
if (random().nextBoolean() && postings.hasPayload()) {
if (random().nextBoolean()) {
postings.getPayload();
if (random().nextBoolean()) {
postings.getPayload(); // get it again
}
}
}
}

View File

@ -169,4 +169,22 @@ public class TestMultiFields extends LuceneTestCase {
r.close();
dir.close();
}
public void testTermDocsEnum() throws Exception {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random())));
Document d = new Document();
d.add(newStringField("f", "j", Field.Store.NO));
w.addDocument(d);
w.commit();
w.addDocument(d);
IndexReader r = w.getReader();
w.close();
DocsEnum de = MultiFields.getTermDocsEnum(r, null, "f", new BytesRef("j"));
assertEquals(0, de.nextDoc());
assertEquals(1, de.nextDoc());
assertEquals(DocIdSetIterator.NO_MORE_DOCS, de.nextDoc());
r.close();
dir.close();
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Iterator;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
@ -94,20 +95,21 @@ public class TestParallelTermEnum extends LuceneTestCase {
Bits liveDocs = pr.getLiveDocs();
FieldsEnum fe = pr.fields().iterator();
Fields fields = pr.fields();
Iterator<String> fe = fields.iterator();
String f = fe.next();
assertEquals("field1", f);
checkTerms(fe.terms(), liveDocs, "brown", "fox", "jumps", "quick", "the");
checkTerms(fields.terms(f), liveDocs, "brown", "fox", "jumps", "quick", "the");
f = fe.next();
assertEquals("field2", f);
checkTerms(fe.terms(), liveDocs, "brown", "fox", "jumps", "quick", "the");
checkTerms(fields.terms(f), liveDocs, "brown", "fox", "jumps", "quick", "the");
f = fe.next();
assertEquals("field3", f);
checkTerms(fe.terms(), liveDocs, "dog", "fox", "jumps", "lazy", "over", "the");
checkTerms(fields.terms(f), liveDocs, "dog", "fox", "jumps", "lazy", "over", "the");
assertNull(fe.next());
assertFalse(fe.hasNext());
}
}

View File

@ -145,8 +145,18 @@ public class TestPayloadProcessorProvider extends LuceneTestCase {
Document doc = new Document();
doc.add(newField("id", "doc" + i, customType));
doc.add(newTextField("content", "doc content " + i, Field.Store.NO));
doc.add(new TextField("p", payloadTS1));
doc.add(new TextField("p", payloadTS2));
if (random.nextBoolean()) {
doc.add(new TextField("p", payloadTS1));
doc.add(new TextField("p", payloadTS2));
} else {
FieldType type = new FieldType(TextField.TYPE_NOT_STORED);
type.setStoreTermVectors(true);
type.setStoreTermVectorPositions(true);
type.setStoreTermVectorPayloads(true);
type.setStoreTermVectorOffsets(random.nextBoolean());
doc.add(new Field("p", payloadTS1, type));
doc.add(new Field("p", payloadTS2, type));
}
writer.addDocument(doc);
if (multipleCommits && (i % 4 == 0)) {
writer.commit();
@ -163,8 +173,8 @@ public class TestPayloadProcessorProvider extends LuceneTestCase {
DocsAndPositionsEnum tpe = MultiFields.getTermPositionsEnum(reader, null, field, text);
while (tpe.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
tpe.nextPosition();
if (tpe.hasPayload()) {
BytesRef payload = tpe.getPayload();
BytesRef payload = tpe.getPayload();
if (payload != null) {
assertEquals(1, payload.length);
assertEquals(1, payload.bytes[0]);
++numPayloads;

View File

@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
@ -200,18 +201,10 @@ public class TestPayloads extends LuceneTestCase {
for (int i = 0; i < freq; i++) {
for (int j = 0; j < numTerms; j++) {
tps[j].nextPosition();
if (tps[j].hasPayload()) {
BytesRef br = tps[j].getPayload();
BytesRef br = tps[j].getPayload();
if (br != null) {
System.arraycopy(br.bytes, br.offset, verifyPayloadData, offset, br.length);
offset += br.length;
// Just to ensure all codecs can
// handle a caller that mucks with the
// returned payload:
if (rarely()) {
br.bytes = new byte[random().nextInt(5)];
}
br.length = 0;
br.offset = 0;
}
}
}
@ -267,11 +260,6 @@ public class TestPayloads extends LuceneTestCase {
tp.nextPosition();
assertEquals("Wrong payload length.", 3 * skipInterval - 2 * numDocs - 1, tp.getPayload().length);
/*
* Test multiple call of getPayload()
*/
assertFalse(tp.hasPayload());
reader.close();
// test long payload
@ -591,4 +579,73 @@ public class TestPayloads extends LuceneTestCase {
dir.close();
}
/** some docs have payload att, some not */
public void testMixupDocs() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, null);
iwc.setMergePolicy(newLogMergePolicy());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
Field field = new TextField("field", "", Field.Store.NO);
TokenStream ts = new MockTokenizer(new StringReader("here we go"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
doc.add(field);
writer.addDocument(doc);
Token withPayload = new Token("withPayload", 0, 11);
withPayload.setPayload(new BytesRef("test"));
ts = new CannedTokenStream(withPayload);
assertTrue(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
writer.addDocument(doc);
ts = new MockTokenizer(new StringReader("another"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
AtomicReader sr = reader.getSequentialSubReaders().get(0);
DocsAndPositionsEnum de = sr.termPositionsEnum(null, "field", new BytesRef("withPayload"));
de.nextDoc();
de.nextPosition();
assertEquals(new BytesRef("test"), de.getPayload());
writer.close();
reader.close();
dir.close();
}
/** some field instances have payload att, some not */
public void testMixupMultiValued() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
Document doc = new Document();
Field field = new TextField("field", "", Field.Store.NO);
TokenStream ts = new MockTokenizer(new StringReader("here we go"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
doc.add(field);
Field field2 = new TextField("field", "", Field.Store.NO);
Token withPayload = new Token("withPayload", 0, 11);
withPayload.setPayload(new BytesRef("test"));
ts = new CannedTokenStream(withPayload);
assertTrue(ts.hasAttribute(PayloadAttribute.class));
field2.setTokenStream(ts);
doc.add(field2);
Field field3 = new TextField("field", "", Field.Store.NO);
ts = new MockTokenizer(new StringReader("nopayload"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field3.setTokenStream(ts);
doc.add(field3);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
SegmentReader sr = getOnlySegmentReader(reader);
DocsAndPositionsEnum de = sr.termPositionsEnum(null, "field", new BytesRef("withPayload"));
de.nextDoc();
de.nextPosition();
assertEquals(new BytesRef("test"), de.getPayload());
writer.close();
reader.close();
dir.close();
}
}

View File

@ -0,0 +1,144 @@
package org.apache.lucene.index;
/*
* 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.
*/
import java.io.StringReader;
import org.apache.lucene.analysis.CannedTokenStream;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
public class TestPayloadsOnVectors extends LuceneTestCase {
/** some docs have payload att, some not */
public void testMixupDocs() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
iwc.setMergePolicy(newLogMergePolicy());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc);
Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
customType.setStoreTermVectors(true);
customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorPayloads(true);
customType.setStoreTermVectorOffsets(random().nextBoolean());
Field field = new Field("field", "", customType);
TokenStream ts = new MockTokenizer(new StringReader("here we go"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
doc.add(field);
writer.addDocument(doc);
Token withPayload = new Token("withPayload", 0, 11);
withPayload.setPayload(new BytesRef("test"));
ts = new CannedTokenStream(withPayload);
assertTrue(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
writer.addDocument(doc);
ts = new MockTokenizer(new StringReader("another"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
Terms terms = reader.getTermVector(1, "field");
assert terms != null;
TermsEnum termsEnum = terms.iterator(null);
assertTrue(termsEnum.seekExact(new BytesRef("withPayload"), true));
DocsAndPositionsEnum de = termsEnum.docsAndPositions(null, null);
assertEquals(0, de.nextDoc());
assertEquals(0, de.nextPosition());
assertEquals(new BytesRef("test"), de.getPayload());
writer.close();
reader.close();
dir.close();
}
/** some field instances have payload att, some not */
public void testMixupMultiValued() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
customType.setStoreTermVectors(true);
customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorPayloads(true);
customType.setStoreTermVectorOffsets(random().nextBoolean());
Field field = new Field("field", "", customType);
TokenStream ts = new MockTokenizer(new StringReader("here we go"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field.setTokenStream(ts);
doc.add(field);
Field field2 = new Field("field", "", customType);
Token withPayload = new Token("withPayload", 0, 11);
withPayload.setPayload(new BytesRef("test"));
ts = new CannedTokenStream(withPayload);
assertTrue(ts.hasAttribute(PayloadAttribute.class));
field2.setTokenStream(ts);
doc.add(field2);
Field field3 = new Field("field", "", customType);
ts = new MockTokenizer(new StringReader("nopayload"), MockTokenizer.WHITESPACE, true);
assertFalse(ts.hasAttribute(PayloadAttribute.class));
field3.setTokenStream(ts);
doc.add(field3);
writer.addDocument(doc);
DirectoryReader reader = writer.getReader();
Terms terms = reader.getTermVector(0, "field");
assert terms != null;
TermsEnum termsEnum = terms.iterator(null);
assertTrue(termsEnum.seekExact(new BytesRef("withPayload"), true));
DocsAndPositionsEnum de = termsEnum.docsAndPositions(null, null);
assertEquals(0, de.nextDoc());
assertEquals(3, de.nextPosition());
assertEquals(new BytesRef("test"), de.getPayload());
writer.close();
reader.close();
dir.close();
}
public void testPayloadsWithoutPositions() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_NOT_STORED);
customType.setStoreTermVectors(true);
customType.setStoreTermVectorPositions(false);
customType.setStoreTermVectorPayloads(true);
customType.setStoreTermVectorOffsets(random().nextBoolean());
doc.add(new Field("field", "foo", customType));
try {
writer.addDocument(doc);
fail();
} catch (IllegalArgumentException expected) {
// expected
}
writer.close();
dir.close();
}
}

View File

@ -23,8 +23,10 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.TreeMap;
@ -731,20 +733,21 @@ public class TestPostingsFormat extends LuceneTestCase {
System.out.println(" now check payload length=" + (position.payload == null ? 0 : position.payload.length));
}
if (position.payload == null || position.payload.length == 0) {
assertFalse("should not have payload", docsAndPositionsEnum.hasPayload());
assertNull("should not have payload", docsAndPositionsEnum.getPayload());
} else {
assertTrue("should have payload but doesn't", docsAndPositionsEnum.hasPayload());
BytesRef payload = docsAndPositionsEnum.getPayload();
assertFalse("2nd call to hasPayload should be false", docsAndPositionsEnum.hasPayload());
assertNotNull("should have payload but doesn't", payload);
assertNotNull("payload should not be null", payload);
assertEquals("payload length is wrong", position.payload.length, payload.length);
for(int byteUpto=0;byteUpto<position.payload.length;byteUpto++) {
assertEquals("payload bytes are wrong",
position.payload[byteUpto],
payload.bytes[payload.offset+byteUpto]);
}
// make a deep copy
payload = BytesRef.deepCopyOf(payload);
assertEquals("2nd call to getPayload returns something different!", payload, docsAndPositionsEnum.getPayload());
}
} else {
if (VERBOSE) {
@ -888,6 +891,26 @@ public class TestPostingsFormat extends LuceneTestCase {
}
}
}
private void testFields(Fields fields) throws Exception {
Iterator<String> iterator = fields.iterator();
while (iterator.hasNext()) {
String field = iterator.next();
try {
iterator.remove();
fail("Fields.iterator() allows for removal");
} catch (UnsupportedOperationException expected) {
// expected;
}
}
assertFalse(iterator.hasNext());
try {
iterator.next();
fail("Fields.iterator() doesn't throw NoSuchElementException when past the end");
} catch (NoSuchElementException expected) {
// expected
}
}
public void test() throws Exception {
Directory dir = newFSDirectory(_TestUtil.getTempDir("testPostingsFormat"));
@ -897,6 +920,7 @@ public class TestPostingsFormat extends LuceneTestCase {
FieldsProducer fieldsProducer = buildIndex(dir, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, indexPayloads);
testFields(fieldsProducer);
//testTerms(fieldsProducer, EnumSet.noneOf(Option.class), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
//testTerms(fieldsProducer, EnumSet.of(Option.LIVE_DOCS), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
//testTerms(fieldsProducer, EnumSet.of(Option.TERM_STATE, Option.LIVE_DOCS, Option.PARTIAL_DOC_CONSUME, Option.PARTIAL_POS_CONSUME), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);

View File

@ -170,7 +170,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
assertTrue(storedNumbers.substring(start, end).equals(term));
if (withPayloads) {
// check that we have a payload and it starts with "pos"
assertTrue(dp.hasPayload());
assertNotNull(dp.getPayload());
BytesRef payload = dp.getPayload();
assertTrue(payload.utf8ToString().startsWith("pos:"));
} // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!
@ -198,7 +198,7 @@ public class TestPostingsOffsets extends LuceneTestCase {
assertTrue(storedNumbers.substring(start, end).equals("hundred"));
if (withPayloads) {
// check that we have a payload and it starts with "pos"
assertTrue(dp.hasPayload());
assertNotNull(dp.getPayload());
BytesRef payload = dp.getPayload();
assertTrue(payload.utf8ToString().startsWith("pos:"));
} // note: withPayloads=false doesnt necessarily mean we dont have them from MockAnalyzer!

View File

@ -25,6 +25,7 @@ import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.MergedIterator;
import org.apache.lucene.util._TestUtil;
public class TestPrefixCodedTerms extends LuceneTestCase {
@ -67,17 +68,20 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
assertFalse(expected.hasNext());
}
@SuppressWarnings("unchecked")
public void testMergeEmpty() {
List<Iterator<Term>> subs = Collections.emptyList();
assertFalse(CoalescedDeletes.mergedIterator(subs).hasNext());
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertFalse(merged.hasNext());
subs = new ArrayList<Iterator<Term>>();
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
subs.add(new PrefixCodedTerms.Builder().finish().iterator());
Iterator<Term> merged = CoalescedDeletes.mergedIterator(subs);
merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertFalse(merged.hasNext());
}
@SuppressWarnings("unchecked")
public void testMergeOne() {
Term t1 = new Term("foo", "a");
PrefixCodedTerms.Builder b1 = new PrefixCodedTerms.Builder();
@ -93,13 +97,14 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
subs.add(pb1.iterator());
subs.add(pb2.iterator());
Iterator<Term> merged = CoalescedDeletes.mergedIterator(subs);
Iterator<Term> merged = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
assertTrue(merged.hasNext());
assertEquals(t1, merged.next());
assertTrue(merged.hasNext());
assertEquals(t2, merged.next());
}
@SuppressWarnings("unchecked")
public void testMergeRandom() {
PrefixCodedTerms pb[] = new PrefixCodedTerms[_TestUtil.nextInt(random(), 2, 10)];
Set<Term> superSet = new TreeSet<Term>();
@ -126,7 +131,7 @@ public class TestPrefixCodedTerms extends LuceneTestCase {
}
Iterator<Term> expected = superSet.iterator();
Iterator<Term> actual = CoalescedDeletes.mergedIterator(subs);
Iterator<Term> actual = new MergedIterator<Term>(subs.toArray(new Iterator[0]));
while (actual.hasNext()) {
assertTrue(expected.hasNext());
assertEquals(expected.next(), actual.next());

View File

@ -67,13 +67,15 @@ public class TestReaderClosed extends LuceneTestCase {
// expected
}
}
// LUCENE-3800
public void testReaderChaining() throws Exception {
assertTrue(reader.getRefCount() > 0);
IndexReader wrappedReader = SlowCompositeReaderWrapper.wrap(reader);
wrappedReader = new ParallelAtomicReader((AtomicReader) wrappedReader);
IndexSearcher searcher = newSearcher(wrappedReader);
TermRangeQuery query = TermRangeQuery.newStringRange("field", "a", "z", true, true);
searcher.search(query, 5);
reader.close(); // close original child reader
@ -84,6 +86,9 @@ public class TestReaderClosed extends LuceneTestCase {
"this IndexReader cannot be used anymore as one of its child readers was closed",
ace.getMessage()
);
} finally {
// shutdown executor: in case of wrap-wrap-wrapping
searcher.getIndexReader().close();
}
}

View File

@ -114,10 +114,9 @@ public class TestSegmentReader extends LuceneTestCase {
}
public void testTerms() throws IOException {
FieldsEnum fields = MultiFields.getFields(reader).iterator();
String field;
while((field = fields.next()) != null) {
Terms terms = fields.terms();
Fields fields = MultiFields.getFields(reader);
for (String field : fields) {
Terms terms = fields.terms(field);
assertNotNull(terms);
TermsEnum termsEnum = terms.iterator(null);
while(termsEnum.next() != null) {

View File

@ -396,13 +396,11 @@ public class TestStressIndexing2 extends LuceneTestCase {
Fields tv1 = r1.getTermVectors(id1);
System.out.println(" d1=" + tv1);
if (tv1 != null) {
FieldsEnum fieldsEnum = tv1.iterator();
String field;
DocsAndPositionsEnum dpEnum = null;
DocsEnum dEnum = null;
while ((field=fieldsEnum.next()) != null) {
for (String field : tv1) {
System.out.println(" " + field + ":");
Terms terms3 = fieldsEnum.terms();
Terms terms3 = tv1.terms(field);
assertNotNull(terms3);
TermsEnum termsEnum3 = terms3.iterator(null);
BytesRef term2;
@ -430,13 +428,11 @@ public class TestStressIndexing2 extends LuceneTestCase {
Fields tv2 = r2.getTermVectors(id2);
System.out.println(" d2=" + tv2);
if (tv2 != null) {
FieldsEnum fieldsEnum = tv2.iterator();
String field;
DocsAndPositionsEnum dpEnum = null;
DocsEnum dEnum = null;
while ((field=fieldsEnum.next()) != null) {
for (String field : tv2) {
System.out.println(" " + field + ":");
Terms terms3 = fieldsEnum.terms();
Terms terms3 = tv2.terms(field);
assertNotNull(terms3);
TermsEnum termsEnum3 = terms3.iterator(null);
BytesRef term2;
@ -469,8 +465,11 @@ public class TestStressIndexing2 extends LuceneTestCase {
// Verify postings
//System.out.println("TEST: create te1");
final FieldsEnum fields1 = MultiFields.getFields(r1).iterator();
final FieldsEnum fields2 = MultiFields.getFields(r2).iterator();
final Fields fields1 = MultiFields.getFields(r1);
final Iterator<String> fields1Enum = fields1.iterator();
final Fields fields2 = MultiFields.getFields(r2);
final Iterator<String> fields2Enum = fields2.iterator();
String field1=null, field2=null;
TermsEnum termsEnum1 = null;
@ -489,16 +488,15 @@ public class TestStressIndexing2 extends LuceneTestCase {
for(;;) {
len1=0;
if (termsEnum1 == null) {
field1 = fields1.next();
if (field1 == null) {
if (!fields1Enum.hasNext()) {
break;
} else {
Terms terms = fields1.terms();
if (terms == null) {
continue;
}
termsEnum1 = terms.iterator(null);
}
field1 = fields1Enum.next();
Terms terms = fields1.terms(field1);
if (terms == null) {
continue;
}
termsEnum1 = terms.iterator(null);
}
term1 = termsEnum1.next();
if (term1 == null) {
@ -523,16 +521,15 @@ public class TestStressIndexing2 extends LuceneTestCase {
for(;;) {
len2=0;
if (termsEnum2 == null) {
field2 = fields2.next();
if (field2 == null) {
if (!fields2Enum.hasNext()) {
break;
} else {
Terms terms = fields2.terms();
if (terms == null) {
continue;
}
termsEnum2 = terms.iterator(null);
}
field2 = fields2Enum.next();
Terms terms = fields2.terms(field2);
if (terms == null) {
continue;
}
termsEnum2 = terms.iterator(null);
}
term2 = termsEnum2.next();
if (term2 == null) {
@ -605,18 +602,17 @@ public class TestStressIndexing2 extends LuceneTestCase {
}
assertTrue(d2 != null);
FieldsEnum fieldsEnum1 = d1.iterator();
FieldsEnum fieldsEnum2 = d2.iterator();
String field1;
while ((field1 = fieldsEnum1.next()) != null) {
Iterator<String> fieldsEnum2 = d2.iterator();
for (String field1 : d1) {
String field2 = fieldsEnum2.next();
assertEquals(field1, field2);
Terms terms1 = fieldsEnum1.terms();
Terms terms1 = d1.terms(field1);
assertNotNull(terms1);
TermsEnum termsEnum1 = terms1.iterator(null);
Terms terms2 = fieldsEnum2.terms();
Terms terms2 = d2.terms(field2);
assertNotNull(terms2);
TermsEnum termsEnum2 = terms2.iterator(null);
@ -689,7 +685,7 @@ public class TestStressIndexing2 extends LuceneTestCase {
assertNull(termsEnum2.next());
}
assertNull(fieldsEnum2.next());
assertFalse(fieldsEnum2.hasNext());
}
private class IndexingThread extends Thread {

View File

@ -74,9 +74,8 @@ public class TestSumDocFreq extends LuceneTestCase {
private void assertSumDocFreq(IndexReader ir) throws Exception {
// compute sumDocFreq across all fields
Fields fields = MultiFields.getFields(ir);
FieldsEnum fieldEnum = fields.iterator();
String f = null;
while ((f = fieldEnum.next()) != null) {
for (String f : fields) {
Terms terms = fields.terms(f);
long sumDocFreq = terms.getSumDocFreq();
if (sumDocFreq == -1) {

Some files were not shown because too many files have changed in this diff Show More