mirror of https://github.com/apache/lucene.git
Merged /lucene/dev/trunk:r1433031-1435376
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1435384 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
619d43122a
|
@ -1,7 +1,11 @@
|
|||
The tools, documentation, etc. in this directory are intended to be helpful to developers of Lucene and Solr. They are not necessarily maintained in the same way that the core code is maintained and so your mileage may vary as to the usefulness of the tools.
|
||||
The tools, documentation, etc. in this directory are intended to be helpful
|
||||
to developers of Lucene and Solr. They are not necessarily maintained in
|
||||
the same way that the core code is maintained and so your mileage may vary
|
||||
as to the usefulness of the tools.
|
||||
|
||||
Description of directories:
|
||||
./size-estimator-lucene-solr -- Spreadsheet for estimating memory and disk usage in Lucene/Solr
|
||||
Description of dev-tools/ contents:
|
||||
|
||||
./size-estimator-lucene-solr.xls -- Spreadsheet for estimating memory and disk usage in Lucene/Solr
|
||||
./eclipse -- Used to generate project descriptors for the Eclipse IDE.
|
||||
./idea -- Similar to Eclipse, but for IntelliJ's IDEA IDE.
|
||||
./maven -- Mavenizes the Lucene/Solr packages
|
||||
|
|
|
@ -391,11 +391,6 @@
|
|||
<artifactId>jcl-over-slf4j</artifactId>
|
||||
<version>${slf4j.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>log4j-over-slf4j</artifactId>
|
||||
<version>${slf4j.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
|
|
|
@ -57,11 +57,6 @@
|
|||
<artifactId>easymock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>log4j-over-slf4j</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<sourceDirectory/>
|
||||
|
|
|
@ -76,12 +76,6 @@
|
|||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<!-- Need log4j api so long as Zookeeper does; see https://issues.apache.org/jira/browse/ZOOKEEPER-850 -->
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>log4j-over-slf4j</artifactId>
|
||||
<optional>true</optional> <!-- let the end user decide... they may be using log4j directly! -->
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>jcl-over-slf4j</artifactId>
|
||||
|
|
|
@ -52,11 +52,6 @@
|
|||
<version>${project.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>log4j-over-slf4j</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<sourceDirectory/>
|
||||
|
|
|
@ -62,10 +62,6 @@
|
|||
<artifactId>slf4j-jdk14</artifactId>
|
||||
<scope>runtime</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>log4j-over-slf4j</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>jcl-over-slf4j</artifactId>
|
||||
|
|
|
@ -19,9 +19,9 @@ import shutil
|
|||
import os
|
||||
import sys
|
||||
|
||||
# Usage: python -u buildRelease.py [-sign gpgKey(eg: 6E68DA61)] [-prepare] [-push userName] [-pushLocal dirName] [-smoke tmpDir] /path/to/checkout version(eg: 3.4.0) rcNum(eg: 0)
|
||||
# Usage: python3.2 -u buildAndPushRelease.py [-sign gpgKey(eg: 6E68DA61)] [-prepare] [-push userName] [-pushLocal dirName] [-smoke tmpDir] /path/to/checkout version(eg: 3.4.0) rcNum(eg: 0)
|
||||
#
|
||||
# EG: python -u buildRelease.py -prepare -push -sign 6E68DA61 mikemccand /lucene/34x 3.4.0 0
|
||||
# EG: python3.2 -u buildAndPushRelease.py -prepare -push -sign 6E68DA61 mikemccand /lucene/34x 3.4.0 0
|
||||
|
||||
# NOTE: if you specify -sign, you have to type in your gpg password at
|
||||
# some point while this runs; it's VERY confusing because the output
|
||||
|
@ -32,15 +32,15 @@ import sys
|
|||
LOG = '/tmp/release.log'
|
||||
|
||||
def log(msg):
|
||||
f = open(LOG, 'ab')
|
||||
f.write(msg)
|
||||
f = open(LOG, mode='ab')
|
||||
f.write(msg.encode('utf-8'))
|
||||
f.close()
|
||||
|
||||
def run(command):
|
||||
log('\n\n%s: RUN: %s\n' % (datetime.datetime.now(), command))
|
||||
if os.system('%s >> %s 2>&1' % (command, LOG)):
|
||||
msg = ' FAILED: %s [see log %s]' % (command, LOG)
|
||||
print msg
|
||||
print(msg)
|
||||
raise RuntimeError(msg)
|
||||
|
||||
def scrubCheckout():
|
||||
|
@ -53,7 +53,7 @@ def scrubCheckout():
|
|||
if match:
|
||||
s = match.group(1)
|
||||
if os.path.exists(s):
|
||||
print ' delete %s' % s
|
||||
print(' delete %s' % s)
|
||||
if os.path.isdir(s) and not os.path.islink(s):
|
||||
shutil.rmtree(s)
|
||||
else:
|
||||
|
@ -69,29 +69,29 @@ def getSVNRev():
|
|||
|
||||
|
||||
def prepare(root, version, gpgKeyID, doTest):
|
||||
print
|
||||
print 'Prepare release...'
|
||||
print()
|
||||
print('Prepare release...')
|
||||
if os.path.exists(LOG):
|
||||
os.remove(LOG)
|
||||
|
||||
os.chdir(root)
|
||||
print ' svn up...'
|
||||
print(' svn up...')
|
||||
run('svn up')
|
||||
|
||||
rev = getSVNRev()
|
||||
print ' svn rev: %s' % rev
|
||||
print(' svn rev: %s' % rev)
|
||||
log('\nSVN rev: %s\n' % rev)
|
||||
|
||||
if doTest:
|
||||
# Don't run tests if we are gonna smoke test after the release...
|
||||
print ' ant clean test'
|
||||
print(' ant clean test')
|
||||
run('ant clean test')
|
||||
|
||||
print ' clean checkout'
|
||||
print(' clean checkout')
|
||||
scrubCheckout()
|
||||
open('rev.txt', 'wb').write(rev)
|
||||
open('rev.txt', mode='wb').write(rev.encode('UTF-8'))
|
||||
|
||||
print ' lucene prepare-release'
|
||||
print(' lucene prepare-release')
|
||||
os.chdir('lucene')
|
||||
cmd = 'ant -Dversion=%s -Dspecversion=%s' % (version, version)
|
||||
if gpgKeyID is not None:
|
||||
|
@ -100,7 +100,7 @@ def prepare(root, version, gpgKeyID, doTest):
|
|||
cmd += ' prepare-release-no-sign'
|
||||
run(cmd)
|
||||
|
||||
print ' solr prepare-release'
|
||||
print(' solr prepare-release')
|
||||
os.chdir('../solr')
|
||||
cmd = 'ant -Dversion=%s -Dspecversion=%s' % (version, version)
|
||||
if gpgKeyID is not None:
|
||||
|
@ -108,105 +108,94 @@ def prepare(root, version, gpgKeyID, doTest):
|
|||
else:
|
||||
cmd += ' prepare-release-no-sign'
|
||||
run(cmd)
|
||||
print ' done!'
|
||||
print
|
||||
print(' done!')
|
||||
print()
|
||||
return rev
|
||||
|
||||
def push(version, root, rev, rcNum, username):
|
||||
print 'Push...'
|
||||
print('Push...')
|
||||
dir = 'lucene-solr-%s-RC%d-rev%s' % (version, rcNum, rev)
|
||||
s = os.popen('ssh %s@people.apache.org "ls -ld public_html/staging_area/%s" 2>&1' % (username, dir)).read()
|
||||
if s.lower().find('no such file or directory') == -1:
|
||||
print ' Remove old dir...'
|
||||
s = os.popen('ssh %s@people.apache.org "ls -ld public_html/staging_area/%s" 2>&1' % (username, dir)).read().decode('UTF-8')
|
||||
if 'no such file or directory' not in s.lower():
|
||||
print(' Remove old dir...')
|
||||
run('ssh %s@people.apache.org "chmod -R u+rwX public_html/staging_area/%s; rm -rf public_html/staging_area/%s"' %
|
||||
(username, dir, dir))
|
||||
run('ssh %s@people.apache.org "mkdir -p public_html/staging_area/%s/lucene public_html/staging_area/%s/solr"' % \
|
||||
(username, dir, dir))
|
||||
print ' Lucene'
|
||||
print(' Lucene')
|
||||
os.chdir('%s/lucene/dist' % root)
|
||||
print ' zip...'
|
||||
print(' zip...')
|
||||
if os.path.exists('lucene.tar.bz2'):
|
||||
os.remove('lucene.tar.bz2')
|
||||
run('tar cjf lucene.tar.bz2 *')
|
||||
print ' copy...'
|
||||
print(' copy...')
|
||||
run('scp lucene.tar.bz2 %s@people.apache.org:public_html/staging_area/%s/lucene' % (username, dir))
|
||||
print ' unzip...'
|
||||
print(' unzip...')
|
||||
run('ssh %s@people.apache.org "cd public_html/staging_area/%s/lucene; tar xjf lucene.tar.bz2; rm -f lucene.tar.bz2"' % (username, dir))
|
||||
os.remove('lucene.tar.bz2')
|
||||
print ' copy changes...'
|
||||
os.chdir('..')
|
||||
run('scp -r build/docs/changes %s@people.apache.org:public_html/staging_area/%s/lucene/changes-%s' % (username, dir, version))
|
||||
|
||||
print ' Solr'
|
||||
print(' Solr')
|
||||
os.chdir('%s/solr/package' % root)
|
||||
print ' zip...'
|
||||
print(' zip...')
|
||||
if os.path.exists('solr.tar.bz2'):
|
||||
os.remove('solr.tar.bz2')
|
||||
run('tar cjf solr.tar.bz2 *')
|
||||
print ' copy...'
|
||||
print(' copy...')
|
||||
run('scp solr.tar.bz2 %s@people.apache.org:public_html/staging_area/%s/solr' % (username, dir))
|
||||
print ' unzip...'
|
||||
print(' unzip...')
|
||||
run('ssh %s@people.apache.org "cd public_html/staging_area/%s/solr; tar xjf solr.tar.bz2; rm -f solr.tar.bz2"' % (username, dir))
|
||||
os.remove('solr.tar.bz2')
|
||||
|
||||
print ' KEYS'
|
||||
run('wget http://people.apache.org/keys/group/lucene.asc')
|
||||
os.rename('lucene.asc', 'KEYS')
|
||||
run('chmod a+r-w KEYS')
|
||||
run('scp KEYS %s@people.apache.org:public_html/staging_area/%s/lucene' % (username, dir))
|
||||
run('scp KEYS %s@people.apache.org:public_html/staging_area/%s/solr' % (username, dir))
|
||||
os.remove('KEYS')
|
||||
|
||||
print ' chmod...'
|
||||
print(' chmod...')
|
||||
run('ssh %s@people.apache.org "chmod -R a+rX-w public_html/staging_area/%s"' % (username, dir))
|
||||
|
||||
print ' done!'
|
||||
print(' done!')
|
||||
url = 'https://people.apache.org/~%s/staging_area/%s' % (username, dir)
|
||||
return url
|
||||
|
||||
def pushLocal(version, root, rev, rcNum, localDir):
|
||||
print 'Push local [%s]...' % localDir
|
||||
print('Push local [%s]...' % localDir)
|
||||
os.makedirs(localDir)
|
||||
|
||||
dir = 'lucene-solr-%s-RC%d-rev%s' % (version, rcNum, rev)
|
||||
os.makedirs('%s/%s/lucene' % (localDir, dir))
|
||||
os.makedirs('%s/%s/solr' % (localDir, dir))
|
||||
print ' Lucene'
|
||||
print(' Lucene')
|
||||
os.chdir('%s/lucene/dist' % root)
|
||||
print ' zip...'
|
||||
print(' zip...')
|
||||
if os.path.exists('lucene.tar.bz2'):
|
||||
os.remove('lucene.tar.bz2')
|
||||
run('tar cjf lucene.tar.bz2 *')
|
||||
|
||||
os.chdir('%s/%s/lucene' % (localDir, dir))
|
||||
print ' unzip...'
|
||||
print(' unzip...')
|
||||
run('tar xjf "%s/lucene/dist/lucene.tar.bz2"' % root)
|
||||
os.remove('%s/lucene/dist/lucene.tar.bz2' % root)
|
||||
print ' copy changes...'
|
||||
print(' copy changes...')
|
||||
run('cp -r "%s/lucene/build/docs/changes" changes-%s' % (root, version))
|
||||
|
||||
print ' Solr'
|
||||
print(' Solr')
|
||||
os.chdir('%s/solr/package' % root)
|
||||
print ' zip...'
|
||||
print(' zip...')
|
||||
if os.path.exists('solr.tar.bz2'):
|
||||
os.remove('solr.tar.bz2')
|
||||
run('tar cjf solr.tar.bz2 *')
|
||||
print ' unzip...'
|
||||
print(' unzip...')
|
||||
os.chdir('%s/%s/solr' % (localDir, dir))
|
||||
run('tar xjf "%s/solr/package/solr.tar.bz2"' % root)
|
||||
os.remove('%s/solr/package/solr.tar.bz2' % root)
|
||||
|
||||
print ' KEYS'
|
||||
print(' KEYS')
|
||||
run('wget http://people.apache.org/keys/group/lucene.asc')
|
||||
os.rename('lucene.asc', 'KEYS')
|
||||
run('chmod a+r-w KEYS')
|
||||
run('cp KEYS ../lucene')
|
||||
|
||||
print ' chmod...'
|
||||
print(' chmod...')
|
||||
os.chdir('..')
|
||||
run('chmod -R a+rX-w .')
|
||||
|
||||
print ' done!'
|
||||
print(' done!')
|
||||
return 'file://%s/%s' % (os.path.abspath(localDir), dir)
|
||||
|
||||
def main():
|
||||
|
@ -231,9 +220,9 @@ def main():
|
|||
smokeTmpDir = sys.argv[idx+1]
|
||||
del sys.argv[idx:idx+2]
|
||||
if os.path.exists(smokeTmpDir):
|
||||
print
|
||||
print 'ERROR: smoke tmpDir "%s" exists; please remove first' % smokeTmpDir
|
||||
print
|
||||
print()
|
||||
print('ERROR: smoke tmpDir "%s" exists; please remove first' % smokeTmpDir)
|
||||
print()
|
||||
sys.exit(1)
|
||||
|
||||
try:
|
||||
|
@ -245,15 +234,15 @@ def main():
|
|||
localStagingDir = sys.argv[idx+1]
|
||||
del sys.argv[idx:idx+2]
|
||||
if os.path.exists(localStagingDir):
|
||||
print
|
||||
print 'ERROR: pushLocal dir "%s" exists; please remove first' % localStagingDir
|
||||
print
|
||||
print()
|
||||
print('ERROR: pushLocal dir "%s" exists; please remove first' % localStagingDir)
|
||||
print()
|
||||
sys.exit(1)
|
||||
|
||||
if doPushRemote and doPushLocal:
|
||||
print
|
||||
print 'ERROR: specify at most one of -push or -pushLocal (got both)'
|
||||
print
|
||||
print()
|
||||
print('ERROR: specify at most one of -push or -pushLocal (got both)')
|
||||
print()
|
||||
sys.exit(1)
|
||||
|
||||
try:
|
||||
|
@ -263,7 +252,7 @@ def main():
|
|||
else:
|
||||
gpgKeyID = sys.argv[idx+1]
|
||||
del sys.argv[idx:idx+2]
|
||||
|
||||
|
||||
root = os.path.abspath(sys.argv[1])
|
||||
version = sys.argv[2]
|
||||
rcNum = int(sys.argv[3])
|
||||
|
@ -272,22 +261,26 @@ def main():
|
|||
rev = prepare(root, version, gpgKeyID, smokeTmpDir is None)
|
||||
else:
|
||||
os.chdir(root)
|
||||
rev = open('rev.txt').read()
|
||||
rev = open('rev.txt', encoding='UTF-8').read()
|
||||
|
||||
if doPushRemote:
|
||||
url = push(version, root, rev, rcNum, username)
|
||||
elif doPushLocal:
|
||||
url = pushLocal(version, root, rev, rcNum, localStagingDir)
|
||||
else:
|
||||
url = NOne
|
||||
url = None
|
||||
|
||||
if url is not None:
|
||||
print ' URL: %s' % url
|
||||
print(' URL: %s' % url)
|
||||
|
||||
if smokeTmpDir is not None:
|
||||
import smokeTestRelease
|
||||
smokeTestRelease.DEBUG = False
|
||||
smokeTestRelease.smokeTest(url, version, smokeTmpDir, gpgKeyID is not None)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
main()
|
||||
try:
|
||||
main()
|
||||
except:
|
||||
import traceback
|
||||
traceback.print_exc()
|
||||
|
|
|
@ -19,6 +19,21 @@ Changes in backwards compatibility policy
|
|||
(Nikola Tanković, Uwe Schindler, Chris Male, Mike McCandless,
|
||||
Robert Muir)
|
||||
|
||||
======================= Lucene 4.2.0 =======================
|
||||
|
||||
Changes in backwards compatibility policy
|
||||
|
||||
* LUCENE-4602: FacetFields now stores facet ordinals in a DocValues field,
|
||||
rather than a payload. This forces rebuilding existing indexes, or do a
|
||||
one time migration using FacetsPayloadMigratingReader. Since DocValues
|
||||
support in-memory caching, CategoryListCache was removed too.
|
||||
(Shai Erera, Michael McCandless)
|
||||
|
||||
Optimizations
|
||||
|
||||
* LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate
|
||||
TermsEnum only if needed to do a seek or get a DocsEnum. (Simon Willnauer)
|
||||
|
||||
* LUCENE-4677, LUCENE-4682: unpacked FSTs now use vInt to encode the node target,
|
||||
to reduce their size (Mike McCandless)
|
||||
|
||||
|
@ -29,6 +44,11 @@ Changes in backwards compatibility policy
|
|||
* LUCENE-3298: FST can now be larger than 2.1 GB / 2.1 B nodes.
|
||||
(James Dyer, Mike McCandless)
|
||||
|
||||
New Features
|
||||
|
||||
* LUCENE-4686: New specialized DGapVInt8IntEncoder for facets (now the
|
||||
default). (Shai Erera)
|
||||
|
||||
======================= Lucene 4.1.0 =======================
|
||||
|
||||
Changes in backwards compatibility policy
|
||||
|
@ -150,7 +170,7 @@ New Features
|
|||
remove stop words then "ghost chr..." could suggest "The Ghost of
|
||||
Christmas Past"; if SynonymFilter is used to map wifi and wireless
|
||||
network to hotspot, then "wirele..." could suggest "wifi router";
|
||||
token normalization likes stemmers, accent removel, etc. would allow
|
||||
token normalization likes stemmers, accent removal, etc. would allow
|
||||
the suggester to ignore such variations. (Robert Muir, Sudarshan
|
||||
Gaikaiwari, Mike McCandless)
|
||||
|
||||
|
@ -202,7 +222,7 @@ API Changes
|
|||
information about the trigger of the merge ie. merge triggered due
|
||||
to a segment merge or a full flush etc. (Simon Willnauer)
|
||||
|
||||
* Lucene-4415: TermsFilter is now immutable. All terms need to be provided
|
||||
* LUCENE-4415: TermsFilter is now immutable. All terms need to be provided
|
||||
as constructor argument. (Simon Willnauer)
|
||||
|
||||
* LUCENE-4520: ValueSource.getSortField no longer throws IOExceptions
|
||||
|
@ -227,6 +247,9 @@ API Changes
|
|||
* LUCENE-4663: Deprecate IndexSearcher.document(int, Set). This was not intended
|
||||
to be final, nor named document(). Use IndexSearcher.doc(int, Set) instead.
|
||||
(Robert Muir)
|
||||
|
||||
* LUCENE-4684: Made DirectSpellChecker extendable.
|
||||
(Martijn van Groningen)
|
||||
|
||||
Bug Fixes
|
||||
|
||||
|
@ -343,6 +366,9 @@ Bug Fixes
|
|||
DEFAULT_ARTICLES list passed to ElisionFilter. (David Leunen via Steve Rowe)
|
||||
|
||||
* LUCENE-4671: Fix CharsRef.subSequence method. (Tim Smith via Robert Muir)
|
||||
|
||||
* LUCENE-4465: Let ConstantScoreQuery's Scorer return its child scorer.
|
||||
(selckin via Uwe Schindler)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
|
|
|
@ -59,7 +59,7 @@ public class MappingCharFilter extends BaseCharFilter {
|
|||
cachedRootArcs = normMap.cachedRootArcs;
|
||||
|
||||
if (map != null) {
|
||||
fstReader = map.getBytesReader(0);
|
||||
fstReader = map.getBytesReader();
|
||||
} else {
|
||||
fstReader = null;
|
||||
}
|
||||
|
|
|
@ -49,7 +49,7 @@ public class NormalizeCharMap {
|
|||
try {
|
||||
// Pre-cache root arcs:
|
||||
final FST.Arc<CharsRef> scratchArc = new FST.Arc<CharsRef>();
|
||||
final FST.BytesReader fstReader = map.getBytesReader(0);
|
||||
final FST.BytesReader fstReader = map.getBytesReader();
|
||||
map.getFirstArc(scratchArc);
|
||||
if (FST.targetHasArcs(scratchArc)) {
|
||||
map.readFirstRealTargetArc(scratchArc.target, scratchArc, fstReader);
|
||||
|
|
|
@ -148,7 +148,7 @@ public class HyphenationCompoundWordTokenFilter extends
|
|||
*/
|
||||
public static HyphenationTree getHyphenationTree(File hyphenationFile)
|
||||
throws IOException {
|
||||
return getHyphenationTree(new InputSource(hyphenationFile.toURL().toExternalForm()));
|
||||
return getHyphenationTree(new InputSource(hyphenationFile.toURI().toASCIIString()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -111,7 +111,7 @@ public class HyphenationTree extends TernaryTree implements PatternConsumer {
|
|||
* @throws IOException In case the parsing fails
|
||||
*/
|
||||
public void loadPatterns(File f) throws IOException {
|
||||
InputSource src = new InputSource(f.toURL().toExternalForm());
|
||||
InputSource src = new InputSource(f.toURI().toASCIIString());
|
||||
loadPatterns(src);
|
||||
}
|
||||
|
||||
|
|
|
@ -98,7 +98,7 @@ public class PatternParser extends DefaultHandler {
|
|||
* @throws IOException In case of an exception while parsing
|
||||
*/
|
||||
public void parse(File file) throws IOException {
|
||||
InputSource src = new InputSource(file.toURL().toExternalForm());
|
||||
InputSource src = new InputSource(file.toURI().toASCIIString());
|
||||
parse(src);
|
||||
}
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ import java.util.Set;
|
|||
* <analyzer>
|
||||
* <tokenizer class="solr.StandardTokenizerFactory"/>
|
||||
* <filter class="solr.TypeTokenFilterFactory" types="stoptypes.txt"
|
||||
* enablePositionIncrements="true" useWhiteList="false"/>
|
||||
* enablePositionIncrements="true" useWhitelist="false"/>
|
||||
* </analyzer>
|
||||
* </fieldType></pre>
|
||||
*/
|
||||
|
|
|
@ -263,7 +263,7 @@ public final class SynonymFilter extends TokenFilter {
|
|||
this.synonyms = synonyms;
|
||||
this.ignoreCase = ignoreCase;
|
||||
this.fst = synonyms.fst;
|
||||
this.fstReader = fst.getBytesReader(0);
|
||||
this.fstReader = fst.getBytesReader();
|
||||
if (fst == null) {
|
||||
throw new IllegalArgumentException("fst must be non-null");
|
||||
}
|
||||
|
|
|
@ -201,10 +201,10 @@ public final class JapaneseTokenizer extends Tokenizer {
|
|||
characterDefinition = unkDictionary.getCharacterDefinition();
|
||||
this.userDictionary = userDictionary;
|
||||
costs = ConnectionCosts.getInstance();
|
||||
fstReader = fst.getBytesReader(0);
|
||||
fstReader = fst.getBytesReader();
|
||||
if (userDictionary != null) {
|
||||
userFST = userDictionary.getFST();
|
||||
userFSTReader = userFST.getBytesReader(0);
|
||||
userFSTReader = userFST.getBytesReader();
|
||||
} else {
|
||||
userFST = null;
|
||||
userFSTReader = null;
|
||||
|
|
|
@ -54,7 +54,7 @@ public final class TokenInfoFST {
|
|||
FST.Arc<Long> firstArc = new FST.Arc<Long>();
|
||||
fst.getFirstArc(firstArc);
|
||||
FST.Arc<Long> arc = new FST.Arc<Long>();
|
||||
final FST.BytesReader fstReader = fst.getBytesReader(0);
|
||||
final FST.BytesReader fstReader = fst.getBytesReader();
|
||||
// TODO: jump to 3040, readNextRealArc to ceiling? (just be careful we don't add bugs)
|
||||
for (int i = 0; i < rootCache.length; i++) {
|
||||
if (fst.findTargetArc(0x3040 + i, firstArc, arc, fstReader) != null) {
|
||||
|
@ -83,8 +83,8 @@ public final class TokenInfoFST {
|
|||
return fst.getFirstArc(arc);
|
||||
}
|
||||
|
||||
public FST.BytesReader getBytesReader(int pos) {
|
||||
return fst.getBytesReader(pos);
|
||||
public FST.BytesReader getBytesReader() {
|
||||
return fst.getBytesReader();
|
||||
}
|
||||
|
||||
/** @lucene.internal for testing only */
|
||||
|
|
|
@ -139,7 +139,7 @@ public final class UserDictionary implements Dictionary {
|
|||
TreeMap<Integer, int[]> result = new TreeMap<Integer, int[]>(); // index, [length, length...]
|
||||
boolean found = false; // true if we found any results
|
||||
|
||||
final FST.BytesReader fstReader = fst.getBytesReader(0);
|
||||
final FST.BytesReader fstReader = fst.getBytesReader();
|
||||
|
||||
FST.Arc<Long> arc = new FST.Arc<Long>();
|
||||
int end = off + len;
|
||||
|
|
|
@ -236,26 +236,22 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
TermsEnum result;
|
||||
if ((reuse != null) && (reuse instanceof BloomFilteredTermsEnum)) {
|
||||
// recycle the existing BloomFilteredTermsEnum by asking the delegate
|
||||
// to recycle its contained TermsEnum
|
||||
BloomFilteredTermsEnum bfte = (BloomFilteredTermsEnum) reuse;
|
||||
if (bfte.filter == filter) {
|
||||
bfte.delegateTermsEnum = delegateTerms
|
||||
.iterator(bfte.delegateTermsEnum);
|
||||
bfte.reset(delegateTerms, bfte.delegateTermsEnum);
|
||||
return bfte;
|
||||
}
|
||||
}
|
||||
// We have been handed something we cannot reuse (either null, wrong
|
||||
// class or wrong filter) so allocate a new object
|
||||
result = new BloomFilteredTermsEnum(delegateTerms.iterator(reuse),
|
||||
filter);
|
||||
return result;
|
||||
return new BloomFilteredTermsEnum(delegateTerms, reuse, filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() throws IOException {
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return delegateTerms.getComparator();
|
||||
}
|
||||
|
||||
|
@ -295,24 +291,43 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
}
|
||||
}
|
||||
|
||||
class BloomFilteredTermsEnum extends TermsEnum {
|
||||
final class BloomFilteredTermsEnum extends TermsEnum {
|
||||
private Terms delegateTerms;
|
||||
private TermsEnum delegateTermsEnum;
|
||||
private TermsEnum reuseDelegate;
|
||||
private final FuzzySet filter;
|
||||
|
||||
TermsEnum delegateTermsEnum;
|
||||
private FuzzySet filter;
|
||||
|
||||
public BloomFilteredTermsEnum(TermsEnum iterator, FuzzySet filter) {
|
||||
this.delegateTermsEnum = iterator;
|
||||
public BloomFilteredTermsEnum(Terms delegateTerms, TermsEnum reuseDelegate, FuzzySet filter) throws IOException {
|
||||
this.delegateTerms = delegateTerms;
|
||||
this.reuseDelegate = reuseDelegate;
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
void reset(Terms delegateTerms, TermsEnum reuseDelegate) throws IOException {
|
||||
this.delegateTerms = delegateTerms;
|
||||
this.reuseDelegate = reuseDelegate;
|
||||
this.delegateTermsEnum = null;
|
||||
}
|
||||
|
||||
private final TermsEnum delegate() throws IOException {
|
||||
if (delegateTermsEnum == null) {
|
||||
/* pull the iterator only if we really need it -
|
||||
* this can be a relativly heavy operation depending on the
|
||||
* delegate postings format and they underlying directory
|
||||
* (clone IndexInput) */
|
||||
delegateTermsEnum = delegateTerms.iterator(reuseDelegate);
|
||||
}
|
||||
return delegateTermsEnum;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final BytesRef next() throws IOException {
|
||||
return delegateTermsEnum.next();
|
||||
return delegate().next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Comparator<BytesRef> getComparator() {
|
||||
return delegateTermsEnum.getComparator();
|
||||
return delegateTerms.getComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -326,51 +341,51 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
if (filter.contains(text) == ContainsResult.NO) {
|
||||
return false;
|
||||
}
|
||||
return delegateTermsEnum.seekExact(text, useCache);
|
||||
return delegate().seekExact(text, useCache);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final SeekStatus seekCeil(BytesRef text, boolean useCache)
|
||||
throws IOException {
|
||||
return delegateTermsEnum.seekCeil(text, useCache);
|
||||
return delegate().seekCeil(text, useCache);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void seekExact(long ord) throws IOException {
|
||||
delegateTermsEnum.seekExact(ord);
|
||||
delegate().seekExact(ord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final BytesRef term() throws IOException {
|
||||
return delegateTermsEnum.term();
|
||||
return delegate().term();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final long ord() throws IOException {
|
||||
return delegateTermsEnum.ord();
|
||||
return delegate().ord();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int docFreq() throws IOException {
|
||||
return delegateTermsEnum.docFreq();
|
||||
return delegate().docFreq();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final long totalTermFreq() throws IOException {
|
||||
return delegateTermsEnum.totalTermFreq();
|
||||
return delegate().totalTermFreq();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
|
||||
DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
return delegateTermsEnum.docsAndPositions(liveDocs, reuse, flags);
|
||||
return delegate().docsAndPositions(liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags)
|
||||
throws IOException {
|
||||
return delegateTermsEnum.docs(liveDocs, reuse, flags);
|
||||
return delegate().docs(liveDocs, reuse, flags);
|
||||
}
|
||||
|
||||
|
||||
|
@ -383,12 +398,10 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
|
|||
private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
|
||||
private SegmentWriteState state;
|
||||
|
||||
// private PostingsFormat delegatePostingsFormat;
|
||||
|
||||
public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
|
||||
SegmentWriteState state, PostingsFormat delegatePostingsFormat) {
|
||||
this.delegateFieldsConsumer = fieldsConsumer;
|
||||
// this.delegatePostingsFormat=delegatePostingsFormat;
|
||||
this.state = state;
|
||||
}
|
||||
|
||||
|
|
|
@ -70,10 +70,6 @@ import org.apache.lucene.util.packed.PackedInts;
|
|||
* queries that rely on advance will (AND BooleanQuery,
|
||||
* PhraseQuery) will be relatively slow!
|
||||
*
|
||||
* <p><b>NOTE</b>: this codec cannot address more than ~2.1 GB
|
||||
* of postings, because the underlying FST uses an int
|
||||
* to address the underlying byte[].
|
||||
*
|
||||
* @lucene.experimental */
|
||||
|
||||
// TODO: Maybe name this 'Cached' or something to reflect
|
||||
|
@ -88,6 +84,13 @@ public final class MemoryPostingsFormat extends PostingsFormat {
|
|||
this(false, PackedInts.DEFAULT);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create MemoryPostingsFormat, specifying advanced FST options.
|
||||
* @param doPackFST true if a packed FST should be built.
|
||||
* NOTE: packed FSTs are limited to ~2.1 GB of postings.
|
||||
* @param acceptableOverheadRatio allowable overhead for packed ints
|
||||
* during FST construction.
|
||||
*/
|
||||
public MemoryPostingsFormat(boolean doPackFST, float acceptableOverheadRatio) {
|
||||
super("Memory");
|
||||
this.doPackFST = doPackFST;
|
||||
|
|
|
@ -272,7 +272,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() throws IOException {
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
}
|
||||
|
||||
|
|
|
@ -833,7 +833,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
if (index == null) {
|
||||
fstReader = null;
|
||||
} else {
|
||||
fstReader = index.getBytesReader(0);
|
||||
fstReader = index.getBytesReader();
|
||||
}
|
||||
|
||||
// TODO: if the automaton is "smallish" we really
|
||||
|
@ -1277,7 +1277,7 @@ public class BlockTreeTermsReader extends FieldsProducer {
|
|||
if (index == null) {
|
||||
fstReader = null;
|
||||
} else {
|
||||
fstReader = index.getBytesReader(0);
|
||||
fstReader = index.getBytesReader();
|
||||
}
|
||||
|
||||
// Init w/ root block; don't use index since it may
|
||||
|
|
|
@ -218,7 +218,7 @@ class Lucene41DocValuesProducer extends DocValuesProducer {
|
|||
final FST<Long> fst = instance;
|
||||
|
||||
// per-thread resources
|
||||
final BytesReader in = fst.getBytesReader(0);
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
final Arc<Long> firstArc = new Arc<Long>();
|
||||
final Arc<Long> scratchArc = new Arc<Long>();
|
||||
final IntsRef scratchInts = new IntsRef();
|
||||
|
|
|
@ -87,7 +87,7 @@ public class FilterAtomicReader extends AtomicReader {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> getComparator() throws IOException {
|
||||
public Comparator<BytesRef> getComparator() {
|
||||
return in.getComparator();
|
||||
}
|
||||
|
||||
|
|
|
@ -80,7 +80,7 @@ public abstract class Terms {
|
|||
* if there are no terms. This method may be invoked
|
||||
* many times; it's best to cache a single instance &
|
||||
* reuse it. */
|
||||
public abstract Comparator<BytesRef> getComparator() throws IOException;
|
||||
public abstract Comparator<BytesRef> getComparator();
|
||||
|
||||
/** Returns the number of terms for this field, or -1 if this
|
||||
* measure isn't stored by the codec. Note that, just like
|
||||
|
|
|
@ -24,6 +24,8 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -248,6 +250,14 @@ public class ConstantScoreQuery extends Query {
|
|||
return super.score(collector, max, firstDocID);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
if (docIdSetIterator instanceof Scorer)
|
||||
return Collections.singletonList(new ChildScorer((Scorer) docIdSetIterator, "constant"));
|
||||
else
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -96,7 +96,7 @@ public class FuzzyQuery extends MultiTermQuery {
|
|||
|
||||
/**
|
||||
* Calls {@link #FuzzyQuery(Term, int, int, int, boolean)
|
||||
* FuzzyQuery(term, minimumSimilarity, prefixLength, defaultMaxExpansions, defaultTranspositions)}.
|
||||
* FuzzyQuery(term, maxEdits, prefixLength, defaultMaxExpansions, defaultTranspositions)}.
|
||||
*/
|
||||
public FuzzyQuery(Term term, int maxEdits, int prefixLength) {
|
||||
this(term, maxEdits, prefixLength, defaultMaxExpansions, defaultTranspositions);
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.lucene.search.DocIdSetIterator;
|
|||
public final class FixedBitSet extends DocIdSet implements Bits {
|
||||
private final long[] bits;
|
||||
private final int numBits;
|
||||
private final int wordLength;
|
||||
|
||||
/** returns the number of 64 bit words it would take to hold numBits */
|
||||
public static int bits2words(int numBits) {
|
||||
|
@ -52,23 +53,29 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
public FixedBitSet(int numBits) {
|
||||
this.numBits = numBits;
|
||||
bits = new long[bits2words(numBits)];
|
||||
wordLength = bits.length;
|
||||
}
|
||||
|
||||
public FixedBitSet(long[]storedBits,int numBits) {
|
||||
public FixedBitSet(long[] storedBits, int numBits) {
|
||||
this.wordLength = bits2words(numBits);
|
||||
if (wordLength > storedBits.length) {
|
||||
throw new IllegalArgumentException("The given long array is too small to hold " + numBits + " bits");
|
||||
}
|
||||
this.numBits = numBits;
|
||||
this.bits = storedBits;
|
||||
}
|
||||
|
||||
/** Makes full copy. */
|
||||
public FixedBitSet(FixedBitSet other) {
|
||||
bits = new long[other.bits.length];
|
||||
System.arraycopy(other.bits, 0, bits, 0, bits.length);
|
||||
bits = new long[other.wordLength];
|
||||
System.arraycopy(other.bits, 0, bits, 0, other.wordLength);
|
||||
numBits = other.numBits;
|
||||
wordLength = other.wordLength;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocIdSetIterator iterator() {
|
||||
return new OpenBitSetIterator(bits, bits.length);
|
||||
return new OpenBitSetIterator(bits, wordLength);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -159,7 +166,7 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
return (i<<6) + subIndex + Long.numberOfTrailingZeros(word);
|
||||
}
|
||||
|
||||
while(++i < bits.length) {
|
||||
while(++i < wordLength) {
|
||||
word = bits[i];
|
||||
if (word != 0) {
|
||||
return (i<<6) + Long.numberOfTrailingZeros(word);
|
||||
|
@ -211,12 +218,12 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
|
||||
/** this = this OR other */
|
||||
public void or(FixedBitSet other) {
|
||||
or(other.bits, other.bits.length);
|
||||
or(other.bits, other.wordLength);
|
||||
}
|
||||
|
||||
private void or(final long[] otherArr, final int otherLen) {
|
||||
final long[] thisArr = this.bits;
|
||||
int pos = Math.min(thisArr.length, otherLen);
|
||||
int pos = Math.min(wordLength, otherLen);
|
||||
while (--pos >= 0) {
|
||||
thisArr[pos] |= otherArr[pos];
|
||||
}
|
||||
|
@ -247,17 +254,17 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
|
||||
/** this = this AND other */
|
||||
public void and(FixedBitSet other) {
|
||||
and(other.bits, other.bits.length);
|
||||
and(other.bits, other.wordLength);
|
||||
}
|
||||
|
||||
private void and(final long[] otherArr, final int otherLen) {
|
||||
final long[] thisArr = this.bits;
|
||||
int pos = Math.min(thisArr.length, otherLen);
|
||||
int pos = Math.min(this.wordLength, otherLen);
|
||||
while(--pos >= 0) {
|
||||
thisArr[pos] &= otherArr[pos];
|
||||
}
|
||||
if (thisArr.length > otherLen) {
|
||||
Arrays.fill(thisArr, otherLen, thisArr.length, 0L);
|
||||
if (this.wordLength > otherLen) {
|
||||
Arrays.fill(thisArr, otherLen, this.wordLength, 0L);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -285,7 +292,7 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
|
||||
private void andNot(final long[] otherArr, final int otherLen) {
|
||||
final long[] thisArr = this.bits;
|
||||
int pos = Math.min(thisArr.length, otherLen);
|
||||
int pos = Math.min(this.wordLength, otherLen);
|
||||
while(--pos >= 0) {
|
||||
thisArr[pos] &= ~otherArr[pos];
|
||||
}
|
||||
|
@ -418,7 +425,7 @@ public final class FixedBitSet extends DocIdSet implements Bits {
|
|||
@Override
|
||||
public int hashCode() {
|
||||
long h = 0;
|
||||
for (int i = bits.length; --i>=0;) {
|
||||
for (int i = wordLength; --i>=0;) {
|
||||
h ^= bits[i];
|
||||
h = (h << 1) | (h >>> 63); // rotate left
|
||||
}
|
||||
|
|
|
@ -47,7 +47,14 @@ public enum Version {
|
|||
@Deprecated
|
||||
LUCENE_41,
|
||||
|
||||
/** Match settings and bugs in Lucene's 5.0 release.
|
||||
/**
|
||||
* Match settings and bugs in Lucene's 4.2 release.
|
||||
* @deprecated (5.0) Use latest
|
||||
*/
|
||||
@Deprecated
|
||||
LUCENE_42,
|
||||
|
||||
/** Match settings and bugs in Lucene's 5.0 release.
|
||||
* <p>
|
||||
* Use this to get the latest & greatest settings, bug
|
||||
* fixes, etc, for Lucene.
|
||||
|
|
|
@ -417,7 +417,7 @@ public final class FST<T> {
|
|||
cachedRootArcs = (Arc<T>[]) new Arc[0x80];
|
||||
final Arc<T> arc = new Arc<T>();
|
||||
getFirstArc(arc);
|
||||
final BytesReader in = getBytesReader(0);
|
||||
final BytesReader in = getBytesReader();
|
||||
if (targetHasArcs(arc)) {
|
||||
readFirstRealTargetArc(arc.target, arc, in);
|
||||
while(true) {
|
||||
|
@ -1246,22 +1246,12 @@ public final class FST<T> {
|
|||
/** Returns a {@link BytesReader} for this FST, positioned at
|
||||
* position 0. */
|
||||
public BytesReader getBytesReader() {
|
||||
return getBytesReader(0);
|
||||
}
|
||||
|
||||
/** Returns a {@link BytesReader} for this FST, positioned at
|
||||
* the provided position. */
|
||||
public BytesReader getBytesReader(long pos) {
|
||||
// TODO: maybe re-use via ThreadLocal?
|
||||
BytesReader in;
|
||||
if (packed) {
|
||||
in = bytes.getForwardReader();
|
||||
} else {
|
||||
in = bytes.getReverseReader();
|
||||
}
|
||||
if (pos != 0) {
|
||||
in.setPosition(pos);
|
||||
}
|
||||
return in;
|
||||
}
|
||||
|
||||
|
@ -1448,7 +1438,7 @@ public final class FST<T> {
|
|||
|
||||
Arc<T> arc = new Arc<T>();
|
||||
|
||||
final BytesReader r = getBytesReader(0);
|
||||
final BytesReader r = getBytesReader();
|
||||
|
||||
final int topN = Math.min(maxDerefNodes, inCounts.size());
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ abstract class FSTEnum<T> {
|
|||
* term before target. */
|
||||
protected FSTEnum(FST<T> fst) {
|
||||
this.fst = fst;
|
||||
fstReader = fst.getBytesReader(0);
|
||||
fstReader = fst.getBytesReader();
|
||||
NO_OUTPUT = fst.outputs.getNoOutput();
|
||||
fst.getFirstArc(getArc(0));
|
||||
output[0] = NO_OUTPUT;
|
||||
|
@ -145,7 +145,7 @@ abstract class FSTEnum<T> {
|
|||
// Arcs are fixed array -- use binary search to find
|
||||
// the target.
|
||||
|
||||
final FST.BytesReader in = fst.getBytesReader(0);
|
||||
final FST.BytesReader in = fst.getBytesReader();
|
||||
int low = arc.arcIdx;
|
||||
int high = arc.numArcs-1;
|
||||
int mid = 0;
|
||||
|
@ -284,7 +284,7 @@ abstract class FSTEnum<T> {
|
|||
// Arcs are fixed array -- use binary search to find
|
||||
// the target.
|
||||
|
||||
final FST.BytesReader in = fst.getBytesReader(0);
|
||||
final FST.BytesReader in = fst.getBytesReader();
|
||||
int low = arc.arcIdx;
|
||||
int high = arc.numArcs-1;
|
||||
int mid = 0;
|
||||
|
@ -434,7 +434,7 @@ abstract class FSTEnum<T> {
|
|||
FST.Arc<T> arc = getArc(upto-1);
|
||||
int targetLabel = getTargetLabel();
|
||||
|
||||
final FST.BytesReader fstReader = fst.getBytesReader(0);
|
||||
final FST.BytesReader fstReader = fst.getBytesReader();
|
||||
|
||||
while(true) {
|
||||
//System.out.println(" cycle target=" + (targetLabel == -1 ? "-1" : (char) targetLabel));
|
||||
|
|
|
@ -39,7 +39,7 @@ public final class Util {
|
|||
// TODO: would be nice not to alloc this on every lookup
|
||||
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
|
||||
|
||||
final BytesReader fstReader = fst.getBytesReader(0);
|
||||
final BytesReader fstReader = fst.getBytesReader();
|
||||
|
||||
// Accumulate output as we go
|
||||
T output = fst.outputs.getNoOutput();
|
||||
|
@ -64,7 +64,7 @@ public final class Util {
|
|||
public static<T> T get(FST<T> fst, BytesRef input) throws IOException {
|
||||
assert fst.inputType == FST.INPUT_TYPE.BYTE1;
|
||||
|
||||
final BytesReader fstReader = fst.getBytesReader(0);
|
||||
final BytesReader fstReader = fst.getBytesReader();
|
||||
|
||||
// TODO: would be nice not to alloc this on every lookup
|
||||
final FST.Arc<T> arc = fst.getFirstArc(new FST.Arc<T>());
|
||||
|
@ -101,7 +101,7 @@ public final class Util {
|
|||
* fit this. */
|
||||
public static IntsRef getByOutput(FST<Long> fst, long targetOutput) throws IOException {
|
||||
|
||||
final BytesReader in = fst.getBytesReader(0);
|
||||
final BytesReader in = fst.getBytesReader();
|
||||
|
||||
// TODO: would be nice not to alloc this on every lookup
|
||||
FST.Arc<Long> arc = fst.getFirstArc(new FST.Arc<Long>());
|
||||
|
@ -291,7 +291,7 @@ public final class Util {
|
|||
|
||||
public TopNSearcher(FST<T> fst, int topN, int maxQueueDepth, Comparator<T> comparator) {
|
||||
this.fst = fst;
|
||||
this.bytesReader = fst.getBytesReader(0);
|
||||
this.bytesReader = fst.getBytesReader();
|
||||
this.topN = topN;
|
||||
this.maxQueueDepth = maxQueueDepth;
|
||||
this.comparator = comparator;
|
||||
|
@ -380,7 +380,7 @@ public final class Util {
|
|||
|
||||
//System.out.println("search topN=" + topN);
|
||||
|
||||
final BytesReader fstReader = fst.getBytesReader(0);
|
||||
final BytesReader fstReader = fst.getBytesReader();
|
||||
final T NO_OUTPUT = fst.outputs.getNoOutput();
|
||||
|
||||
// TODO: we could enable FST to sorting arcs by weight
|
||||
|
@ -603,7 +603,7 @@ public final class Util {
|
|||
emitDotState(out, "initial", "point", "white", "");
|
||||
|
||||
final T NO_OUTPUT = fst.outputs.getNoOutput();
|
||||
final BytesReader r = fst.getBytesReader(0);
|
||||
final BytesReader r = fst.getBytesReader();
|
||||
|
||||
// final FST.Arc<T> scratchArc = new FST.Arc<T>();
|
||||
|
||||
|
|
|
@ -83,14 +83,22 @@ import org.junit.Ignore;
|
|||
@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
|
||||
public class TestBackwardsCompatibility extends LuceneTestCase {
|
||||
|
||||
// Uncomment these cases & run them on an older Lucene
|
||||
// version, to generate an index to test backwards
|
||||
// compatibility. Then, cd to build/test/index.cfs and
|
||||
// run "zip index.<VERSION>.cfs.zip *"; cd to
|
||||
// build/test/index.nocfs and run "zip
|
||||
// index.<VERSION>.nocfs.zip *". Then move those 2 zip
|
||||
// files to your trunk checkout and add them to the
|
||||
// oldNames array.
|
||||
// Uncomment these cases & run them on an older Lucene version,
|
||||
// to generate indexes to test backwards compatibility. These
|
||||
// indexes will be created under directory /tmp/idx/.
|
||||
//
|
||||
// However, you must first disable the Lucene TestSecurityManager,
|
||||
// which will otherwise disallow writing outside of the build/
|
||||
// directory - to do this, comment out the "java.security.manager"
|
||||
// <sysproperty> under the "test-macro" <macrodef>.
|
||||
//
|
||||
// Zip up the generated indexes:
|
||||
//
|
||||
// cd /tmp/idx/index.cfs ; zip index.<VERSION>.cfs.zip *
|
||||
// cd /tmp/idx/index.nocfs ; zip index.<VERSION>.nocfs.zip *
|
||||
//
|
||||
// Then move those 2 zip files to your trunk checkout and add them
|
||||
// to the oldNames array.
|
||||
|
||||
/*
|
||||
public void testCreateCFS() throws IOException {
|
||||
|
@ -151,6 +159,8 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
final static String[] oldNames = {"40.cfs",
|
||||
"40.nocfs",
|
||||
"41.cfs",
|
||||
"41.nocfs",
|
||||
};
|
||||
|
||||
final String[] unsupportedNames = {"19.cfs",
|
||||
|
@ -545,7 +555,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
|
||||
public File createIndex(String dirName, boolean doCFS, boolean fullyMerged) throws IOException {
|
||||
// we use a real directory name that is not cleaned up, because this method is only used to create backwards indexes:
|
||||
File indexDir = new File("/tmp/4x", dirName);
|
||||
File indexDir = new File("/tmp/idx", dirName);
|
||||
_TestUtil.rmDir(indexDir);
|
||||
Directory dir = newFSDirectory(indexDir);
|
||||
LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
|
||||
|
|
|
@ -355,16 +355,20 @@ public class TestIndexWriterWithThreads extends LuceneTestCase {
|
|||
StackTraceElement[] trace = new Exception().getStackTrace();
|
||||
boolean sawAbortOrFlushDoc = false;
|
||||
boolean sawClose = false;
|
||||
boolean sawMerge = false;
|
||||
for (int i = 0; i < trace.length; i++) {
|
||||
if ("abort".equals(trace[i].getMethodName()) ||
|
||||
"finishDocument".equals(trace[i].getMethodName())) {
|
||||
sawAbortOrFlushDoc = true;
|
||||
}
|
||||
if ("merge".equals(trace[i])) {
|
||||
sawMerge = true;
|
||||
}
|
||||
if ("close".equals(trace[i].getMethodName())) {
|
||||
sawClose = true;
|
||||
}
|
||||
}
|
||||
if (sawAbortOrFlushDoc && !sawClose) {
|
||||
if (sawAbortOrFlushDoc && !sawClose && !sawMerge) {
|
||||
if (onlyOnce)
|
||||
doFail = false;
|
||||
//System.out.println(Thread.currentThread().getName() + ": now fail");
|
||||
|
|
Binary file not shown.
Binary file not shown.
|
@ -32,6 +32,7 @@ import org.apache.lucene.index.IndexDocument;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.ThreadedIndexingAndSearchingTestCase;
|
||||
|
@ -294,6 +295,7 @@ public class TestNRTManager extends ThreadedIndexingAndSearchingTestCase {
|
|||
*/
|
||||
public void testThreadStarvationNoDeleteNRTReader() throws IOException, InterruptedException {
|
||||
IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
|
||||
conf.setMergePolicy(random().nextBoolean() ? NoMergePolicy.COMPOUND_FILES : NoMergePolicy.NO_COMPOUND_FILES);
|
||||
Directory d = newDirectory();
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final CountDownLatch signal = new CountDownLatch(1);
|
||||
|
|
|
@ -265,7 +265,18 @@ public class TestFixedBitSet extends LuceneTestCase {
|
|||
}
|
||||
|
||||
private FixedBitSet makeFixedBitSet(int[] a, int numBits) {
|
||||
FixedBitSet bs = new FixedBitSet(numBits);
|
||||
FixedBitSet bs;
|
||||
if (random().nextBoolean()) {
|
||||
int bits2words = FixedBitSet.bits2words(numBits);
|
||||
long[] words = new long[bits2words + random().nextInt(100)];
|
||||
for (int i = bits2words; i < words.length; i++) {
|
||||
words[i] = random().nextLong();
|
||||
}
|
||||
bs = new FixedBitSet(words, numBits);
|
||||
|
||||
} else {
|
||||
bs = new FixedBitSet(numBits);
|
||||
}
|
||||
for (int e: a) {
|
||||
bs.set(e);
|
||||
}
|
||||
|
@ -291,6 +302,23 @@ public class TestFixedBitSet extends LuceneTestCase {
|
|||
checkPrevSetBitArray(new int[] {0}, 1);
|
||||
checkPrevSetBitArray(new int[] {0,2}, 3);
|
||||
}
|
||||
|
||||
|
||||
private void checkNextSetBitArray(int [] a, int numBits) {
|
||||
FixedBitSet obs = makeFixedBitSet(a, numBits);
|
||||
BitSet bs = makeBitSet(a);
|
||||
doNextSetBit(bs, obs);
|
||||
}
|
||||
|
||||
public void testNextBitSet() {
|
||||
int[] setBits = new int[0+random().nextInt(1000)];
|
||||
for (int i = 0; i < setBits.length; i++) {
|
||||
setBits[i] = random().nextInt(setBits.length);
|
||||
}
|
||||
checkNextSetBitArray(setBits, setBits.length + random().nextInt(10));
|
||||
|
||||
checkNextSetBitArray(new int[0], setBits.length + random().nextInt(10));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -1033,7 +1033,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
throws IOException {
|
||||
if (FST.targetHasArcs(arc)) {
|
||||
int childCount = 0;
|
||||
BytesReader fstReader = fst.getBytesReader(0);
|
||||
BytesReader fstReader = fst.getBytesReader();
|
||||
for (arc = fst.readFirstTargetArc(arc, arc, fstReader);;
|
||||
arc = fst.readNextArc(arc, fstReader), childCount++)
|
||||
{
|
||||
|
@ -1168,12 +1168,12 @@ public class TestFSTs extends LuceneTestCase {
|
|||
assertEquals(nothing, startArc.nextFinalOutput);
|
||||
|
||||
FST.Arc<Long> arc = fst.readFirstTargetArc(startArc, new FST.Arc<Long>(),
|
||||
fst.getBytesReader(0));
|
||||
fst.getBytesReader());
|
||||
assertEquals('a', arc.label);
|
||||
assertEquals(17, arc.nextFinalOutput.longValue());
|
||||
assertTrue(arc.isFinal());
|
||||
|
||||
arc = fst.readNextArc(arc, fst.getBytesReader(0));
|
||||
arc = fst.readNextArc(arc, fst.getBytesReader());
|
||||
assertEquals('b', arc.label);
|
||||
assertFalse(arc.isFinal());
|
||||
assertEquals(42, arc.output.longValue());
|
||||
|
@ -1303,7 +1303,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
//Util.toDot(fst, w, false, false);
|
||||
//w.close();
|
||||
|
||||
BytesReader reader = fst.getBytesReader(0);
|
||||
BytesReader reader = fst.getBytesReader();
|
||||
|
||||
//System.out.println("testing: " + allPrefixes.size() + " prefixes");
|
||||
for (String prefix : allPrefixes) {
|
||||
|
@ -1424,7 +1424,7 @@ public class TestFSTs extends LuceneTestCase {
|
|||
//Util.toDot(fst, w, false, false);
|
||||
//w.close();
|
||||
|
||||
BytesReader reader = fst.getBytesReader(0);
|
||||
BytesReader reader = fst.getBytesReader();
|
||||
|
||||
//System.out.println("testing: " + allPrefixes.size() + " prefixes");
|
||||
for (String prefix : allPrefixes) {
|
||||
|
|
|
@ -30,19 +30,13 @@
|
|||
|
||||
<property name="examples.dir" location="src/examples"/>
|
||||
|
||||
<path id="classpath">
|
||||
<path refid="base.classpath" />
|
||||
<pathelement location="${build.dir}/classes/java" />
|
||||
<pathelement location="${build.dir}/classes/examples" />
|
||||
</path>
|
||||
|
||||
<path id="examples.classpath">
|
||||
<path refid="classpath" />
|
||||
<pathelement location="${build.dir}/classes/java" />
|
||||
<pathelement path="${analyzers-common.jar}" />
|
||||
</path>
|
||||
|
||||
<path id="test.classpath">
|
||||
<path id="test.classpath">
|
||||
<path refid="test.base.classpath" />
|
||||
<pathelement location="${build.dir}/classes/examples" />
|
||||
<!-- TODO, cut over tests to MockAnalyzer etc and nuke this dependency -->
|
||||
|
|
|
@ -90,8 +90,9 @@ public class TaxonomyMergeUtils {
|
|||
|
||||
DirectoryReader reader = DirectoryReader.open(srcIndexDir, -1);
|
||||
List<AtomicReaderContext> leaves = reader.leaves();
|
||||
AtomicReader wrappedLeaves[] = new AtomicReader[leaves.size()];
|
||||
for (int i = 0; i < leaves.size(); i++) {
|
||||
int numReaders = leaves.size();
|
||||
AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];
|
||||
for (int i = 0; i < numReaders; i++) {
|
||||
wrappedLeaves[i] = new OrdinalMappingAtomicReader(leaves.get(i).reader(), ordinalMap, params);
|
||||
}
|
||||
try {
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
|
|||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.RAMDirectory;
|
||||
|
||||
|
@ -80,12 +79,12 @@ public class MultiCLIndexer {
|
|||
// Initialize PerDimensionIndexingParams
|
||||
static {
|
||||
Map<CategoryPath, CategoryListParams> paramsMap = new HashMap<CategoryPath,CategoryListParams>();
|
||||
paramsMap.put(new CategoryPath("0"), new CategoryListParams(new Term("$Digits", "Zero")));
|
||||
paramsMap.put(new CategoryPath("1"), new CategoryListParams(new Term("$Digits", "One")));
|
||||
paramsMap.put(new CategoryPath("2"), new CategoryListParams(new Term("$Digits", "Two")));
|
||||
paramsMap.put(new CategoryPath("3"), new CategoryListParams(new Term("$Digits", "Three")));
|
||||
paramsMap.put(new CategoryPath("4"), new CategoryListParams(new Term("$Digits", "Four")));
|
||||
paramsMap.put(new CategoryPath("5"), new CategoryListParams(new Term("$Digits", "Five")));
|
||||
paramsMap.put(new CategoryPath("0"), new CategoryListParams("$Digits$Zero"));
|
||||
paramsMap.put(new CategoryPath("1"), new CategoryListParams("$Digits$One"));
|
||||
paramsMap.put(new CategoryPath("2"), new CategoryListParams("$Digits$Two"));
|
||||
paramsMap.put(new CategoryPath("3"), new CategoryListParams("$Digits$Three"));
|
||||
paramsMap.put(new CategoryPath("4"), new CategoryListParams("$Digits$Four"));
|
||||
paramsMap.put(new CategoryPath("5"), new CategoryListParams("$Digits$Five"));
|
||||
MULTI_IPARAMS = new PerDimensionIndexingParams(paramsMap);
|
||||
}
|
||||
|
||||
|
|
|
@ -114,7 +114,7 @@ public class AssociationsFacetFields extends FacetFields {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected FieldType fieldType() {
|
||||
protected FieldType drillDownFieldType() {
|
||||
return DRILL_DOWN_TYPE;
|
||||
}
|
||||
|
||||
|
|
|
@ -2,9 +2,8 @@ package org.apache.lucene.facet.associations;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.facet.search.PayloadIterator;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.store.ByteArrayDataInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
|
@ -30,25 +29,23 @@ import org.apache.lucene.util.BytesRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class AssociationsPayloadIterator<T extends CategoryAssociation> {
|
||||
public abstract class AssociationsIterator<T extends CategoryAssociation> {
|
||||
|
||||
private final PayloadIterator pi;
|
||||
private final T association;
|
||||
private final String dvField;
|
||||
private final BytesRef bytes = new BytesRef(32);
|
||||
|
||||
private BinaryDocValues current;
|
||||
|
||||
/**
|
||||
* Marking whether there are associations (at all) in the given index
|
||||
*/
|
||||
private boolean hasAssociations = false;
|
||||
|
||||
/**
|
||||
* Construct a new associations iterator. The given
|
||||
* {@link CategoryAssociation} is used to deserialize the association values.
|
||||
* It is assumed that all association values can be deserialized with the
|
||||
* given {@link CategoryAssociation}.
|
||||
*/
|
||||
public AssociationsPayloadIterator(String field, T association) throws IOException {
|
||||
pi = new PayloadIterator(new Term(field, association.getCategoryListID()));
|
||||
public AssociationsIterator(String field, T association) throws IOException {
|
||||
this.association = association;
|
||||
this.dvField = field + association.getCategoryListID();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -57,8 +54,8 @@ public abstract class AssociationsPayloadIterator<T extends CategoryAssociation>
|
|||
* of the documents belonging to the association given to the constructor.
|
||||
*/
|
||||
public final boolean setNextReader(AtomicReaderContext context) throws IOException {
|
||||
hasAssociations = pi.setNextReader(context);
|
||||
return hasAssociations;
|
||||
current = context.reader().getBinaryDocValues(dvField);
|
||||
return current != null;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -68,15 +65,11 @@ public abstract class AssociationsPayloadIterator<T extends CategoryAssociation>
|
|||
* extending classes.
|
||||
*/
|
||||
protected final boolean setNextDoc(int docID) throws IOException {
|
||||
if (!hasAssociations) { // there are no associations at all
|
||||
return false;
|
||||
current.get(docID, bytes);
|
||||
if (bytes.length == 0) {
|
||||
return false; // no associations for the requested document
|
||||
}
|
||||
|
||||
BytesRef bytes = pi.getPayload(docID);
|
||||
if (bytes == null) { // no associations for the requested document
|
||||
return false;
|
||||
}
|
||||
|
||||
ByteArrayDataInput in = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
|
||||
while (!in.eof()) {
|
||||
int ordinal = in.readInt();
|
|
@ -22,15 +22,18 @@ import org.apache.lucene.util.collections.IntToFloatMap;
|
|||
*/
|
||||
|
||||
/**
|
||||
* An {@link AssociationsPayloadIterator} over integer association values.
|
||||
* An {@link AssociationsIterator} over integer association values.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class FloatAssociationsPayloadIterator extends AssociationsPayloadIterator<CategoryFloatAssociation> {
|
||||
public class FloatAssociationsIterator extends AssociationsIterator<CategoryFloatAssociation> {
|
||||
|
||||
private final IntToFloatMap ordinalAssociations = new IntToFloatMap();
|
||||
|
||||
public FloatAssociationsPayloadIterator(String field, CategoryFloatAssociation association) throws IOException {
|
||||
|
||||
/**
|
||||
* Constructs a new {@link FloatAssociationsIterator}.
|
||||
*/
|
||||
public FloatAssociationsIterator(String field, CategoryFloatAssociation association) throws IOException {
|
||||
super(field, association);
|
||||
}
|
||||
|
|
@ -22,15 +22,18 @@ import org.apache.lucene.util.collections.IntToIntMap;
|
|||
*/
|
||||
|
||||
/**
|
||||
* An {@link AssociationsPayloadIterator} over integer association values.
|
||||
* An {@link AssociationsIterator} over integer association values.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class IntAssociationsPayloadIterator extends AssociationsPayloadIterator<CategoryIntAssociation> {
|
||||
public class IntAssociationsIterator extends AssociationsIterator<CategoryIntAssociation> {
|
||||
|
||||
private final IntToIntMap ordinalAssociations = new IntToIntMap();
|
||||
|
||||
public IntAssociationsPayloadIterator(String field, CategoryIntAssociation association) throws IOException {
|
||||
/**
|
||||
* Constructs a new {@link IntAssociationsIterator}.
|
||||
*/
|
||||
public IntAssociationsIterator(String field, CategoryIntAssociation association) throws IOException {
|
||||
super(field, association);
|
||||
}
|
||||
|
|
@ -56,10 +56,9 @@ public class CountingListBuilder implements CategoryListBuilder {
|
|||
private static final class NoPartitionsOrdinalsEncoder extends OrdinalsEncoder {
|
||||
|
||||
private final IntEncoder encoder;
|
||||
private final String name;
|
||||
private final String name = "";
|
||||
|
||||
NoPartitionsOrdinalsEncoder(CategoryListParams categoryListParams) {
|
||||
name = categoryListParams.getTerm().text();
|
||||
encoder = categoryListParams.createEncoder();
|
||||
}
|
||||
|
||||
|
@ -91,7 +90,7 @@ public class CountingListBuilder implements CategoryListBuilder {
|
|||
final HashMap<String,IntsRef> partitionOrdinals = new HashMap<String,IntsRef>();
|
||||
for (int i = 0; i < ordinals.length; i++) {
|
||||
int ordinal = ordinals.ints[i];
|
||||
final String name = PartitionsUtils.partitionNameByOrdinal(indexingParams, categoryListParams, ordinal);
|
||||
final String name = PartitionsUtils.partitionNameByOrdinal(indexingParams, ordinal);
|
||||
IntsRef partitionOrds = partitionOrdinals.get(name);
|
||||
if (partitionOrds == null) {
|
||||
partitionOrds = new IntsRef(32);
|
||||
|
|
|
@ -4,17 +4,14 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StraightBytesDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
|
@ -51,32 +48,6 @@ import org.apache.lucene.util.IntsRef;
|
|||
*/
|
||||
public class FacetFields {
|
||||
|
||||
// a TokenStream for writing the counting list payload
|
||||
private static final class CountingListStream extends TokenStream {
|
||||
private final PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private Iterator<Entry<String,BytesRef>> categoriesData;
|
||||
|
||||
CountingListStream() {}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (!categoriesData.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Entry<String,BytesRef> entry = categoriesData.next();
|
||||
termAtt.setEmpty().append(entry.getKey());
|
||||
payloadAtt.setPayload(entry.getValue());
|
||||
return true;
|
||||
}
|
||||
|
||||
void setCategoriesData(Map<String,BytesRef> categoriesData) {
|
||||
this.categoriesData = categoriesData.entrySet().iterator();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// The counting list is written in a payload, but we don't store it
|
||||
// nor need norms.
|
||||
private static final FieldType COUNTING_LIST_PAYLOAD_TYPE = new FieldType();
|
||||
|
@ -94,9 +65,7 @@ public class FacetFields {
|
|||
// Therefore we set its IndexOptions to DOCS_ONLY.
|
||||
private static final FieldType DRILL_DOWN_TYPE = new FieldType(TextField.TYPE_NOT_STORED);
|
||||
static {
|
||||
// TODO: once we cutover to DocValues, we can set it to DOCS_ONLY for this
|
||||
// FacetFields (not associations)
|
||||
DRILL_DOWN_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
DRILL_DOWN_TYPE.setIndexOptions(IndexOptions.DOCS_ONLY);
|
||||
DRILL_DOWN_TYPE.freeze();
|
||||
}
|
||||
|
||||
|
@ -175,10 +144,20 @@ public class FacetFields {
|
|||
* Returns the {@link FieldType} with which the drill-down terms should be
|
||||
* indexed. The default is {@link IndexOptions#DOCS_ONLY}.
|
||||
*/
|
||||
protected FieldType fieldType() {
|
||||
protected FieldType drillDownFieldType() {
|
||||
return DRILL_DOWN_TYPE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the counting list data to the document under the given field. Note that
|
||||
* the field is determined by the {@link CategoryListParams}.
|
||||
*/
|
||||
protected void addCountingListData(Document doc, Map<String,BytesRef> categoriesData, String field) {
|
||||
for (Entry<String,BytesRef> entry : categoriesData.entrySet()) {
|
||||
doc.add(new StraightBytesDocValuesField(field + entry.getKey(), entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
/** Adds the needed facet fields to the document. */
|
||||
public void addFields(Document doc, Iterable<CategoryPath> categories) throws IOException {
|
||||
if (categories == null) {
|
||||
|
@ -198,7 +177,7 @@ public class FacetFields {
|
|||
IntsRef ordinals = new IntsRef(32); // should be enough for most common applications
|
||||
for (Entry<CategoryListParams, Iterable<CategoryPath>> e : categoryLists.entrySet()) {
|
||||
final CategoryListParams clp = e.getKey();
|
||||
final String field = clp.getTerm().field();
|
||||
final String field = clp.field;
|
||||
|
||||
// build category list data
|
||||
ordinals.length = 0; // reset
|
||||
|
@ -214,13 +193,11 @@ public class FacetFields {
|
|||
Map<String,BytesRef> categoriesData = getCategoryListData(clp, ordinals, e.getValue());
|
||||
|
||||
// add the counting list data
|
||||
CountingListStream ts = new CountingListStream();
|
||||
ts.setCategoriesData(categoriesData);
|
||||
doc.add(new Field(field, ts, COUNTING_LIST_PAYLOAD_TYPE));
|
||||
addCountingListData(doc, categoriesData, field);
|
||||
|
||||
// add the drill-down field
|
||||
DrillDownStream drillDownStream = getDrillDownStream(e.getValue());
|
||||
Field drillDown = new Field(field, drillDownStream, fieldType());
|
||||
Field drillDown = new Field(field, drillDownStream, drillDownFieldType());
|
||||
doc.add(drillDown);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,258 @@
|
|||
package org.apache.lucene.facet.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 java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
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.FilterAtomicReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* A {@link FilterAtomicReader} for migrating a facets index which encodes
|
||||
* category ordinals in a payload to {@link DocValues}. To migrate the index,
|
||||
* you should build a mapping from a field (String) to term ({@link Term}),
|
||||
* which denotes under which DocValues field to put the data encoded in the
|
||||
* matching term's payload. You can follow the code example below to migrate an
|
||||
* existing index:
|
||||
*
|
||||
* <pre class="prettyprint">
|
||||
* // Add the index and migrate payload to DocValues on the go
|
||||
* DirectoryReader reader = DirectoryReader.open(oldDir);
|
||||
* IndexWriterConfig conf = new IndexWriterConfig(VER, ANALYZER);
|
||||
* IndexWriter writer = new IndexWriter(newDir, conf);
|
||||
* List<AtomicReaderContext> leaves = reader.leaves();
|
||||
* AtomicReader wrappedLeaves[] = new AtomicReader[leaves.size()];
|
||||
* for (int i = 0; i < leaves.size(); i++) {
|
||||
* wrappedLeaves[i] = new FacetPayloadMigrationReader(leaves.get(i).reader(),
|
||||
* fieldTerms);
|
||||
* }
|
||||
* writer.addIndexes(new MultiReader(wrappedLeaves));
|
||||
* writer.commit();
|
||||
* </pre>
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> to build the field-to-term map you can use
|
||||
* {@link #buildFieldTermsMap(Directory, FacetIndexingParams)}, as long as the
|
||||
* index to migrate contains the ordinals payload under
|
||||
* {@link #PAYLOAD_TERM_TEXT}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class FacetsPayloadMigrationReader extends FilterAtomicReader {
|
||||
|
||||
private class PayloadMigratingDocValues extends DocValues {
|
||||
|
||||
private final DocsAndPositionsEnum dpe;
|
||||
|
||||
public PayloadMigratingDocValues(DocsAndPositionsEnum dpe) {
|
||||
this.dpe = dpe;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Source loadDirectSource() throws IOException {
|
||||
return new PayloadMigratingSource(getType(), dpe);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Source loadSource() throws IOException {
|
||||
throw new UnsupportedOperationException("in-memory Source is not supported by this reader");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type getType() {
|
||||
return Type.BYTES_VAR_STRAIGHT;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class PayloadMigratingSource extends Source {
|
||||
|
||||
private final DocsAndPositionsEnum dpe;
|
||||
private int curDocID;
|
||||
|
||||
protected PayloadMigratingSource(Type type, DocsAndPositionsEnum dpe) {
|
||||
super(type);
|
||||
this.dpe = dpe;
|
||||
if (dpe == null) {
|
||||
curDocID = DocIdSetIterator.NO_MORE_DOCS;
|
||||
} else {
|
||||
try {
|
||||
curDocID = dpe.nextDoc();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
if (curDocID > docID) {
|
||||
// document does not exist
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
|
||||
try {
|
||||
if (curDocID < docID) {
|
||||
curDocID = dpe.advance(docID);
|
||||
if (curDocID != docID) { // requested document does not have a payload
|
||||
ref.length = 0;
|
||||
return ref;
|
||||
}
|
||||
}
|
||||
|
||||
// we're on the document
|
||||
dpe.nextPosition();
|
||||
ref.copyBytes(dpe.getPayload());
|
||||
return ref;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/** The {@link Term} text of the ordinals payload. */
|
||||
public static final String PAYLOAD_TERM_TEXT = "$fulltree$";
|
||||
|
||||
/**
|
||||
* A utility method for building the field-to-Term map, given the
|
||||
* {@link FacetIndexingParams} and the directory of the index to migrate. The
|
||||
* map that will be built will correspond to partitions as well as multiple
|
||||
* {@link CategoryListParams}.
|
||||
* <p>
|
||||
* <b>NOTE:</b> since {@link CategoryListParams} no longer define a
|
||||
* {@link Term}, this method assumes that the term used by the different
|
||||
* {@link CategoryListParams} is {@link #PAYLOAD_TERM_TEXT}. If this is not
|
||||
* the case, then you should build the map yourself, using the terms in your
|
||||
* index.
|
||||
*/
|
||||
public static Map<String,Term> buildFieldTermsMap(Directory dir, FacetIndexingParams fip) throws IOException {
|
||||
// only add field-Term mapping that will actually have DocValues in the end.
|
||||
// therefore traverse the index terms and add what exists. this pertains to
|
||||
// multiple CLPs, as well as partitions
|
||||
DirectoryReader reader = DirectoryReader.open(dir);
|
||||
final Map<String,Term> fieldTerms = new HashMap<String,Term>();
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
for (CategoryListParams clp : fip.getAllCategoryListParams()) {
|
||||
Terms terms = context.reader().terms(clp.field);
|
||||
if (terms != null) {
|
||||
TermsEnum te = terms.iterator(null);
|
||||
BytesRef termBytes = null;
|
||||
while ((termBytes = te.next()) != null) {
|
||||
String term = termBytes.utf8ToString();
|
||||
if (term.startsWith(PAYLOAD_TERM_TEXT )) {
|
||||
if (term.equals(PAYLOAD_TERM_TEXT)) {
|
||||
fieldTerms.put(clp.field, new Term(clp.field, term));
|
||||
} else {
|
||||
fieldTerms.put(clp.field + term.substring(PAYLOAD_TERM_TEXT.length()), new Term(clp.field, term));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
return fieldTerms;
|
||||
}
|
||||
|
||||
private final Map<String,Term> fieldTerms;
|
||||
|
||||
/**
|
||||
* Wraps an {@link AtomicReader} and migrates the payload to {@link DocValues}
|
||||
* fields by using the given mapping.
|
||||
*/
|
||||
public FacetsPayloadMigrationReader(AtomicReader in, Map<String,Term> fieldTerms) {
|
||||
super(in);
|
||||
this.fieldTerms = fieldTerms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
Term term = fieldTerms.get(field);
|
||||
if (term == null) {
|
||||
return super.docValues(field);
|
||||
} else {
|
||||
DocsAndPositionsEnum dpe = null;
|
||||
Fields fields = fields();
|
||||
if (fields != null) {
|
||||
Terms terms = fields.terms(term.field());
|
||||
if (terms != null) {
|
||||
TermsEnum te = terms.iterator(null); // no use for reusing
|
||||
if (te.seekExact(term.bytes(), true)) {
|
||||
// we're not expected to be called for deleted documents
|
||||
dpe = te.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
|
||||
}
|
||||
}
|
||||
}
|
||||
// we shouldn't return null, even if the term does not exist or has no
|
||||
// payloads, since we already marked the field as having DocValues.
|
||||
return new PayloadMigratingDocValues(dpe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FieldInfos getFieldInfos() {
|
||||
FieldInfos innerInfos = super.getFieldInfos();
|
||||
ArrayList<FieldInfo> infos = new ArrayList<FieldInfo>(innerInfos.size());
|
||||
// if there are partitions, then the source index contains one field for all their terms
|
||||
// while with DocValues, we simulate that by multiple fields.
|
||||
HashSet<String> leftoverFields = new HashSet<String>(fieldTerms.keySet());
|
||||
int number = -1;
|
||||
for (FieldInfo info : innerInfos) {
|
||||
if (fieldTerms.containsKey(info.name)) {
|
||||
// mark this field as having a DocValues
|
||||
infos.add(new FieldInfo(info.name, true, info.number,
|
||||
info.hasVectors(), info.omitsNorms(), info.hasPayloads(),
|
||||
info.getIndexOptions(), Type.BYTES_VAR_STRAIGHT,
|
||||
info.getNormType(), info.attributes()));
|
||||
leftoverFields.remove(info.name);
|
||||
} else {
|
||||
infos.add(info);
|
||||
}
|
||||
number = Math.max(number, info.number);
|
||||
}
|
||||
for (String field : leftoverFields) {
|
||||
infos.add(new FieldInfo(field, false, ++number, false, false, false,
|
||||
null, Type.BYTES_VAR_STRAIGHT, null, null));
|
||||
}
|
||||
return new FieldInfos(infos.toArray(new FieldInfo[infos.size()]));
|
||||
}
|
||||
|
||||
}
|
|
@ -25,13 +25,10 @@ import org.apache.lucene.facet.index.params.CategoryListParams;
|
|||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.FilterAtomicReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.encoding.IntDecoder;
|
||||
|
@ -41,8 +38,8 @@ import org.apache.lucene.util.encoding.IntEncoder;
|
|||
* A {@link FilterAtomicReader} for updating facets ordinal references,
|
||||
* based on an ordinal map. You should use this code in conjunction with merging
|
||||
* taxonomies - after you merge taxonomies, you receive an {@link OrdinalMap}
|
||||
* which maps the 'old' payloads to the 'new' ones. You can use that map to
|
||||
* re-map the payloads which contain the facets information (ordinals) either
|
||||
* which maps the 'old' ordinals to the 'new' ones. You can use that map to
|
||||
* re-map the doc values which contain the facets information (ordinals) either
|
||||
* before or while merging the indexes.
|
||||
* <p>
|
||||
* For re-mapping the ordinals during index merge, do the following:
|
||||
|
@ -70,9 +67,8 @@ import org.apache.lucene.util.encoding.IntEncoder;
|
|||
public class OrdinalMappingAtomicReader extends FilterAtomicReader {
|
||||
|
||||
private final int[] ordinalMap;
|
||||
// a little obtuse: but we dont need to create Term objects this way
|
||||
private final Map<String,Map<BytesRef,CategoryListParams>> termMap =
|
||||
new HashMap<String,Map<BytesRef,CategoryListParams>>(1);
|
||||
|
||||
private final Map<String,CategoryListParams> dvFieldMap = new HashMap<String,CategoryListParams>();
|
||||
|
||||
/**
|
||||
* Wraps an AtomicReader, mapping ordinals according to the ordinalMap.
|
||||
|
@ -91,125 +87,85 @@ public class OrdinalMappingAtomicReader extends FilterAtomicReader {
|
|||
super(in);
|
||||
this.ordinalMap = ordinalMap;
|
||||
for (CategoryListParams params: indexingParams.getAllCategoryListParams()) {
|
||||
Term term = params.getTerm();
|
||||
Map<BytesRef,CategoryListParams> fieldMap = termMap.get(term.field());
|
||||
if (fieldMap == null) {
|
||||
fieldMap = new HashMap<BytesRef,CategoryListParams>(1);
|
||||
termMap.put(term.field(), fieldMap);
|
||||
}
|
||||
fieldMap.put(term.bytes(), params);
|
||||
dvFieldMap.put(params.field, params);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields getTermVectors(int docID) throws IOException {
|
||||
Fields fields = super.getTermVectors(docID);
|
||||
if (fields == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new OrdinalMappingFields(fields);
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
DocValues inner = super.docValues(field);
|
||||
if (inner == null) {
|
||||
return inner;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Fields fields() throws IOException {
|
||||
Fields fields = super.fields();
|
||||
if (fields == null) {
|
||||
return null;
|
||||
} else {
|
||||
return new OrdinalMappingFields(fields);
|
||||
}
|
||||
}
|
||||
|
||||
private class OrdinalMappingFields extends FilterFields {
|
||||
|
||||
public OrdinalMappingFields(Fields in) {
|
||||
super(in);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Terms terms(String field) throws IOException {
|
||||
Terms terms = super.terms(field);
|
||||
if (terms == null) {
|
||||
return terms;
|
||||
}
|
||||
Map<BytesRef,CategoryListParams> termsMap = termMap.get(field);
|
||||
if (termsMap == null) {
|
||||
return terms;
|
||||
} else {
|
||||
return new OrdinalMappingTerms(terms, termsMap);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class OrdinalMappingTerms extends FilterTerms {
|
||||
private final Map<BytesRef,CategoryListParams> termsMap;
|
||||
|
||||
public OrdinalMappingTerms(Terms in, Map<BytesRef,CategoryListParams> termsMap) {
|
||||
super(in);
|
||||
this.termsMap = termsMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TermsEnum iterator(TermsEnum reuse) throws IOException {
|
||||
// TODO: should we reuse the inner termsenum?
|
||||
return new OrdinalMappingTermsEnum(super.iterator(reuse), termsMap);
|
||||
CategoryListParams clp = dvFieldMap.get(field);
|
||||
if (clp == null) {
|
||||
return inner;
|
||||
} else {
|
||||
return new OrdinalMappingDocValues(inner, clp);
|
||||
}
|
||||
}
|
||||
|
||||
private class OrdinalMappingTermsEnum extends FilterTermsEnum {
|
||||
private final Map<BytesRef,CategoryListParams> termsMap;
|
||||
private class OrdinalMappingDocValues extends DocValues {
|
||||
|
||||
private final CategoryListParams clp;
|
||||
private final DocValues delegate;
|
||||
|
||||
public OrdinalMappingTermsEnum(TermsEnum in, Map<BytesRef,CategoryListParams> termsMap) {
|
||||
super(in);
|
||||
this.termsMap = termsMap;
|
||||
public OrdinalMappingDocValues(DocValues delegate, CategoryListParams clp) {
|
||||
this.delegate = delegate;
|
||||
this.clp = clp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
|
||||
// TODO: we could reuse our D&P enum if we need
|
||||
DocsAndPositionsEnum inner = super.docsAndPositions(liveDocs, reuse, flags);
|
||||
if (inner == null) {
|
||||
return inner;
|
||||
}
|
||||
|
||||
CategoryListParams params = termsMap.get(term());
|
||||
if (params == null) {
|
||||
return inner;
|
||||
}
|
||||
|
||||
return new OrdinalMappingDocsAndPositionsEnum(inner, params);
|
||||
protected Source loadSource() throws IOException {
|
||||
return new OrdinalMappingSource(getType(), clp, delegate.getSource());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Source loadDirectSource() throws IOException {
|
||||
return new OrdinalMappingSource(getType(), clp, delegate.getDirectSource());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type getType() {
|
||||
return Type.BYTES_VAR_STRAIGHT;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private class OrdinalMappingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
|
||||
private class OrdinalMappingSource extends Source {
|
||||
|
||||
private final IntEncoder encoder;
|
||||
private final IntDecoder decoder;
|
||||
private final IntsRef ordinals = new IntsRef(32);
|
||||
private final BytesRef payloadOut = new BytesRef();
|
||||
|
||||
public OrdinalMappingDocsAndPositionsEnum(DocsAndPositionsEnum in, CategoryListParams params) {
|
||||
super(in);
|
||||
encoder = params.createEncoder();
|
||||
private final Source delegate;
|
||||
|
||||
protected OrdinalMappingSource(Type type, CategoryListParams clp, Source delegate) {
|
||||
super(type);
|
||||
this.delegate = delegate;
|
||||
encoder = clp.createEncoder();
|
||||
decoder = encoder.createMatchingDecoder();
|
||||
}
|
||||
|
||||
|
||||
@SuppressWarnings("synthetic-access")
|
||||
@Override
|
||||
public BytesRef getPayload() throws IOException {
|
||||
BytesRef payload = super.getPayload();
|
||||
if (payload == null) {
|
||||
return payload;
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref = delegate.getBytes(docID, ref);
|
||||
if (ref == null || ref.length == 0) {
|
||||
return ref;
|
||||
} else {
|
||||
decoder.decode(payload, ordinals);
|
||||
decoder.decode(ref, ordinals);
|
||||
|
||||
// map the ordinals
|
||||
for (int i = 0; i < ordinals.length; i++) {
|
||||
ordinals.ints[i] = ordinalMap[ordinals.ints[i]];
|
||||
}
|
||||
|
||||
encoder.encode(ordinals, payloadOut);
|
||||
return payloadOut;
|
||||
encoder.encode(ordinals, ref);
|
||||
return ref;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -4,15 +4,13 @@ import java.io.IOException;
|
|||
import java.io.Serializable;
|
||||
|
||||
import org.apache.lucene.facet.search.CategoryListIterator;
|
||||
import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
|
||||
import org.apache.lucene.facet.search.DocValuesCategoryListIterator;
|
||||
import org.apache.lucene.facet.util.PartitionsUtils;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.encoding.DGapIntEncoder;
|
||||
import org.apache.lucene.util.encoding.DGapVInt8IntEncoder;
|
||||
import org.apache.lucene.util.encoding.IntDecoder;
|
||||
import org.apache.lucene.util.encoding.IntEncoder;
|
||||
import org.apache.lucene.util.encoding.SortingIntEncoder;
|
||||
import org.apache.lucene.util.encoding.UniqueValuesIntEncoder;
|
||||
import org.apache.lucene.util.encoding.VInt8IntEncoder;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -38,39 +36,26 @@ import org.apache.lucene.util.encoding.VInt8IntEncoder;
|
|||
*/
|
||||
public class CategoryListParams implements Serializable {
|
||||
|
||||
/** The default term used to store the facets information. */
|
||||
public static final Term DEFAULT_TERM = new Term("$facets", "$fulltree$");
|
||||
/** The default field used to store the facets information. */
|
||||
public static final String DEFAULT_FIELD = "$facets";
|
||||
|
||||
private final Term term;
|
||||
public final String field;
|
||||
|
||||
private final int hashCode;
|
||||
|
||||
/**
|
||||
* Constructs a default category list parameters object, using
|
||||
* {@link #DEFAULT_TERM}.
|
||||
*/
|
||||
/** Constructs a default category list parameters object, using {@link #DEFAULT_FIELD}. */
|
||||
public CategoryListParams() {
|
||||
this(DEFAULT_TERM);
|
||||
this(DEFAULT_FIELD);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a category list parameters object, using the given {@link Term}.
|
||||
* @param term who's payload hold the category-list.
|
||||
*/
|
||||
public CategoryListParams(Term term) {
|
||||
this.term = term;
|
||||
/** Constructs a category list parameters object, using the given field. */
|
||||
public CategoryListParams(String field) {
|
||||
this.field = field;
|
||||
// Pre-compute the hashCode because these objects are immutable. Saves
|
||||
// some time on the comparisons later.
|
||||
this.hashCode = term.hashCode();
|
||||
this.hashCode = field.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* A {@link Term} who's payload holds the category-list.
|
||||
*/
|
||||
public final Term getTerm() {
|
||||
return term;
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows to override how categories are encoded and decoded. A matching
|
||||
* {@link IntDecoder} is provided by the {@link IntEncoder}.
|
||||
|
@ -92,7 +77,7 @@ public class CategoryListParams implements Serializable {
|
|||
* counting facets.
|
||||
*/
|
||||
public IntEncoder createEncoder() {
|
||||
return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
|
||||
return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapVInt8IntEncoder()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -110,7 +95,7 @@ public class CategoryListParams implements Serializable {
|
|||
// The above hashcodes might equal each other in the case of a collision,
|
||||
// so at this point only directly term equality testing will settle
|
||||
// the equality test.
|
||||
return this.term.equals(other.term);
|
||||
return field.equals(other.field);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -120,9 +105,9 @@ public class CategoryListParams implements Serializable {
|
|||
|
||||
/** Create the {@link CategoryListIterator} for the specified partition. */
|
||||
public CategoryListIterator createCategoryListIterator(int partition) throws IOException {
|
||||
String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
|
||||
Term payloadTerm = new Term(term.field(), categoryListTermStr);
|
||||
return new PayloadCategoryListIteraor(payloadTerm, createEncoder().createMatchingDecoder());
|
||||
String categoryListTermStr = PartitionsUtils.partitionName(partition);
|
||||
String docValuesField = field + categoryListTermStr;
|
||||
return new DocValuesCategoryListIterator(docValuesField, createEncoder().createMatchingDecoder());
|
||||
}
|
||||
|
||||
}
|
|
@ -3,7 +3,7 @@ package org.apache.lucene.facet.search;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.BinaryDocValues;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.encoding.IntDecoder;
|
||||
|
@ -25,57 +25,62 @@ import org.apache.lucene.util.encoding.IntDecoder;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/**
|
||||
* A {@link CategoryListIterator} which reads the category ordinals from a
|
||||
* payload.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class PayloadCategoryListIteraor implements CategoryListIterator {
|
||||
|
||||
private final IntDecoder decoder;
|
||||
private final Term term;
|
||||
private final PayloadIterator pi;
|
||||
private final int hashCode;
|
||||
/** A {@link CategoryListIterator} which reads the ordinals from a {@link BinaryDocValues}. */
|
||||
public class DocValuesCategoryListIterator implements CategoryListIterator {
|
||||
|
||||
public PayloadCategoryListIteraor(Term term, IntDecoder decoder) throws IOException {
|
||||
pi = new PayloadIterator(term);
|
||||
private final IntDecoder decoder;
|
||||
private final String field;
|
||||
private final int hashCode;
|
||||
private final BytesRef bytes = new BytesRef(32);
|
||||
|
||||
private BinaryDocValues current;
|
||||
|
||||
/**
|
||||
* Constructs a new {@link DocValuesCategoryListIterator}.
|
||||
*/
|
||||
public DocValuesCategoryListIterator(String field, IntDecoder decoder) {
|
||||
this.field = field;
|
||||
this.decoder = decoder;
|
||||
hashCode = term.hashCode();
|
||||
this.term = term;
|
||||
this.hashCode = field.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (!(other instanceof PayloadCategoryListIteraor)) {
|
||||
return false;
|
||||
}
|
||||
PayloadCategoryListIteraor that = (PayloadCategoryListIteraor) other;
|
||||
if (hashCode != that.hashCode) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// Hash codes are the same, check equals() to avoid cases of hash-collisions.
|
||||
return term.equals(that.term);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return hashCode;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (!(o instanceof DocValuesCategoryListIterator)) {
|
||||
return false;
|
||||
}
|
||||
DocValuesCategoryListIterator other = (DocValuesCategoryListIterator) o;
|
||||
if (hashCode != other.hashCode) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// Hash codes are the same, check equals() to avoid cases of hash-collisions.
|
||||
return field.equals(other.field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean setNextReader(AtomicReaderContext context) throws IOException {
|
||||
return pi.setNextReader(context);
|
||||
current = context.reader().getBinaryDocValues(field);
|
||||
return current != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getOrdinals(int docID, IntsRef ints) throws IOException {
|
||||
current.get(docID, bytes);
|
||||
ints.length = 0;
|
||||
BytesRef payload = pi.getPayload(docID);
|
||||
if (payload != null) {
|
||||
decoder.decode(payload, ints);
|
||||
if (bytes.length > 0) {
|
||||
decoder.decode(bytes, ints);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return field;
|
||||
}
|
||||
|
||||
}
|
|
@ -55,7 +55,7 @@ public final class DrillDown {
|
|||
CategoryListParams clp = iParams.getCategoryListParams(path);
|
||||
char[] buffer = new char[path.fullPathLength()];
|
||||
iParams.drillDownTermText(path, buffer);
|
||||
return new Term(clp.getTerm().field(), String.valueOf(buffer));
|
||||
return new Term(clp.field, String.valueOf(buffer));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,114 +0,0 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocsAndPositionsEnum;
|
||||
import org.apache.lucene.index.Fields;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* A utility class for iterating through a posting list of a given term and
|
||||
* retrieving the payload of the first position in every document. For
|
||||
* efficiency, this class does not check if documents passed to
|
||||
* {@link #getPayload(int)} are deleted, since it is usually used to iterate on
|
||||
* payloads of documents that matched a query. If you need to skip over deleted
|
||||
* documents, you should do so before calling {@link #getPayload(int)}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class PayloadIterator {
|
||||
|
||||
private TermsEnum reuseTE;
|
||||
private DocsAndPositionsEnum dpe;
|
||||
private boolean hasMore;
|
||||
private int curDocID;
|
||||
|
||||
private final Term term;
|
||||
|
||||
public PayloadIterator(Term term) throws IOException {
|
||||
this.term = term;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the {@link AtomicReaderContext} for which {@link #getPayload(int)}
|
||||
* calls will be made. Returns true iff this reader has payload for any of the
|
||||
* documents belonging to the {@link Term} given to the constructor.
|
||||
*/
|
||||
public boolean setNextReader(AtomicReaderContext context) throws IOException {
|
||||
hasMore = false;
|
||||
Fields fields = context.reader().fields();
|
||||
if (fields != null) {
|
||||
Terms terms = fields.terms(term.field());
|
||||
if (terms != null) {
|
||||
reuseTE = terms.iterator(reuseTE);
|
||||
if (reuseTE.seekExact(term.bytes(), true)) {
|
||||
// this class is usually used to iterate on whatever a Query matched
|
||||
// if it didn't match deleted documents, we won't receive them. if it
|
||||
// did, we should iterate on them too, therefore we pass liveDocs=null
|
||||
dpe = reuseTE.docsAndPositions(null, dpe, DocsAndPositionsEnum.FLAG_PAYLOADS);
|
||||
if (dpe != null && (curDocID = dpe.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
hasMore = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return hasMore;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link BytesRef payload} of the given document, or {@code null}
|
||||
* if the document does not exist, there are no more documents in the posting
|
||||
* list, or the document exists but has not payload. The given document IDs
|
||||
* are treated as local to the reader given to
|
||||
* {@link #setNextReader(AtomicReaderContext)}.
|
||||
*/
|
||||
public BytesRef getPayload(int docID) throws IOException {
|
||||
if (!hasMore) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (curDocID > docID) {
|
||||
// document does not exist
|
||||
return null;
|
||||
}
|
||||
|
||||
if (curDocID < docID) {
|
||||
curDocID = dpe.advance(docID);
|
||||
if (curDocID != docID) { // requested document does not have a payload
|
||||
if (curDocID == DocIdSetIterator.NO_MORE_DOCS) { // no more docs in this reader
|
||||
hasMore = false;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
// we're on the document
|
||||
assert dpe.freq() == 1 : "expecting freq=1 (got " + dpe.freq() + ") term=" + term + " doc=" + curDocID;
|
||||
int pos = dpe.nextPosition();
|
||||
assert pos != -1 : "no positions for term=" + term + " doc=" + curDocID;
|
||||
return dpe.getPayload();
|
||||
}
|
||||
|
||||
}
|
|
@ -10,6 +10,7 @@ import java.util.Map.Entry;
|
|||
import java.util.logging.Level;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.facet.search.params.FacetRequest;
|
||||
import org.apache.lucene.facet.search.params.FacetSearchParams;
|
||||
|
@ -110,9 +111,8 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
|
||||
if (isUsingComplements) {
|
||||
try {
|
||||
totalFacetCounts = TotalFacetCountsCache.getSingleton()
|
||||
.getTotalCounts(indexReader, taxonomyReader,
|
||||
searchParams.getFacetIndexingParams(), searchParams.getCategoryListCache());
|
||||
totalFacetCounts = TotalFacetCountsCache.getSingleton().getTotalCounts(indexReader, taxonomyReader,
|
||||
searchParams.getFacetIndexingParams());
|
||||
if (totalFacetCounts != null) {
|
||||
docids = ScoredDocIdsUtils.getComplementSet(docids, indexReader);
|
||||
} else {
|
||||
|
@ -242,20 +242,29 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
int maxDoc = -1;
|
||||
while (iterator.next()) {
|
||||
int docID = iterator.getDocID();
|
||||
while (docID >= maxDoc) { // find the segment which contains this document
|
||||
if (!contexts.hasNext()) {
|
||||
throw new RuntimeException("ScoredDocIDs contains documents outside this reader's segments !?");
|
||||
}
|
||||
current = contexts.next();
|
||||
maxDoc = current.docBase + current.reader().maxDoc();
|
||||
if (docID < maxDoc) { // segment has docs, check if it has categories
|
||||
boolean validSegment = categoryListIter.setNextReader(current);
|
||||
validSegment &= aggregator.setNextReader(current);
|
||||
if (!validSegment) { // if categoryList or aggregtor say it's an invalid segment, skip all docs
|
||||
while (docID < maxDoc && iterator.next()) {
|
||||
docID = iterator.getDocID();
|
||||
if (docID >= maxDoc) {
|
||||
boolean iteratorDone = false;
|
||||
do { // find the segment which contains this document
|
||||
if (!contexts.hasNext()) {
|
||||
throw new RuntimeException("ScoredDocIDs contains documents outside this reader's segments !?");
|
||||
}
|
||||
current = contexts.next();
|
||||
maxDoc = current.docBase + current.reader().maxDoc();
|
||||
if (docID < maxDoc) { // segment has docs, check if it has categories
|
||||
boolean validSegment = categoryListIter.setNextReader(current);
|
||||
validSegment &= aggregator.setNextReader(current);
|
||||
if (!validSegment) { // if categoryList or aggregtor say it's an invalid segment, skip all docs
|
||||
while (docID < maxDoc && iterator.next()) {
|
||||
docID = iterator.getDocID();
|
||||
}
|
||||
if (docID < maxDoc) {
|
||||
iteratorDone = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} while (docID >= maxDoc);
|
||||
if (iteratorDone) { // iterator finished, terminate the loop
|
||||
break;
|
||||
}
|
||||
}
|
||||
docID -= current.docBase;
|
||||
|
@ -312,19 +321,17 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
|
|||
|
||||
HashMap<CategoryListIterator, Aggregator> categoryLists = new HashMap<CategoryListIterator, Aggregator>();
|
||||
|
||||
FacetIndexingParams indexingParams = searchParams.getFacetIndexingParams();
|
||||
for (FacetRequest facetRequest : searchParams.getFacetRequests()) {
|
||||
Aggregator categoryAggregator = facetRequest.createAggregator(
|
||||
isUsingComplements, facetArrays, taxonomyReader);
|
||||
Aggregator categoryAggregator = facetRequest.createAggregator(isUsingComplements, facetArrays, taxonomyReader);
|
||||
|
||||
CategoryListIterator cli = facetRequest.createCategoryListIterator(taxonomyReader, searchParams, partition);
|
||||
CategoryListIterator cli = indexingParams.getCategoryListParams(facetRequest.categoryPath).createCategoryListIterator(partition);
|
||||
|
||||
// get the aggregator
|
||||
Aggregator old = categoryLists.put(cli, categoryAggregator);
|
||||
|
||||
if (old != null && !old.equals(categoryAggregator)) {
|
||||
// TODO (Facet): create a more meaningful RE class, and throw it.
|
||||
throw new RuntimeException(
|
||||
"Overriding existing category list with different aggregator. THAT'S A NO NO!");
|
||||
throw new RuntimeException("Overriding existing category list with different aggregator");
|
||||
}
|
||||
// if the aggregator is the same we're covered
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ public class TopKFacetResultsHandler extends FacetResultsHandler {
|
|||
public IntermediateFacetResult fetchPartitionResult(FacetArrays facetArrays, int offset)
|
||||
throws IOException {
|
||||
TopKFacetResult res = null;
|
||||
int ordinal = taxonomyReader.getOrdinal(facetRequest.getCategoryPath());
|
||||
int ordinal = taxonomyReader.getOrdinal(facetRequest.categoryPath);
|
||||
if (ordinal != TaxonomyReader.INVALID_ORDINAL) {
|
||||
double value = 0;
|
||||
if (isSelfPartition(ordinal, facetArrays, offset)) {
|
||||
|
@ -79,7 +79,7 @@ public class TopKFacetResultsHandler extends FacetResultsHandler {
|
|||
@Override
|
||||
public IntermediateFacetResult mergeResults(IntermediateFacetResult... tmpResults) throws IOException {
|
||||
|
||||
int ordinal = taxonomyReader.getOrdinal(facetRequest.getCategoryPath());
|
||||
int ordinal = taxonomyReader.getOrdinal(facetRequest.categoryPath);
|
||||
MutableFacetResultNode resNode = new MutableFacetResultNode(ordinal, 0);
|
||||
|
||||
int totalFacets = 0;
|
||||
|
|
|
@ -34,35 +34,37 @@ import org.apache.lucene.util.collections.IntToObjectMap;
|
|||
*/
|
||||
|
||||
/**
|
||||
* Generates {@link FacetResult} from the count arrays aggregated for a particular
|
||||
* {@link FacetRequest}.
|
||||
* The generated {@link FacetResult} is a subtree of the taxonomy tree.
|
||||
* Its root node, {@link FacetResult#getFacetResultNode()},
|
||||
* is the facet specified by {@link FacetRequest#getCategoryPath()},
|
||||
* and the enumerated children, {@link FacetResultNode#getSubResults()}, of each node in that
|
||||
* {@link FacetResult} are the top K ( = {@link FacetRequest#getNumResults()}) among its children
|
||||
* in the taxonomy.
|
||||
* Top in the sense {@link FacetRequest#getSortBy()},
|
||||
* which can be by the values aggregated in the count arrays, or by ordinal numbers;
|
||||
* also specified is the sort order, {@link FacetRequest#getSortOrder()},
|
||||
* ascending or descending, of these values or ordinals before their top K are selected.
|
||||
* The depth (number of levels excluding the root) of the
|
||||
* {@link FacetResult} tree is specified by {@link FacetRequest#getDepth()}.
|
||||
* Generates {@link FacetResult} from the count arrays aggregated for a
|
||||
* particular {@link FacetRequest}. The generated {@link FacetResult} is a
|
||||
* subtree of the taxonomy tree. Its root node,
|
||||
* {@link FacetResult#getFacetResultNode()}, is the facet specified by
|
||||
* {@link FacetRequest#categoryPath}, and the enumerated children,
|
||||
* {@link FacetResultNode#getSubResults()}, of each node in that
|
||||
* {@link FacetResult} are the top K ( = {@link FacetRequest#getNumResults()})
|
||||
* among its children in the taxonomy. Top in the sense
|
||||
* {@link FacetRequest#getSortBy()}, which can be by the values aggregated in
|
||||
* the count arrays, or by ordinal numbers; also specified is the sort order,
|
||||
* {@link FacetRequest#getSortOrder()}, ascending or descending, of these values
|
||||
* or ordinals before their top K are selected. The depth (number of levels
|
||||
* excluding the root) of the {@link FacetResult} tree is specified by
|
||||
* {@link FacetRequest#getDepth()}.
|
||||
* <p>
|
||||
* Because the number of selected children of each node is restricted,
|
||||
* and not the overall number of nodes in the {@link FacetResult}, facets not selected
|
||||
* Because the number of selected children of each node is restricted, and not
|
||||
* the overall number of nodes in the {@link FacetResult}, facets not selected
|
||||
* into {@link FacetResult} might have better values, or ordinals, (typically,
|
||||
* higher counts), than facets that are selected into the {@link FacetResult}.
|
||||
* <p>
|
||||
* The generated {@link FacetResult} also provides with
|
||||
* {@link FacetResult#getNumValidDescendants()}, which returns the total number of facets
|
||||
* that are descendants of the root node, no deeper than {@link FacetRequest#getDepth()}, and
|
||||
* which have valid value. The rootnode itself is not counted here.
|
||||
* Valid value is determined by the {@link FacetResultsHandler}.
|
||||
* {@link TopKInEachNodeHandler} defines valid as != 0.
|
||||
* The generated {@link FacetResult} also provides with
|
||||
* {@link FacetResult#getNumValidDescendants()}, which returns the total number
|
||||
* of facets that are descendants of the root node, no deeper than
|
||||
* {@link FacetRequest#getDepth()}, and which have valid value. The rootnode
|
||||
* itself is not counted here. Valid value is determined by the
|
||||
* {@link FacetResultsHandler}. {@link TopKInEachNodeHandler} defines valid as
|
||||
* != 0.
|
||||
* <p>
|
||||
* <b>NOTE:</b> this code relies on the assumption that {@link TaxonomyReader#INVALID_ORDINAL} == -1, a smaller
|
||||
* value than any valid ordinal.
|
||||
* <b>NOTE:</b> this code relies on the assumption that
|
||||
* {@link TaxonomyReader#INVALID_ORDINAL} == -1, a smaller value than any valid
|
||||
* ordinal.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
@ -109,7 +111,7 @@ public class TopKInEachNodeHandler extends FacetResultsHandler {
|
|||
|
||||
// get the root of the result tree to be returned, and the depth of that result tree
|
||||
// (depth means number of node levels excluding the root).
|
||||
int rootNode = this.taxonomyReader.getOrdinal(this.facetRequest.getCategoryPath());
|
||||
int rootNode = this.taxonomyReader.getOrdinal(facetRequest.categoryPath);
|
||||
if (rootNode == TaxonomyReader.INVALID_ORDINAL) {
|
||||
return null;
|
||||
}
|
||||
|
@ -767,7 +769,7 @@ public class TopKInEachNodeHandler extends FacetResultsHandler {
|
|||
@Override
|
||||
public FacetResult renderFacetResult(IntermediateFacetResult tmpResult) throws IOException {
|
||||
IntermediateFacetResultWithHash tmp = (IntermediateFacetResultWithHash) tmpResult;
|
||||
int ordinal = this.taxonomyReader.getOrdinal(this.facetRequest.getCategoryPath());
|
||||
int ordinal = this.taxonomyReader.getOrdinal(this.facetRequest.categoryPath);
|
||||
if ((tmp == null) || (ordinal == TaxonomyReader.INVALID_ORDINAL)) {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@ import org.apache.lucene.facet.index.params.CategoryListParams;
|
|||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.facet.search.aggregator.CountingAggregator;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListCache;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListData;
|
||||
import org.apache.lucene.facet.search.params.CountFacetRequest;
|
||||
import org.apache.lucene.facet.search.params.FacetRequest;
|
||||
import org.apache.lucene.facet.search.params.FacetSearchParams;
|
||||
|
@ -155,9 +153,8 @@ public class TotalFacetCounts {
|
|||
private static final List<FacetRequest> DUMMY_REQ = Arrays.asList(
|
||||
new FacetRequest[] { new CountFacetRequest(CategoryPath.EMPTY, 1) });
|
||||
|
||||
static TotalFacetCounts compute(final IndexReader indexReader,
|
||||
final TaxonomyReader taxonomy, final FacetIndexingParams facetIndexingParams,
|
||||
final CategoryListCache clCache) throws IOException {
|
||||
static TotalFacetCounts compute(final IndexReader indexReader, final TaxonomyReader taxonomy,
|
||||
final FacetIndexingParams facetIndexingParams) throws IOException {
|
||||
int partitionSize = PartitionsUtils.partitionSize(facetIndexingParams, taxonomy);
|
||||
final int[][] counts = new int[(int) Math.ceil(taxonomy.getSize() /(float) partitionSize)][partitionSize];
|
||||
FacetSearchParams newSearchParams = new FacetSearchParams(DUMMY_REQ, facetIndexingParams);
|
||||
|
@ -170,8 +167,7 @@ public class TotalFacetCounts {
|
|||
Aggregator aggregator = new CountingAggregator(counts[partition]);
|
||||
HashMap<CategoryListIterator, Aggregator> map = new HashMap<CategoryListIterator, Aggregator>();
|
||||
for (CategoryListParams clp: facetIndexingParams.getAllCategoryListParams()) {
|
||||
final CategoryListIterator cli = clIteraor(clCache, clp, partition);
|
||||
map.put(cli, aggregator);
|
||||
map.put(clp.createCategoryListIterator(partition), aggregator);
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
@ -181,14 +177,4 @@ public class TotalFacetCounts {
|
|||
return new TotalFacetCounts(taxonomy, facetIndexingParams, counts, CreationType.Computed);
|
||||
}
|
||||
|
||||
static CategoryListIterator clIteraor(CategoryListCache clCache, CategoryListParams clp, int partition)
|
||||
throws IOException {
|
||||
if (clCache != null) {
|
||||
CategoryListData cld = clCache.get(clp);
|
||||
if (cld != null) {
|
||||
return cld.iterator(partition);
|
||||
}
|
||||
}
|
||||
return clp.createCategoryListIterator(partition);
|
||||
}
|
||||
}
|
|
@ -7,12 +7,10 @@ import java.util.LinkedHashMap;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListCache;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -80,16 +78,20 @@ public final class TotalFacetCountsCache {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the total facet counts for a reader/taxonomy pair and facet indexing parameters.
|
||||
* If not in cache, computed here and added to the cache for later use.
|
||||
* @param indexReader the documents index
|
||||
* @param taxonomy the taxonomy index
|
||||
* @param facetIndexingParams facet indexing parameters
|
||||
* @param clCache category list cache for faster computation, can be null
|
||||
* Get the total facet counts for a reader/taxonomy pair and facet indexing
|
||||
* parameters. If not in cache, computed here and added to the cache for later
|
||||
* use.
|
||||
*
|
||||
* @param indexReader
|
||||
* the documents index
|
||||
* @param taxonomy
|
||||
* the taxonomy index
|
||||
* @param facetIndexingParams
|
||||
* facet indexing parameters
|
||||
* @return the total facet counts.
|
||||
*/
|
||||
public TotalFacetCounts getTotalCounts(IndexReader indexReader, TaxonomyReader taxonomy,
|
||||
FacetIndexingParams facetIndexingParams, CategoryListCache clCache) throws IOException {
|
||||
FacetIndexingParams facetIndexingParams) throws IOException {
|
||||
// create the key
|
||||
TFCKey key = new TFCKey(indexReader, taxonomy, facetIndexingParams);
|
||||
// it is important that this call is not synchronized, so that available TFC
|
||||
|
@ -99,7 +101,7 @@ public final class TotalFacetCountsCache {
|
|||
markRecentlyUsed(key);
|
||||
return tfc;
|
||||
}
|
||||
return computeAndCache(key, clCache);
|
||||
return computeAndCache(key);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -149,10 +151,10 @@ public final class TotalFacetCountsCache {
|
|||
* matter this method is synchronized, which is not too bad, because there is
|
||||
* lots of work done in the computations.
|
||||
*/
|
||||
private synchronized TotalFacetCounts computeAndCache(TFCKey key, CategoryListCache clCache) throws IOException {
|
||||
private synchronized TotalFacetCounts computeAndCache(TFCKey key) throws IOException {
|
||||
TotalFacetCounts tfc = cache.get(key);
|
||||
if (tfc == null) {
|
||||
tfc = TotalFacetCounts.compute(key.indexReader, key.taxonomy, key.facetIndexingParams, clCache);
|
||||
tfc = TotalFacetCounts.compute(key.indexReader, key.taxonomy, key.facetIndexingParams);
|
||||
lruKeys.add(key);
|
||||
cache.put(key,tfc);
|
||||
trimCache();
|
||||
|
@ -161,16 +163,22 @@ public final class TotalFacetCountsCache {
|
|||
}
|
||||
|
||||
/**
|
||||
* Load {@link TotalFacetCounts} matching input parameters from the provided outputFile
|
||||
* and add them into the cache for the provided indexReader, taxonomy, and facetIndexingParams.
|
||||
* If a {@link TotalFacetCounts} for these parameters already exists in the cache, it will be
|
||||
* replaced by the loaded one.
|
||||
* @param inputFile file from which to read the data
|
||||
* @param indexReader the documents index
|
||||
* @param taxonomy the taxonomy index
|
||||
* @param facetIndexingParams the facet indexing parameters
|
||||
* @throws IOException on error
|
||||
* @see #store(File, IndexReader, TaxonomyReader, FacetIndexingParams, CategoryListCache)
|
||||
* Load {@link TotalFacetCounts} matching input parameters from the provided
|
||||
* outputFile and add them into the cache for the provided indexReader,
|
||||
* taxonomy, and facetIndexingParams. If a {@link TotalFacetCounts} for these
|
||||
* parameters already exists in the cache, it will be replaced by the loaded
|
||||
* one.
|
||||
*
|
||||
* @param inputFile
|
||||
* file from which to read the data
|
||||
* @param indexReader
|
||||
* the documents index
|
||||
* @param taxonomy
|
||||
* the taxonomy index
|
||||
* @param facetIndexingParams
|
||||
* the facet indexing parameters
|
||||
* @throws IOException
|
||||
* on error
|
||||
*/
|
||||
public synchronized void load(File inputFile, IndexReader indexReader, TaxonomyReader taxonomy,
|
||||
FacetIndexingParams facetIndexingParams) throws IOException {
|
||||
|
@ -185,21 +193,27 @@ public final class TotalFacetCountsCache {
|
|||
}
|
||||
|
||||
/**
|
||||
* Store the {@link TotalFacetCounts} matching input parameters into the provided outputFile,
|
||||
* making them available for a later call to {@link #load(File, IndexReader, TaxonomyReader, FacetIndexingParams)}.
|
||||
* If these {@link TotalFacetCounts} are available in the cache, they are used. But if they are
|
||||
* not in the cache, this call will first compute them (which will also add them to the cache).
|
||||
* @param outputFile file to store in.
|
||||
* @param indexReader the documents index
|
||||
* @param taxonomy the taxonomy index
|
||||
* @param facetIndexingParams the facet indexing parameters
|
||||
* @param clCache category list cache for faster computation, can be null
|
||||
* @throws IOException on error
|
||||
* Store the {@link TotalFacetCounts} matching input parameters into the
|
||||
* provided outputFile, making them available for a later call to
|
||||
* {@link #load(File, IndexReader, TaxonomyReader, FacetIndexingParams)}. If
|
||||
* these {@link TotalFacetCounts} are available in the cache, they are used.
|
||||
* But if they are not in the cache, this call will first compute them (which
|
||||
* will also add them to the cache).
|
||||
*
|
||||
* @param outputFile
|
||||
* file to store in.
|
||||
* @param indexReader
|
||||
* the documents index
|
||||
* @param taxonomy
|
||||
* the taxonomy index
|
||||
* @param facetIndexingParams
|
||||
* the facet indexing parameters
|
||||
* @throws IOException
|
||||
* on error
|
||||
* @see #load(File, IndexReader, TaxonomyReader, FacetIndexingParams)
|
||||
* @see #getTotalCounts(IndexReader, TaxonomyReader, FacetIndexingParams, CategoryListCache)
|
||||
*/
|
||||
public void store(File outputFile, IndexReader indexReader, TaxonomyReader taxonomy,
|
||||
FacetIndexingParams facetIndexingParams, CategoryListCache clCache) throws IOException {
|
||||
FacetIndexingParams facetIndexingParams) throws IOException {
|
||||
File parentFile = outputFile.getParentFile();
|
||||
if (
|
||||
( outputFile.exists() && (!outputFile.isFile() || !outputFile.canWrite())) ||
|
||||
|
@ -207,7 +221,7 @@ public final class TotalFacetCountsCache {
|
|||
) {
|
||||
throw new IllegalArgumentException("Exepecting a writable file: "+outputFile);
|
||||
}
|
||||
TotalFacetCounts tfc = getTotalCounts(indexReader, taxonomy, facetIndexingParams, clCache);
|
||||
TotalFacetCounts tfc = getTotalCounts(indexReader, taxonomy, facetIndexingParams);
|
||||
TotalFacetCounts.storeToFile(outputFile, tfc);
|
||||
}
|
||||
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.apache.lucene.facet.search.aggregator.associations;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.facet.associations.CategoryFloatAssociation;
|
||||
import org.apache.lucene.facet.associations.FloatAssociationsPayloadIterator;
|
||||
import org.apache.lucene.facet.associations.FloatAssociationsIterator;
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
|
@ -37,15 +37,15 @@ public class AssociationFloatSumAggregator implements Aggregator {
|
|||
|
||||
protected final String field;
|
||||
protected final float[] sumArray;
|
||||
protected final FloatAssociationsPayloadIterator associations;
|
||||
protected final FloatAssociationsIterator associations;
|
||||
|
||||
public AssociationFloatSumAggregator(float[] sumArray) throws IOException {
|
||||
this(CategoryListParams.DEFAULT_TERM.field(), sumArray);
|
||||
this(CategoryListParams.DEFAULT_FIELD, sumArray);
|
||||
}
|
||||
|
||||
public AssociationFloatSumAggregator(String field, float[] sumArray) throws IOException {
|
||||
this.field = field;
|
||||
associations = new FloatAssociationsPayloadIterator(field, new CategoryFloatAssociation());
|
||||
associations = new FloatAssociationsIterator(field, new CategoryFloatAssociation());
|
||||
this.sumArray = sumArray;
|
||||
}
|
||||
|
||||
|
|
|
@ -3,7 +3,7 @@ package org.apache.lucene.facet.search.aggregator.associations;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.facet.associations.CategoryIntAssociation;
|
||||
import org.apache.lucene.facet.associations.IntAssociationsPayloadIterator;
|
||||
import org.apache.lucene.facet.associations.IntAssociationsIterator;
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
|
@ -37,15 +37,15 @@ public class AssociationIntSumAggregator implements Aggregator {
|
|||
|
||||
protected final String field;
|
||||
protected final int[] sumArray;
|
||||
protected final IntAssociationsPayloadIterator associations;
|
||||
protected final IntAssociationsIterator associations;
|
||||
|
||||
public AssociationIntSumAggregator(int[] sumArray) throws IOException {
|
||||
this(CategoryListParams.DEFAULT_TERM.field(), sumArray);
|
||||
this(CategoryListParams.DEFAULT_FIELD, sumArray);
|
||||
}
|
||||
|
||||
public AssociationIntSumAggregator(String field, int[] sumArray) throws IOException {
|
||||
this.field = field;
|
||||
associations = new IntAssociationsPayloadIterator(field, new CategoryIntAssociation());
|
||||
associations = new IntAssociationsIterator(field, new CategoryIntAssociation());
|
||||
this.sumArray = sumArray;
|
||||
}
|
||||
|
||||
|
|
|
@ -1,61 +0,0 @@
|
|||
package org.apache.lucene.facet.search.cache;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Cache for {@link CategoryListData}, per {@link CategoryListParams}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class CategoryListCache {
|
||||
|
||||
private HashMap<CategoryListParams, CategoryListData>
|
||||
cldMap = new HashMap<CategoryListParams,CategoryListData>();
|
||||
|
||||
/**
|
||||
* Fetch the cached {@link CategoryListData} for a given {@link CategoryListParams}.
|
||||
*/
|
||||
public CategoryListData get(CategoryListParams clp) {
|
||||
return cldMap.get(clp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Register a pre-computed {@link CategoryListData}.
|
||||
*/
|
||||
public void register(CategoryListParams clp, CategoryListData clData) {
|
||||
cldMap.put(clp,clData);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load and register {@link CategoryListData}.
|
||||
*/
|
||||
public void loadAndRegister(CategoryListParams clp,
|
||||
IndexReader reader, TaxonomyReader taxo, FacetIndexingParams iparams) throws IOException {
|
||||
CategoryListData clData = new CategoryListData(reader, taxo, iparams, clp);
|
||||
register(clp,clData);
|
||||
}
|
||||
}
|
|
@ -1,133 +0,0 @@
|
|||
package org.apache.lucene.facet.search.cache;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.CategoryListIterator;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Category list data maintained in RAM.
|
||||
* <p>
|
||||
* Speeds up facets accumulation when more RAM is available.
|
||||
* <p>
|
||||
* Note that this will consume more memory: one int (4 bytes) for each category
|
||||
* of each document.
|
||||
* <p>
|
||||
* Note: at the moment this class is insensitive to updates of the index, and,
|
||||
* in particular, does not make use of Lucene's ability to refresh a single
|
||||
* segment.
|
||||
* <p>
|
||||
* See {@link CategoryListCache#register(CategoryListParams, CategoryListData)}
|
||||
* and
|
||||
* {@link CategoryListCache#loadAndRegister(CategoryListParams, IndexReader, TaxonomyReader, FacetIndexingParams)}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class CategoryListData {
|
||||
|
||||
// TODO (Facet): experiment with different orders - p-d-c vs. current d-p-c.
|
||||
private transient volatile int[][][] docPartitionCategories;
|
||||
|
||||
/**
|
||||
* Empty constructor for extensions with modified computation of the data.
|
||||
*/
|
||||
protected CategoryListData() {
|
||||
}
|
||||
|
||||
/** Compute category list data for caching for faster iteration. */
|
||||
CategoryListData(IndexReader reader, TaxonomyReader taxo, FacetIndexingParams iparams, CategoryListParams clp)
|
||||
throws IOException {
|
||||
|
||||
int[][][]dpf = new int[reader.maxDoc()][][];
|
||||
int numPartitions = (int)Math.ceil(taxo.getSize()/(double)iparams.getPartitionSize());
|
||||
IntsRef ordinals = new IntsRef(32);
|
||||
for (int part = 0; part < numPartitions; part++) {
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
CategoryListIterator cli = clp.createCategoryListIterator(part);
|
||||
if (cli.setNextReader(context)) {
|
||||
final int maxDoc = context.reader().maxDoc();
|
||||
for (int i = 0; i < maxDoc; i++) {
|
||||
cli.getOrdinals(i, ordinals);
|
||||
if (ordinals.length > 0) {
|
||||
int doc = i + context.docBase;
|
||||
if (dpf[doc] == null) {
|
||||
dpf[doc] = new int[numPartitions][];
|
||||
}
|
||||
if (dpf[doc][part] == null) {
|
||||
dpf[doc][part] = new int[ordinals.length];
|
||||
}
|
||||
for (int j = 0; j < ordinals.length; j++) {
|
||||
dpf[doc][part][j] = ordinals.ints[j];
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
docPartitionCategories = dpf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate on the category list data for the specified partition.
|
||||
*/
|
||||
public CategoryListIterator iterator(int partition) throws IOException {
|
||||
return new RAMCategoryListIterator(partition, docPartitionCategories);
|
||||
}
|
||||
|
||||
/** Internal: category list iterator over uncompressed category info in RAM */
|
||||
private static class RAMCategoryListIterator implements CategoryListIterator {
|
||||
|
||||
private int docBase;
|
||||
private final int part;
|
||||
private final int[][][] dpc;
|
||||
|
||||
RAMCategoryListIterator(int part, int[][][] docPartitionCategories) {
|
||||
this.part = part;
|
||||
dpc = docPartitionCategories;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean setNextReader(AtomicReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
return dpc != null && dpc.length > part;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getOrdinals(int docID, IntsRef ints) throws IOException {
|
||||
ints.length = 0;
|
||||
docID += docBase;
|
||||
if (dpc.length > docID && dpc[docID] != null && dpc[docID][part] != null) {
|
||||
if (ints.ints.length < dpc[docID][part].length) {
|
||||
ints.grow(dpc[docID][part].length);
|
||||
}
|
||||
ints.length = 0;
|
||||
for (int i = 0; i < dpc[docID][part].length; i++) {
|
||||
ints.ints[ints.length++] = dpc[docID][part][i];
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -1,22 +0,0 @@
|
|||
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
<html><head></head>
|
||||
<body>
|
||||
Caching to speed up facets accumulation.
|
||||
</body>
|
||||
</html>
|
|
@ -2,15 +2,11 @@ package org.apache.lucene.facet.search.params;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.search.CategoryListIterator;
|
||||
import org.apache.lucene.facet.search.FacetArrays;
|
||||
import org.apache.lucene.facet.search.FacetResultsHandler;
|
||||
import org.apache.lucene.facet.search.TopKFacetResultsHandler;
|
||||
import org.apache.lucene.facet.search.TopKInEachNodeHandler;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListCache;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListData;
|
||||
import org.apache.lucene.facet.taxonomy.CategoryPath;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
|
||||
|
@ -64,7 +60,7 @@ public abstract class FacetRequest implements Cloneable {
|
|||
*/
|
||||
public static final ResultMode DEFAULT_RESULT_MODE = ResultMode.PER_NODE_IN_TREE;
|
||||
|
||||
private final CategoryPath categoryPath;
|
||||
public final CategoryPath categoryPath;
|
||||
private final int numResults;
|
||||
private int numLabel;
|
||||
private int depth;
|
||||
|
@ -133,17 +129,6 @@ public abstract class FacetRequest implements Cloneable {
|
|||
this.sortBy = sortBy;
|
||||
}
|
||||
|
||||
/**
|
||||
* The root category of this facet request. The categories that are returned
|
||||
* as a result of this request will all be descendants of this root.
|
||||
* <p>
|
||||
* <b>NOTE:</b> you should not modify the returned {@link CategoryPath}, or
|
||||
* otherwise some methonds may not work properly, e.g. {@link #hashCode()}.
|
||||
*/
|
||||
public final CategoryPath getCategoryPath() {
|
||||
return categoryPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* How deeply to look under the given category. If the depth is 0,
|
||||
* only the category itself is counted. If the depth is 1, its immediate
|
||||
|
@ -160,24 +145,22 @@ public abstract class FacetRequest implements Cloneable {
|
|||
* will have their category paths calculated, and the rest will only be
|
||||
* available as ordinals (category numbers) and will have null paths.
|
||||
* <P>
|
||||
* If Integer.MAX_VALUE is specified, all
|
||||
* results are labled.
|
||||
* If Integer.MAX_VALUE is specified, all results are labled.
|
||||
* <P>
|
||||
* The purpose of this parameter is to avoid having to run the whole
|
||||
* faceted search again when the user asks for more values for the facet;
|
||||
* The application can ask (getNumResults()) for more values than it needs
|
||||
* to show, but keep getNumLabel() only the number it wants to immediately
|
||||
* show. The slow-down caused by finding more values is negligible, because
|
||||
* the slowest part - finding the categories' paths, is avoided.
|
||||
* The purpose of this parameter is to avoid having to run the whole faceted
|
||||
* search again when the user asks for more values for the facet; The
|
||||
* application can ask (getNumResults()) for more values than it needs to
|
||||
* show, but keep getNumLabel() only the number it wants to immediately show.
|
||||
* The slow-down caused by finding more values is negligible, because the
|
||||
* slowest part - finding the categories' paths, is avoided.
|
||||
* <p>
|
||||
* Depending on the {@link #getResultMode() LimitsMode},
|
||||
* this limit is applied globally or per results node.
|
||||
* In the global mode, if this limit is 3,
|
||||
* only 3 top results would be labeled.
|
||||
* In the per-node mode, if this limit is 3,
|
||||
* 3 top children of {@link #getCategoryPath() the target category} would be labeled,
|
||||
* as well as 3 top children of each of them, and so forth, until the depth defined
|
||||
* by {@link #getDepth()}.
|
||||
* Depending on the {@link #getResultMode() LimitsMode}, this limit is applied
|
||||
* globally or per results node. In the global mode, if this limit is 3, only
|
||||
* 3 top results would be labeled. In the per-node mode, if this limit is 3, 3
|
||||
* top children of {@link #categoryPath the target category} would be labeled,
|
||||
* as well as 3 top children of each of them, and so forth, until the depth
|
||||
* defined by {@link #getDepth()}.
|
||||
*
|
||||
* @see #getResultMode()
|
||||
*/
|
||||
public final int getNumLabel() {
|
||||
|
@ -185,20 +168,18 @@ public abstract class FacetRequest implements Cloneable {
|
|||
}
|
||||
|
||||
/**
|
||||
* The number of sub-categories to return (at most).
|
||||
* If the sub-categories are returned.
|
||||
* The number of sub-categories to return (at most). If the sub-categories are
|
||||
* returned.
|
||||
* <p>
|
||||
* If Integer.MAX_VALUE is specified, all
|
||||
* sub-categories are returned.
|
||||
* If Integer.MAX_VALUE is specified, all sub-categories are returned.
|
||||
* <p>
|
||||
* Depending on the {@link #getResultMode() LimitsMode},
|
||||
* this limit is applied globally or per results node.
|
||||
* In the global mode, if this limit is 3,
|
||||
* only 3 top results would be computed.
|
||||
* In the per-node mode, if this limit is 3,
|
||||
* 3 top children of {@link #getCategoryPath() the target category} would be returned,
|
||||
* as well as 3 top children of each of them, and so forth, until the depth defined
|
||||
* by {@link #getDepth()}.
|
||||
* Depending on the {@link #getResultMode() LimitsMode}, this limit is applied
|
||||
* globally or per results node. In the global mode, if this limit is 3, only
|
||||
* 3 top results would be computed. In the per-node mode, if this limit is 3,
|
||||
* 3 top children of {@link #categoryPath the target category} would be
|
||||
* returned, as well as 3 top children of each of them, and so forth, until
|
||||
* the depth defined by {@link #getDepth()}.
|
||||
*
|
||||
* @see #getResultMode()
|
||||
*/
|
||||
public final int getNumResults() {
|
||||
|
@ -319,24 +300,6 @@ public abstract class FacetRequest implements Cloneable {
|
|||
public abstract Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Create the category list iterator for the specified partition. If a non
|
||||
* null cache is provided which contains the required data, use it for the
|
||||
* iteration.
|
||||
*/
|
||||
public CategoryListIterator createCategoryListIterator(TaxonomyReader taxo, FacetSearchParams sParams, int partition)
|
||||
throws IOException {
|
||||
CategoryListCache clCache = sParams.getCategoryListCache();
|
||||
CategoryListParams clParams = sParams.getFacetIndexingParams().getCategoryListParams(categoryPath);
|
||||
if (clCache != null) {
|
||||
CategoryListData clData = clCache.get(clParams);
|
||||
if (clData != null) {
|
||||
return clData.iterator(partition);
|
||||
}
|
||||
}
|
||||
return clParams.createCategoryListIterator(partition);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the value of a category used for facets computations for this
|
||||
* request. For a count request this would be the count for that facet, i.e.
|
||||
|
|
|
@ -4,7 +4,6 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.cache.CategoryListCache;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -71,14 +70,6 @@ public class FacetSearchParams {
|
|||
this.facetRequests = facetRequests;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link CategoryListCache}. By default returns {@code null}, you
|
||||
* should override if you want to use a cache.
|
||||
*/
|
||||
public CategoryListCache getCategoryListCache() {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link FacetIndexingParams} that were passed to the
|
||||
* constructor.
|
||||
|
|
|
@ -28,7 +28,10 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
|||
|
||||
/**
|
||||
* A {@link FacetRequest} for weighting facets according to their float
|
||||
* association by summing the association values.
|
||||
* association by summing the association values. Note that this class caches
|
||||
* the associations data in-memory by default. You can override
|
||||
* {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
|
||||
* {@link AssociationFloatSumAggregator} which does otherwise.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
|
|
@ -4,6 +4,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.facet.search.FacetArrays;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
import org.apache.lucene.facet.search.aggregator.associations.AssociationFloatSumAggregator;
|
||||
import org.apache.lucene.facet.search.aggregator.associations.AssociationIntSumAggregator;
|
||||
import org.apache.lucene.facet.search.params.FacetRequest;
|
||||
import org.apache.lucene.facet.taxonomy.CategoryPath;
|
||||
|
@ -28,7 +29,10 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
|||
|
||||
/**
|
||||
* A {@link FacetRequest} for weighting facets according to their integer
|
||||
* association by summing the association values.
|
||||
* association by summing the association values. Note that this class caches
|
||||
* the associations data in-memory by default. You can override
|
||||
* {@link #createAggregator(boolean, FacetArrays, TaxonomyReader)} to return an
|
||||
* {@link AssociationFloatSumAggregator} which does otherwise.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
|
|
|
@ -39,7 +39,7 @@ public class FacetResult {
|
|||
/**
|
||||
* Facet result node matching the root of the {@link #getFacetRequest() facet request}.
|
||||
* @see #getFacetRequest()
|
||||
* @see FacetRequest#getCategoryPath()
|
||||
* @see FacetRequest#categoryPath
|
||||
*/
|
||||
public final FacetResultNode getFacetResultNode() {
|
||||
return this.rootNode;
|
||||
|
|
|
@ -4,9 +4,6 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import org.apache.lucene.facet.search.CategoryListIterator;
|
||||
import org.apache.lucene.facet.search.FacetArrays;
|
||||
import org.apache.lucene.facet.search.ScoredDocIDs;
|
||||
import org.apache.lucene.facet.search.aggregator.Aggregator;
|
||||
|
@ -16,6 +13,7 @@ import org.apache.lucene.facet.search.results.FacetResult;
|
|||
import org.apache.lucene.facet.search.results.FacetResultNode;
|
||||
import org.apache.lucene.facet.search.results.MutableFacetResultNode;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -205,7 +203,7 @@ public abstract class Sampler {
|
|||
private static class OverSampledFacetRequest extends FacetRequest {
|
||||
final FacetRequest orig;
|
||||
public OverSampledFacetRequest(FacetRequest orig, int num) {
|
||||
super(orig.getCategoryPath(), num);
|
||||
super(orig.categoryPath, num);
|
||||
this.orig = orig;
|
||||
setDepth(orig.getDepth());
|
||||
setNumLabel(orig.getNumLabel());
|
||||
|
@ -214,12 +212,6 @@ public abstract class Sampler {
|
|||
setSortOrder(orig.getSortOrder());
|
||||
}
|
||||
|
||||
@Override
|
||||
public CategoryListIterator createCategoryListIterator(TaxonomyReader taxo, FacetSearchParams sParams,
|
||||
int partition) throws IOException {
|
||||
return orig.createCategoryListIterator(taxo, sParams, partition);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy)
|
||||
throws IOException {
|
||||
|
|
|
@ -92,8 +92,8 @@ public class CategoryPath implements Comparable<CategoryPath> {
|
|||
*/
|
||||
@Override
|
||||
public int compareTo(CategoryPath other) {
|
||||
int length = this.length < other.length ? this.length : other.length;
|
||||
for (int i = 0, j = 0; i < length; i++, j++) {
|
||||
final int len = length < other.length ? length : other.length;
|
||||
for (int i = 0, j = 0; i < len; i++, j++) {
|
||||
int cmp = components[i].compareTo(other.components[j]);
|
||||
if (cmp < 0) return -1; // this is 'before'
|
||||
if (cmp > 0) return 1; // this is 'after'
|
||||
|
|
|
@ -1,6 +1,5 @@
|
|||
package org.apache.lucene.facet.util;
|
||||
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
|
||||
|
@ -28,15 +27,9 @@ import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
|||
*/
|
||||
public final class PartitionsUtils {
|
||||
|
||||
/**
|
||||
* Get the offset for a given partition. That is, what is the minimum number an
|
||||
* ordinal could be for a particular partition.
|
||||
*/
|
||||
public final static int partitionOffset(FacetIndexingParams iParams,
|
||||
int partitionNumber, final TaxonomyReader taxonomyReader) {
|
||||
return partitionNumber * partitionSize(iParams, taxonomyReader);
|
||||
}
|
||||
|
||||
/** The prefix that is added to the name of the partition. */
|
||||
public static final String PART_NAME_PREFIX = "$part";
|
||||
|
||||
/**
|
||||
* Get the partition size in this parameter, or return the size of the taxonomy, which
|
||||
* is smaller. (Guarantees usage of as little memory as possible at search time).
|
||||
|
@ -58,21 +51,21 @@ public final class PartitionsUtils {
|
|||
/**
|
||||
* Partition name by category ordinal
|
||||
*/
|
||||
public final static String partitionNameByOrdinal(
|
||||
FacetIndexingParams iParams, CategoryListParams clParams, int ordinal) {
|
||||
public final static String partitionNameByOrdinal(FacetIndexingParams iParams, int ordinal) {
|
||||
int partition = partitionNumber(iParams, ordinal);
|
||||
return partitionName(clParams, partition);
|
||||
return partitionName(partition);
|
||||
}
|
||||
|
||||
/**
|
||||
* Partition name by its number
|
||||
*/
|
||||
public final static String partitionName(CategoryListParams clParams, int partition) {
|
||||
String term = clParams.getTerm().text();
|
||||
/** Partition name by its number */
|
||||
public final static String partitionName(int partition) {
|
||||
// TODO would be good if this method isn't called when partitions are not enabled.
|
||||
// perhaps through some specialization code.
|
||||
if (partition == 0) {
|
||||
return term; // for backwards compatibility we do not add a partition number in this case
|
||||
// since regular faceted search code goes through this method too,
|
||||
// return the same value for partition 0 and when there are no partitions
|
||||
return "";
|
||||
}
|
||||
return term + partition;
|
||||
return PART_NAME_PREFIX + Integer.toString(partition);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -49,20 +49,20 @@ import org.apache.lucene.util.IntsRef;
|
|||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class ChunksIntEncoder extends IntEncoder {
|
||||
|
||||
|
||||
/** Holds the values which must be encoded, outside the indicator. */
|
||||
protected final IntsRef encodeQueue;
|
||||
|
||||
|
||||
/** Represents bits flag byte. */
|
||||
protected int indicator = 0;
|
||||
|
||||
|
||||
/** Counts the current ordinal of the encoded value. */
|
||||
protected byte ordinal = 0;
|
||||
|
||||
|
||||
protected ChunksIntEncoder(int chunkSize) {
|
||||
encodeQueue = new IntsRef(chunkSize);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Encodes the values of the current chunk. First it writes the indicator, and
|
||||
* then it encodes the values outside the indicator.
|
||||
|
@ -76,17 +76,40 @@ public abstract class ChunksIntEncoder extends IntEncoder {
|
|||
|
||||
buf.bytes[buf.length++] = ((byte) indicator);
|
||||
for (int i = 0; i < encodeQueue.length; i++) {
|
||||
VInt8.encode(encodeQueue.ints[i], buf);
|
||||
// it is better if the encoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int value = encodeQueue.ints[i];
|
||||
if ((value & ~0x7F) == 0) {
|
||||
buf.bytes[buf.length] = (byte) value;
|
||||
buf.length++;
|
||||
} else if ((value & ~0x3FFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 1] = (byte) (value & 0x7F);
|
||||
buf.length += 2;
|
||||
} else if ((value & ~0x1FFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 2] = (byte) (value & 0x7F);
|
||||
buf.length += 3;
|
||||
} else if ((value & ~0xFFFFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 3] = (byte) (value & 0x7F);
|
||||
buf.length += 4;
|
||||
} else {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 4] = (byte) (value & 0x7F);
|
||||
buf.length += 5;
|
||||
}
|
||||
}
|
||||
|
||||
reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void reset() {
|
||||
ordinal = 0;
|
||||
indicator = 0;
|
||||
encodeQueue.length = 0;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class DGapIntDecoder extends IntDecoder {
|
||||
public final class DGapIntDecoder extends IntDecoder {
|
||||
|
||||
private final IntDecoder decoder;
|
||||
|
||||
|
@ -35,13 +35,8 @@ public class DGapIntDecoder extends IntDecoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void reset() {
|
||||
decoder.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
decoder.doDecode(buf, values, upto);
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
decoder.decode(buf, values);
|
||||
int prev = 0;
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
values.ints[i] += prev;
|
||||
|
@ -51,7 +46,7 @@ public class DGapIntDecoder extends IntDecoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DGap (" + decoder.toString() + ")";
|
||||
return "DGap(" + decoder.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class DGapIntEncoder extends IntEncoderFilter {
|
||||
public final class DGapIntEncoder extends IntEncoderFilter {
|
||||
|
||||
/** Initializes with the given encoder. */
|
||||
public DGapIntEncoder(IntEncoder encoder) {
|
||||
|
@ -43,14 +43,15 @@ public class DGapIntEncoder extends IntEncoderFilter {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
int prev = 0;
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
int tmp = values.ints[i];
|
||||
values.ints[i] -= prev;
|
||||
prev = tmp;
|
||||
}
|
||||
encoder.doEncode(values, buf, upto);
|
||||
encoder.encode(values, buf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -60,7 +61,7 @@ public class DGapIntEncoder extends IntEncoderFilter {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DGap (" + encoder.toString() + ")";
|
||||
return "DGap(" + encoder.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
package org.apache.lucene.util.encoding;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Decodes values encoded by {@link DGapVInt8IntDecoder}.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class DGapVInt8IntDecoder extends IntDecoder {
|
||||
|
||||
@Override
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
|
||||
// grow the buffer up front, even if by a large number of values (buf.length)
|
||||
// that saves the need to check inside the loop for every decoded value if
|
||||
// the buffer needs to grow.
|
||||
if (values.ints.length < buf.length) {
|
||||
values.ints = new int[ArrayUtil.oversize(buf.length, RamUsageEstimator.NUM_BYTES_INT)];
|
||||
}
|
||||
|
||||
// it is better if the decoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int upto = buf.offset + buf.length;
|
||||
int value = 0;
|
||||
int offset = buf.offset;
|
||||
int prev = 0;
|
||||
while (offset < upto) {
|
||||
byte b = buf.bytes[offset++];
|
||||
if (b >= 0) {
|
||||
values.ints[values.length] = ((value << 7) | b) + prev;
|
||||
value = 0;
|
||||
prev = values.ints[values.length];
|
||||
values.length++;
|
||||
} else {
|
||||
value = (value << 7) | (b & 0x7F);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DGapVInt8";
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
package org.apache.lucene.util.encoding;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* An {@link IntEncoder} which implements variable length encoding for the gap
|
||||
* between values. It's a specialized form of the combination of
|
||||
* {@link DGapIntEncoder} and {@link VInt8IntEncoder}.
|
||||
*
|
||||
* @see VInt8IntEncoder
|
||||
* @see DGapIntEncoder
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class DGapVInt8IntEncoder extends IntEncoder {
|
||||
|
||||
@Override
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
int maxBytesNeeded = 5 * values.length; // at most 5 bytes per VInt
|
||||
if (buf.bytes.length < maxBytesNeeded) {
|
||||
buf.grow(maxBytesNeeded);
|
||||
}
|
||||
|
||||
int upto = values.offset + values.length;
|
||||
int prev = 0;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
// it is better if the encoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int value = values.ints[i] - prev;
|
||||
if ((value & ~0x7F) == 0) {
|
||||
buf.bytes[buf.length] = (byte) value;
|
||||
buf.length++;
|
||||
} else if ((value & ~0x3FFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 1] = (byte) (value & 0x7F);
|
||||
buf.length += 2;
|
||||
} else if ((value & ~0x1FFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 2] = (byte) (value & 0x7F);
|
||||
buf.length += 3;
|
||||
} else if ((value & ~0xFFFFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 3] = (byte) (value & 0x7F);
|
||||
buf.length += 4;
|
||||
} else {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 4] = (byte) (value & 0x7F);
|
||||
buf.length += 5;
|
||||
}
|
||||
prev = values.ints[i];
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public IntDecoder createMatchingDecoder() {
|
||||
return new DGapVInt8IntDecoder();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "DGapVInt8";
|
||||
}
|
||||
|
||||
}
|
|
@ -45,10 +45,13 @@ public class EightFlagsIntDecoder extends IntDecoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
while (buf.offset < upto) {
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
int upto = buf.offset + buf.length;
|
||||
int offset = buf.offset;
|
||||
while (offset < upto) {
|
||||
// read indicator
|
||||
int indicator = buf.bytes[buf.offset++] & 0xFF;
|
||||
int indicator = buf.bytes[offset++] & 0xFF;
|
||||
int ordinal = 0;
|
||||
|
||||
int capacityNeeded = values.length + 8;
|
||||
|
@ -59,11 +62,21 @@ public class EightFlagsIntDecoder extends IntDecoder {
|
|||
// process indicator, until we read 8 values, or end-of-buffer
|
||||
while (ordinal != 8) {
|
||||
if (DECODE_TABLE[indicator][ordinal++] == 0) {
|
||||
if (buf.offset == upto) { // end of buffer
|
||||
if (offset == upto) { // end of buffer
|
||||
return;
|
||||
}
|
||||
// decode the value from the stream.
|
||||
values.ints[values.length++] = VInt8.decode(buf) + 2;
|
||||
// it is better if the decoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int value = 0;
|
||||
while (true) {
|
||||
byte b = buf.bytes[offset++];
|
||||
if (b >= 0) {
|
||||
values.ints[values.length++] = ((value << 7) | b) + 2;
|
||||
break;
|
||||
} else {
|
||||
value = (value << 7) | (b & 0x7F);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
values.ints[values.length++] = 1;
|
||||
}
|
||||
|
@ -73,7 +86,7 @@ public class EightFlagsIntDecoder extends IntDecoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "EightFlags (VInt8)";
|
||||
return "EightFlags(VInt8)";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -59,7 +59,9 @@ public class EightFlagsIntEncoder extends ChunksIntEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
int value = values.ints[i];
|
||||
if (value == 1) {
|
||||
|
@ -88,7 +90,7 @@ public class EightFlagsIntEncoder extends ChunksIntEncoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "EightFlags (VInt)";
|
||||
return "EightFlags(VInt)";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -45,10 +45,13 @@ public class FourFlagsIntDecoder extends IntDecoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
while (buf.offset < upto) {
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
int upto = buf.offset + buf.length;
|
||||
int offset = buf.offset;
|
||||
while (offset < upto) {
|
||||
// read indicator
|
||||
int indicator = buf.bytes[buf.offset++] & 0xFF;
|
||||
int indicator = buf.bytes[offset++] & 0xFF;
|
||||
int ordinal = 0;
|
||||
|
||||
int capacityNeeded = values.length + 4;
|
||||
|
@ -59,11 +62,21 @@ public class FourFlagsIntDecoder extends IntDecoder {
|
|||
while (ordinal != 4) {
|
||||
byte decodeVal = DECODE_TABLE[indicator][ordinal++];
|
||||
if (decodeVal == 0) {
|
||||
if (buf.offset == upto) { // end of buffer
|
||||
if (offset == upto) { // end of buffer
|
||||
return;
|
||||
}
|
||||
// decode the value from the stream.
|
||||
values.ints[values.length++] = VInt8.decode(buf) + 4;
|
||||
// it is better if the decoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int value = 0;
|
||||
while (true) {
|
||||
byte b = buf.bytes[offset++];
|
||||
if (b >= 0) {
|
||||
values.ints[values.length++] = ((value << 7) | b) + 4;
|
||||
break;
|
||||
} else {
|
||||
value = (value << 7) | (b & 0x7F);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
values.ints[values.length++] = decodeVal;
|
||||
}
|
||||
|
@ -73,7 +86,7 @@ public class FourFlagsIntDecoder extends IntDecoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FourFlags (VInt8)";
|
||||
return "FourFlags(VInt8)";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -65,7 +65,9 @@ public class FourFlagsIntEncoder extends ChunksIntEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
int value = values.ints[i];
|
||||
if (value <= 3) {
|
||||
|
@ -94,7 +96,7 @@ public class FourFlagsIntEncoder extends ChunksIntEncoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FourFlags (VInt)";
|
||||
return "FourFlags(VInt)";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,44 +27,10 @@ import org.apache.lucene.util.IntsRef;
|
|||
*/
|
||||
public abstract class IntDecoder {
|
||||
|
||||
/**
|
||||
* Performs the actual decoding. Values should be read from
|
||||
* {@link BytesRef#offset} up to {@code upto}. Also, {@code values} offset and
|
||||
* length are set to 0 and the encoder is expected to update
|
||||
* {@link IntsRef#length}, but not {@link IntsRef#offset}.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE:</b> it is ok to use the buffer's offset as the current position in
|
||||
* the buffer (and modify it), it will be reset by
|
||||
* {@link #decode(BytesRef, IntsRef)}.
|
||||
*/
|
||||
protected abstract void doDecode(BytesRef buf, IntsRef values, int upto);
|
||||
|
||||
/**
|
||||
* Called before {@link #doDecode(BytesRef, IntsRef, int)} so that decoders
|
||||
* can reset their state.
|
||||
*/
|
||||
protected void reset() {
|
||||
// do nothing by default
|
||||
}
|
||||
|
||||
/**
|
||||
* Decodes the values from the buffer into the given {@link IntsRef}. Note
|
||||
* that {@code values.offset} and {@code values.length} are set to 0.
|
||||
*/
|
||||
public final void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0; // must do that because we cannot grow() them otherwise
|
||||
|
||||
// some decoders may use the buffer's offset as a position index, so save
|
||||
// current offset.
|
||||
int bufOffset = buf.offset;
|
||||
|
||||
reset();
|
||||
doDecode(buf, values, buf.offset + buf.length);
|
||||
assert values.offset == 0 : "offset should not have been modified by the decoder.";
|
||||
|
||||
// fix offset
|
||||
buf.offset = bufOffset;
|
||||
}
|
||||
public abstract void decode(BytesRef buf, IntsRef values);
|
||||
|
||||
}
|
||||
|
|
|
@ -31,32 +31,11 @@ public abstract class IntEncoder {
|
|||
|
||||
public IntEncoder() {}
|
||||
|
||||
/**
|
||||
* Performs the actual encoding. Values should be read from
|
||||
* {@link IntsRef#offset} up to {@code upto}. Also, it is guaranteed that
|
||||
* {@code buf's} offset and length are set to 0 and the encoder is expected to
|
||||
* update {@link BytesRef#length}, but not {@link BytesRef#offset}.
|
||||
*/
|
||||
protected abstract void doEncode(IntsRef values, BytesRef buf, int upto);
|
||||
|
||||
/**
|
||||
* Called before {@link #doEncode(IntsRef, BytesRef, int)} so that encoders
|
||||
* can reset their state.
|
||||
*/
|
||||
protected void reset() {
|
||||
// do nothing by default
|
||||
}
|
||||
|
||||
/**
|
||||
* Encodes the values to the given buffer. Note that the buffer's offset and
|
||||
* length are set to 0.
|
||||
*/
|
||||
public final void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
reset();
|
||||
doEncode(values, buf, values.offset + values.length);
|
||||
assert buf.offset == 0 : "offset should not have been modified by the encoder.";
|
||||
}
|
||||
public abstract void encode(IntsRef values, BytesRef buf);
|
||||
|
||||
/**
|
||||
* Returns an {@link IntDecoder} which can decode the values that were encoded
|
||||
|
|
|
@ -31,9 +31,4 @@ public abstract class IntEncoderFilter extends IntEncoder {
|
|||
this.encoder = encoder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
encoder.reset();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -42,14 +42,10 @@ public class NOnesIntDecoder extends FourFlagsIntDecoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void reset() {
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
internalBuffer.length = 0;
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
super.doDecode(buf, internalBuffer, upto);
|
||||
super.decode(buf, internalBuffer);
|
||||
if (values.ints.length < internalBuffer.length) {
|
||||
// need space for internalBuffer.length to internalBuffer.length*N,
|
||||
// grow mildly at first
|
||||
|
@ -84,7 +80,7 @@ public class NOnesIntDecoder extends FourFlagsIntDecoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "NOnes (" + n + ") (" + super.toString() + ")";
|
||||
return "NOnes(" + n + ") (" + super.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -65,19 +65,15 @@ public class NOnesIntEncoder extends FourFlagsIntEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void reset() {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
internalBuffer.length = 0;
|
||||
super.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
// make sure the internal buffer is large enough
|
||||
if (values.length > internalBuffer.ints.length) {
|
||||
internalBuffer.grow(values.length);
|
||||
}
|
||||
|
||||
int onesCounter = 0;
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
int value = values.ints[i];
|
||||
if (value == 1) {
|
||||
|
@ -102,7 +98,7 @@ public class NOnesIntEncoder extends FourFlagsIntEncoder {
|
|||
--onesCounter;
|
||||
internalBuffer.ints[internalBuffer.length++] = 1;
|
||||
}
|
||||
super.doEncode(internalBuffer, buf, internalBuffer.length);
|
||||
super.encode(internalBuffer, buf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -112,7 +108,7 @@ public class NOnesIntEncoder extends FourFlagsIntEncoder {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "NOnes (" + n + ") (" + super.toString() + ")";
|
||||
return "NOnes(" + n + ") (" + super.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,7 +1,9 @@
|
|||
package org.apache.lucene.util.encoding;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -25,19 +27,24 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleIntDecoder extends IntDecoder {
|
||||
public final class SimpleIntDecoder extends IntDecoder {
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
while (buf.offset < upto) {
|
||||
if (values.length == values.ints.length) {
|
||||
values.grow(values.length + 10); // grow by few items, however not too many
|
||||
}
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
int numValues = buf.length / 4; // every value is 4 bytes
|
||||
if (values.ints.length < numValues) { // offset and length are 0
|
||||
values.ints = new int[ArrayUtil.oversize(numValues, RamUsageEstimator.NUM_BYTES_INT)];
|
||||
}
|
||||
|
||||
int offset = buf.offset;
|
||||
int upto = buf.offset + buf.length;
|
||||
while (offset < upto) {
|
||||
values.ints[values.length++] =
|
||||
((buf.bytes[buf.offset++] & 0xFF) << 24) |
|
||||
((buf.bytes[buf.offset++] & 0xFF) << 16) |
|
||||
((buf.bytes[buf.offset++] & 0xFF) << 8) |
|
||||
(buf.bytes[buf.offset++] & 0xFF);
|
||||
((buf.bytes[offset++] & 0xFF) << 24) |
|
||||
((buf.bytes[offset++] & 0xFF) << 16) |
|
||||
((buf.bytes[offset++] & 0xFF) << 8) |
|
||||
(buf.bytes[offset++] & 0xFF);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,16 +25,18 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleIntEncoder extends IntEncoder {
|
||||
public final class SimpleIntEncoder extends IntEncoder {
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
// ensure there's enough room in the buffer
|
||||
int bytesNeeded = values.length * 4;
|
||||
if (buf.bytes.length < bytesNeeded) {
|
||||
buf.grow(bytesNeeded);
|
||||
}
|
||||
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
int value = values.ints[i];
|
||||
buf.bytes[buf.length++] = (byte) (value >>> 24);
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SortingIntEncoder extends IntEncoderFilter {
|
||||
public final class SortingIntEncoder extends IntEncoderFilter {
|
||||
|
||||
/** Initializes with the given encoder. */
|
||||
public SortingIntEncoder(IntEncoder encoder) {
|
||||
|
@ -36,9 +36,9 @@ public class SortingIntEncoder extends IntEncoderFilter {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
Arrays.sort(values.ints, values.offset, upto);
|
||||
encoder.doEncode(values, buf, upto);
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
Arrays.sort(values.ints, values.offset, values.offset + values.length);
|
||||
encoder.encode(values, buf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -48,7 +48,7 @@ public class SortingIntEncoder extends IntEncoderFilter {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Sorting (" + encoder.toString() + ")";
|
||||
return "Sorting(" + encoder.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -36,9 +36,10 @@ public final class UniqueValuesIntEncoder extends IntEncoderFilter {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
int prev = values.ints[values.offset];
|
||||
int idx = values.offset + 1;
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = idx; i < upto; i++) {
|
||||
if (values.ints[i] != prev) {
|
||||
values.ints[idx++] = values.ints[i];
|
||||
|
@ -46,7 +47,7 @@ public final class UniqueValuesIntEncoder extends IntEncoderFilter {
|
|||
}
|
||||
}
|
||||
values.length = idx - values.offset;
|
||||
encoder.doEncode(values, buf, idx);
|
||||
encoder.encode(values, buf);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -56,7 +57,7 @@ public final class UniqueValuesIntEncoder extends IntEncoderFilter {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Unique (" + encoder.toString() + ")";
|
||||
return "Unique(" + encoder.toString() + ")";
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,138 +0,0 @@
|
|||
package org.apache.lucene.util.encoding;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* Variable-length encoding of 32-bit integers, into 8-bit bytes. A number is
|
||||
* encoded as follows:
|
||||
* <ul>
|
||||
* <li>If it is less than 127 and non-negative (i.e., if the number uses only 7
|
||||
* bits), it is encoded as as single byte: 0bbbbbbb.
|
||||
* <li>If its highest nonzero bit is greater than bit 6 (0x40), it is
|
||||
* represented as a series of bytes, each byte's 7 LSB containing bits from the
|
||||
* original value, with the MSB set for all but the last byte. The first encoded
|
||||
* byte contains the highest nonzero bits from the original; the second byte
|
||||
* contains the next 7 MSB; and so on, with the last byte containing the 7 LSB
|
||||
* of the original.
|
||||
* </ul>
|
||||
* Examples:
|
||||
* <ol>
|
||||
* <li>n = 117 = 1110101: This has fewer than 8 significant bits, and so is
|
||||
* encoded as 01110101 = 0x75.
|
||||
* <li>n = 100000 = (binary) 11000011010100000. This has 17 significant bits,
|
||||
* and so needs three Vint8 bytes. Left-zero-pad it to a multiple of 7 bits,
|
||||
* then split it into chunks of 7 and add an MSB, 0 for the last byte, 1 for the
|
||||
* others: 1|0000110 1|0001101 0|0100000 = 0x86 0x8D 0x20.
|
||||
* </ol>
|
||||
* {@link #encode(int, BytesRef)} and {@link #decode(BytesRef)} will correctly
|
||||
* handle any 32-bit integer, but for negative numbers, and positive numbers
|
||||
* with more than 28 significant bits, encoding requires 5 bytes; this is not an
|
||||
* efficient encoding scheme for large positive numbers or any negative number.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class VInt8 {
|
||||
|
||||
/** The maximum number of bytes needed to encode an integer. */
|
||||
public static final int MAXIMUM_BYTES_NEEDED = 5;
|
||||
|
||||
/**
|
||||
* Decodes an int from the given bytes, starting at {@link BytesRef#offset}.
|
||||
* Returns the decoded bytes and updates {@link BytesRef#offset}.
|
||||
*/
|
||||
public static int decode(BytesRef bytes) {
|
||||
/*
|
||||
This is the original code of this method, but a Hotspot bug
|
||||
corrupted the for-loop of DataInput.readVInt() (see LUCENE-2975)
|
||||
so the loop was unwounded here too, to be on the safe side
|
||||
int value = 0;
|
||||
while (true) {
|
||||
byte first = bytes.bytes[bytes.offset++];
|
||||
value |= first & 0x7F;
|
||||
if ((first & 0x80) == 0) {
|
||||
return value;
|
||||
}
|
||||
value <<= 7;
|
||||
}
|
||||
*/
|
||||
|
||||
// byte 1
|
||||
byte b = bytes.bytes[bytes.offset++];
|
||||
if (b >= 0) return b;
|
||||
|
||||
// byte 2
|
||||
int value = b & 0x7F;
|
||||
b = bytes.bytes[bytes.offset++];
|
||||
value = (value << 7) | b & 0x7F;
|
||||
if (b >= 0) return value;
|
||||
|
||||
// byte 3
|
||||
b = bytes.bytes[bytes.offset++];
|
||||
value = (value << 7) | b & 0x7F;
|
||||
if (b >= 0) return value;
|
||||
|
||||
// byte 4
|
||||
b = bytes.bytes[bytes.offset++];
|
||||
value = (value << 7) | b & 0x7F;
|
||||
if (b >= 0) return value;
|
||||
|
||||
// byte 5
|
||||
b = bytes.bytes[bytes.offset++];
|
||||
return (value << 7) | b & 0x7F;
|
||||
}
|
||||
|
||||
/**
|
||||
* Encodes the given number into bytes, starting at {@link BytesRef#length}.
|
||||
* Assumes that the array is large enough.
|
||||
*/
|
||||
public static void encode(int value, BytesRef bytes) {
|
||||
if ((value & ~0x7F) == 0) {
|
||||
bytes.bytes[bytes.length] = (byte) value;
|
||||
bytes.length++;
|
||||
} else if ((value & ~0x3FFF) == 0) {
|
||||
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
bytes.bytes[bytes.length + 1] = (byte) (value & 0x7F);
|
||||
bytes.length += 2;
|
||||
} else if ((value & ~0x1FFFFF) == 0) {
|
||||
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
bytes.bytes[bytes.length + 2] = (byte) (value & 0x7F);
|
||||
bytes.length += 3;
|
||||
} else if ((value & ~0xFFFFFFF) == 0) {
|
||||
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
bytes.bytes[bytes.length + 3] = (byte) (value & 0x7F);
|
||||
bytes.length += 4;
|
||||
} else {
|
||||
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
|
||||
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
bytes.bytes[bytes.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
bytes.bytes[bytes.length + 4] = (byte) (value & 0x7F);
|
||||
bytes.length += 5;
|
||||
}
|
||||
}
|
||||
|
||||
private VInt8() {
|
||||
// Just making it impossible to instantiate.
|
||||
}
|
||||
|
||||
}
|
|
@ -1,7 +1,9 @@
|
|||
package org.apache.lucene.util.encoding;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -25,15 +27,32 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class VInt8IntDecoder extends IntDecoder {
|
||||
public final class VInt8IntDecoder extends IntDecoder {
|
||||
|
||||
@Override
|
||||
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
|
||||
while (buf.offset < upto) {
|
||||
if (values.length == values.ints.length) {
|
||||
values.grow(values.length + 10); // grow by few items, however not too many
|
||||
public void decode(BytesRef buf, IntsRef values) {
|
||||
values.offset = values.length = 0;
|
||||
|
||||
// grow the buffer up front, even if by a large number of values (buf.length)
|
||||
// that saves the need to check inside the loop for every decoded value if
|
||||
// the buffer needs to grow.
|
||||
if (values.ints.length < buf.length) {
|
||||
values.ints = new int[ArrayUtil.oversize(buf.length, RamUsageEstimator.NUM_BYTES_INT)];
|
||||
}
|
||||
|
||||
// it is better if the decoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int upto = buf.offset + buf.length;
|
||||
int value = 0;
|
||||
int offset = buf.offset;
|
||||
while (offset < upto) {
|
||||
byte b = buf.bytes[offset++];
|
||||
if (b >= 0) {
|
||||
values.ints[values.length++] = (value << 7) | b;
|
||||
value = 0;
|
||||
} else {
|
||||
value = (value << 7) | (b & 0x7F);
|
||||
}
|
||||
values.ints[values.length++] = VInt8.decode(buf);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -47,17 +47,47 @@ import org.apache.lucene.util.IntsRef;
|
|||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class VInt8IntEncoder extends IntEncoder {
|
||||
public final class VInt8IntEncoder extends IntEncoder {
|
||||
|
||||
@Override
|
||||
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
|
||||
public void encode(IntsRef values, BytesRef buf) {
|
||||
buf.offset = buf.length = 0;
|
||||
int maxBytesNeeded = 5 * values.length; // at most 5 bytes per VInt
|
||||
if (buf.bytes.length < maxBytesNeeded) {
|
||||
buf.grow(maxBytesNeeded);
|
||||
}
|
||||
|
||||
int upto = values.offset + values.length;
|
||||
for (int i = values.offset; i < upto; i++) {
|
||||
VInt8.encode(values.ints[i], buf);
|
||||
// it is better if the encoding is inlined like so, and not e.g.
|
||||
// in a utility method
|
||||
int value = values.ints[i];
|
||||
if ((value & ~0x7F) == 0) {
|
||||
buf.bytes[buf.length] = (byte) value;
|
||||
buf.length++;
|
||||
} else if ((value & ~0x3FFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 1] = (byte) (value & 0x7F);
|
||||
buf.length += 2;
|
||||
} else if ((value & ~0x1FFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 2] = (byte) (value & 0x7F);
|
||||
buf.length += 3;
|
||||
} else if ((value & ~0xFFFFFFF) == 0) {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 3] = (byte) (value & 0x7F);
|
||||
buf.length += 4;
|
||||
} else {
|
||||
buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
|
||||
buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
|
||||
buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
|
||||
buf.bytes[buf.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
|
||||
buf.bytes[buf.length + 4] = (byte) (value & 0x7F);
|
||||
buf.length += 5;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.lucene.store.Directory;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||
import org.apache.lucene.util._TestUtil;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -64,7 +65,7 @@ import org.junit.BeforeClass;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Base faceted search test. */
|
||||
@SuppressCodecs({"SimpleText"})
|
||||
public abstract class FacetTestBase extends LuceneTestCase {
|
||||
|
||||
/** Holds a search and taxonomy Directories pair. */
|
||||
|
@ -266,13 +267,12 @@ public abstract class FacetTestBase extends LuceneTestCase {
|
|||
FacetIndexingParams iParams = getFacetIndexingParams(Integer.MAX_VALUE);
|
||||
String delim = String.valueOf(iParams.getFacetDelimChar());
|
||||
Map<CategoryPath, Integer> res = new HashMap<CategoryPath, Integer>();
|
||||
HashSet<Term> handledTerms = new HashSet<Term>();
|
||||
HashSet<String> handledTerms = new HashSet<String>();
|
||||
for (CategoryListParams clp : iParams.getAllCategoryListParams()) {
|
||||
Term baseTerm = new Term(clp.getTerm().field());
|
||||
if (!handledTerms.add(baseTerm)) {
|
||||
if (!handledTerms.add(clp.field)) {
|
||||
continue; // already handled this term (for another list)
|
||||
}
|
||||
Terms terms = MultiFields.getTerms(indexReader, baseTerm.field());
|
||||
Terms terms = MultiFields.getTerms(indexReader, clp.field);
|
||||
if (terms == null) {
|
||||
continue;
|
||||
}
|
||||
|
@ -297,7 +297,7 @@ public abstract class FacetTestBase extends LuceneTestCase {
|
|||
FacetResultNode topResNode = fr.getFacetResultNode();
|
||||
FacetRequest freq = fr.getFacetRequest();
|
||||
if (VERBOSE) {
|
||||
System.out.println(freq.getCategoryPath().toString()+ "\t\t" + topResNode);
|
||||
System.out.println(freq.categoryPath.toString()+ "\t\t" + topResNode);
|
||||
}
|
||||
assertCountsAndCardinality(facetCountsTruth, topResNode, freq.getNumResults());
|
||||
}
|
||||
|
|
|
@ -2,14 +2,9 @@ package org.apache.lucene.facet;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.facet.index.FacetFields;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.search.FacetsCollector;
|
||||
import org.apache.lucene.facet.search.params.CountFacetRequest;
|
||||
|
@ -23,7 +18,6 @@ import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
|
|||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
|
@ -116,15 +110,6 @@ public class FacetTestUtils {
|
|||
return collectors;
|
||||
}
|
||||
|
||||
public static void add(FacetIndexingParams iParams, RandomIndexWriter iw,
|
||||
TaxonomyWriter tw, String... strings) throws IOException {
|
||||
Document d = new Document();
|
||||
FacetFields facetFields = new FacetFields(tw, iParams);
|
||||
facetFields.addFields(d, Collections.singletonList(new CategoryPath(strings)));
|
||||
d.add(new TextField("content", "alpha", Field.Store.YES));
|
||||
iw.addDocument(d);
|
||||
}
|
||||
|
||||
public static class IndexTaxonomyReaderPair {
|
||||
public DirectoryReader indexReader;
|
||||
public DirectoryTaxonomyReader taxReader;
|
||||
|
|
|
@ -0,0 +1,398 @@
|
|||
package org.apache.lucene.facet.index;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.facet.index.params.FacetIndexingParams;
|
||||
import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
|
||||
import org.apache.lucene.facet.search.CategoryListIterator;
|
||||
import org.apache.lucene.facet.search.DrillDown;
|
||||
import org.apache.lucene.facet.search.FacetsCollector;
|
||||
import org.apache.lucene.facet.search.params.CountFacetRequest;
|
||||
import org.apache.lucene.facet.search.params.FacetRequest;
|
||||
import org.apache.lucene.facet.search.params.FacetSearchParams;
|
||||
import org.apache.lucene.facet.search.results.FacetResult;
|
||||
import org.apache.lucene.facet.search.results.FacetResultNode;
|
||||
import org.apache.lucene.facet.taxonomy.CategoryPath;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
|
||||
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
|
||||
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
|
||||
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
|
||||
import org.apache.lucene.facet.util.PartitionsUtils;
|
||||
import org.apache.lucene.index.AtomicReader;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.MultiReader;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.MultiCollector;
|
||||
import org.apache.lucene.search.PrefixQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.TotalHitCountCollector;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/** Tests facets index migration from payload to DocValues.*/
|
||||
public class TestFacetsPayloadMigrationReader extends LuceneTestCase {
|
||||
|
||||
private static class PayloadFacetFields extends FacetFields {
|
||||
|
||||
private static final class CountingListStream extends TokenStream {
|
||||
private final PayloadAttribute payloadAtt = addAttribute(PayloadAttribute.class);
|
||||
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
|
||||
private final Iterator<Entry<String,BytesRef>> categoriesData;
|
||||
|
||||
CountingListStream(Map<String,BytesRef> categoriesData) {
|
||||
this.categoriesData = categoriesData.entrySet().iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (!categoriesData.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Entry<String,BytesRef> entry = categoriesData.next();
|
||||
termAtt.setEmpty().append(FacetsPayloadMigrationReader.PAYLOAD_TERM_TEXT + entry.getKey());
|
||||
payloadAtt.setPayload(entry.getValue());
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final FieldType COUNTING_LIST_PAYLOAD_TYPE = new FieldType();
|
||||
static {
|
||||
COUNTING_LIST_PAYLOAD_TYPE.setIndexed(true);
|
||||
COUNTING_LIST_PAYLOAD_TYPE.setTokenized(true);
|
||||
COUNTING_LIST_PAYLOAD_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
COUNTING_LIST_PAYLOAD_TYPE.setStored(false);
|
||||
COUNTING_LIST_PAYLOAD_TYPE.setOmitNorms(true);
|
||||
COUNTING_LIST_PAYLOAD_TYPE.freeze();
|
||||
}
|
||||
|
||||
public PayloadFacetFields(TaxonomyWriter taxonomyWriter, FacetIndexingParams params) {
|
||||
super(taxonomyWriter, params);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FieldType drillDownFieldType() {
|
||||
// Since the payload is indexed in the same field as the drill-down terms,
|
||||
// we must set IndexOptions to DOCS_AND_FREQS_AND_POSITIONS
|
||||
final FieldType type = new FieldType(TextField.TYPE_NOT_STORED);
|
||||
type.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
|
||||
type.freeze();
|
||||
return type;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addCountingListData(Document doc, Map<String,BytesRef> categoriesData, String field) {
|
||||
CountingListStream ts = new CountingListStream(categoriesData);
|
||||
doc.add(new Field(field, ts, COUNTING_LIST_PAYLOAD_TYPE));
|
||||
}
|
||||
}
|
||||
|
||||
private static final String[] DIMENSIONS = new String[] { "dim1", "dim2", "dim3.1", "dim3.2" };
|
||||
|
||||
private HashMap<String,Integer> createIndex(Directory indexDir, Directory taxoDir, FacetIndexingParams fip)
|
||||
throws Exception {
|
||||
Random random = random();
|
||||
IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
|
||||
conf.setMaxBufferedDocs(2); // force few segments
|
||||
conf.setMergePolicy(NoMergePolicy.COMPOUND_FILES); // avoid merges so that we're left with few segments
|
||||
IndexWriter indexWriter = new IndexWriter(indexDir, conf);
|
||||
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
|
||||
|
||||
FacetFields facetFields = new PayloadFacetFields(taxoWriter, fip);
|
||||
|
||||
HashMap<String,Integer> expectedCounts = new HashMap<String,Integer>(DIMENSIONS.length);
|
||||
int numDocs = atLeast(10);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
Document doc = new Document();
|
||||
int numCategories = random.nextInt(3) + 1;
|
||||
ArrayList<CategoryPath> categories = new ArrayList<CategoryPath>(numCategories);
|
||||
HashSet<String> docDimensions = new HashSet<String>();
|
||||
while (numCategories-- > 0) {
|
||||
String dim = DIMENSIONS[random.nextInt(DIMENSIONS.length)];
|
||||
// we should only increment the expected count by 1 per document
|
||||
docDimensions.add(dim);
|
||||
categories.add(new CategoryPath(dim, Integer.toString(i), Integer.toString(numCategories)));
|
||||
}
|
||||
facetFields.addFields(doc, categories);
|
||||
doc.add(new StringField("docid", Integer.toString(i), Store.YES));
|
||||
doc.add(new TextField("foo", "content" + i, Store.YES));
|
||||
indexWriter.addDocument(doc);
|
||||
|
||||
// update expected count per dimension
|
||||
for (String dim : docDimensions) {
|
||||
Integer val = expectedCounts.get(dim);
|
||||
if (val == null) {
|
||||
expectedCounts.put(dim, Integer.valueOf(1));
|
||||
} else {
|
||||
expectedCounts.put(dim, Integer.valueOf(val.intValue() + 1));
|
||||
}
|
||||
}
|
||||
|
||||
if (random.nextDouble() < 0.2) { // add some documents that will be deleted
|
||||
doc = new Document();
|
||||
doc.add(new StringField("del", "key", Store.NO));
|
||||
facetFields.addFields(doc, Collections.singletonList(new CategoryPath("dummy")));
|
||||
indexWriter.addDocument(doc);
|
||||
}
|
||||
}
|
||||
|
||||
indexWriter.commit();
|
||||
taxoWriter.commit();
|
||||
|
||||
// delete the docs that were marked for deletion. note that the 'dummy'
|
||||
// category is not removed from the taxonomy, so must account for it when we
|
||||
// verify the migrated index.
|
||||
indexWriter.deleteDocuments(new Term("del", "key"));
|
||||
indexWriter.commit();
|
||||
|
||||
IOUtils.close(indexWriter, taxoWriter);
|
||||
|
||||
return expectedCounts;
|
||||
}
|
||||
|
||||
private void migrateIndex(Directory indexDir, FacetIndexingParams fip) throws Exception {
|
||||
final Map<String,Term> fieldTerms = FacetsPayloadMigrationReader.buildFieldTermsMap(indexDir, fip);
|
||||
DirectoryReader reader = DirectoryReader.open(indexDir);
|
||||
List<AtomicReaderContext> leaves = reader.leaves();
|
||||
int numReaders = leaves.size();
|
||||
AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];
|
||||
for (int i = 0; i < numReaders; i++) {
|
||||
wrappedLeaves[i] = new FacetsPayloadMigrationReader(leaves.get(i).reader(), fieldTerms);
|
||||
}
|
||||
|
||||
IndexWriter writer = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, null));
|
||||
writer.deleteAll();
|
||||
try {
|
||||
writer.addIndexes(new MultiReader(wrappedLeaves));
|
||||
writer.commit();
|
||||
} finally {
|
||||
reader.close();
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyMigratedIndex(Directory indexDir, Directory taxoDir, HashMap<String,Integer> expectedCounts,
|
||||
FacetIndexingParams fip) throws Exception {
|
||||
DirectoryReader indexReader = DirectoryReader.open(indexDir);
|
||||
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
|
||||
IndexSearcher searcher = new IndexSearcher(indexReader);
|
||||
|
||||
assertFalse("index should not have deletions", indexReader.hasDeletions());
|
||||
|
||||
verifyNotFacetsData(indexReader, searcher);
|
||||
verifyFacetedSearch(expectedCounts, fip, indexReader, taxoReader, searcher);
|
||||
verifyDrillDown(expectedCounts, fip, indexReader, taxoReader, searcher);
|
||||
verifyIndexOrdinals(indexReader, taxoReader, fip);
|
||||
|
||||
IOUtils.close(indexReader, taxoReader);
|
||||
}
|
||||
|
||||
private void verifyNotFacetsData(DirectoryReader indexReader, IndexSearcher searcher) throws IOException {
|
||||
// verify that non facets data was not damaged
|
||||
TotalHitCountCollector total = new TotalHitCountCollector();
|
||||
searcher.search(new PrefixQuery(new Term("foo", "content")), total);
|
||||
assertEquals("invalid number of results for content query", total.getTotalHits(), indexReader.maxDoc());
|
||||
|
||||
int numDocIDs = 0;
|
||||
for (AtomicReaderContext context : indexReader.leaves()) {
|
||||
Terms docIDs = context.reader().terms("docid");
|
||||
assertNotNull(docIDs);
|
||||
TermsEnum te = docIDs.iterator(null);
|
||||
while (te.next() != null) {
|
||||
++numDocIDs;
|
||||
}
|
||||
}
|
||||
assertEquals("invalid number of docid terms", indexReader.maxDoc(), numDocIDs);
|
||||
}
|
||||
|
||||
private void verifyFacetedSearch(Map<String,Integer> expectedCounts, FacetIndexingParams fip,
|
||||
DirectoryReader indexReader, TaxonomyReader taxoReader, IndexSearcher searcher) throws IOException {
|
||||
// run faceted search and assert expected counts
|
||||
ArrayList<FacetRequest> requests = new ArrayList<FacetRequest>(expectedCounts.size());
|
||||
for (String dim : expectedCounts.keySet()) {
|
||||
requests.add(new CountFacetRequest(new CategoryPath(dim), 5));
|
||||
}
|
||||
FacetSearchParams fsp = new FacetSearchParams(requests, fip);
|
||||
FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
|
||||
MatchAllDocsQuery base = new MatchAllDocsQuery();
|
||||
searcher.search(base, fc);
|
||||
List<FacetResult> facetResults = fc.getFacetResults();
|
||||
assertEquals(requests.size(), facetResults.size());
|
||||
for (FacetResult res : facetResults) {
|
||||
FacetResultNode node = res.getFacetResultNode();
|
||||
String dim = node.getLabel().components[0];
|
||||
assertEquals("wrong count for " + dim, expectedCounts.get(dim).intValue(), (int) node.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyDrillDown(Map<String,Integer> expectedCounts, FacetIndexingParams fip, DirectoryReader indexReader,
|
||||
TaxonomyReader taxoReader, IndexSearcher searcher) throws IOException {
|
||||
// verify drill-down
|
||||
for (String dim : expectedCounts.keySet()) {
|
||||
CategoryPath drillDownCP = new CategoryPath(dim);
|
||||
ArrayList<FacetRequest> request = new ArrayList<FacetRequest>(1);
|
||||
request.add(new CountFacetRequest(drillDownCP, 10));
|
||||
FacetSearchParams fsp = new FacetSearchParams(request, fip);
|
||||
Query drillDown = DrillDown.query(fsp, new MatchAllDocsQuery(), drillDownCP);
|
||||
TotalHitCountCollector total = new TotalHitCountCollector();
|
||||
FacetsCollector fc = new FacetsCollector(fsp, indexReader, taxoReader);
|
||||
searcher.search(drillDown, MultiCollector.wrap(fc, total));
|
||||
assertTrue("no results for drill-down query " + drillDown, total.getTotalHits() > 0);
|
||||
List<FacetResult> facetResults = fc.getFacetResults();
|
||||
assertEquals(1, facetResults.size());
|
||||
FacetResultNode rootNode = facetResults.get(0).getFacetResultNode();
|
||||
assertEquals("wrong count for " + dim, expectedCounts.get(dim).intValue(), (int) rootNode.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyIndexOrdinals(DirectoryReader indexReader, TaxonomyReader taxoReader, FacetIndexingParams fip)
|
||||
throws IOException {
|
||||
// verify that the ordinals in the index match the ones in the taxonomy, and vice versa
|
||||
|
||||
// collect all fields which have DocValues, to assert later that all were
|
||||
// visited i.e. that during migration we didn't add FieldInfos with no
|
||||
// DocValues
|
||||
HashSet<String> docValuesFields = new HashSet<String>();
|
||||
for (AtomicReaderContext context : indexReader.leaves()) {
|
||||
FieldInfos infos = context.reader().getFieldInfos();
|
||||
for (FieldInfo info : infos) {
|
||||
if (info.hasDocValues()) {
|
||||
docValuesFields.add(info.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// check that all visited ordinals are found in the taxonomy and vice versa
|
||||
boolean[] foundOrdinals = new boolean[taxoReader.getSize()];
|
||||
for (int i = 0; i < foundOrdinals.length; i++) {
|
||||
foundOrdinals[i] = false; // init to be on the safe side
|
||||
}
|
||||
foundOrdinals[0] = true; // ROOT ordinals isn't indexed
|
||||
// mark 'dummy' category ordinal as seen
|
||||
int dummyOrdinal = taxoReader.getOrdinal(new CategoryPath("dummy"));
|
||||
if (dummyOrdinal > 0) {
|
||||
foundOrdinals[dummyOrdinal] = true;
|
||||
}
|
||||
|
||||
int partitionSize = fip.getPartitionSize();
|
||||
int numPartitions = (int) Math.ceil(taxoReader.getSize() / (double) partitionSize);
|
||||
final IntsRef ordinals = new IntsRef(32);
|
||||
for (String dim : DIMENSIONS) {
|
||||
CategoryListParams clp = fip.getCategoryListParams(new CategoryPath(dim));
|
||||
int partitionOffset = 0;
|
||||
for (int partition = 0; partition < numPartitions; partition++, partitionOffset += partitionSize) {
|
||||
final CategoryListIterator cli = clp.createCategoryListIterator(partition);
|
||||
for (AtomicReaderContext context : indexReader.leaves()) {
|
||||
if (cli.setNextReader(context)) { // not all fields may exist in all segments
|
||||
// remove that field from the list of DocValues fields
|
||||
docValuesFields.remove(clp.field + PartitionsUtils.partitionName(partition));
|
||||
int maxDoc = context.reader().maxDoc();
|
||||
for (int doc = 0; doc < maxDoc; doc++) {
|
||||
cli.getOrdinals(doc, ordinals);
|
||||
for (int j = 0; j < ordinals.length; j++) {
|
||||
// verify that the ordinal is recognized by the taxonomy
|
||||
int ordinal = ordinals.ints[j] + partitionOffset;
|
||||
assertTrue("should not have received dummy ordinal (" + dummyOrdinal + ")", dummyOrdinal != ordinal);
|
||||
assertNotNull("missing category for ordinal " + ordinal, taxoReader.getPath(ordinal));
|
||||
foundOrdinals[ordinal] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue("some fields which have docValues were not visited: " + docValuesFields, docValuesFields.isEmpty());
|
||||
|
||||
for (int i = 0; i < foundOrdinals.length; i++) {
|
||||
assertTrue("ordinal " + i + " not visited", foundOrdinals[i]);
|
||||
}
|
||||
}
|
||||
|
||||
private void doTestMigration(final int partitionSize) throws Exception {
|
||||
// create a facets index with PayloadFacetFields and check it after migration
|
||||
Directory indexDir = newDirectory();
|
||||
Directory taxoDir = newDirectory();
|
||||
|
||||
// set custom CLP fields for two dimensions and use the default ($facets) for the other two
|
||||
HashMap<CategoryPath,CategoryListParams> params = new HashMap<CategoryPath,CategoryListParams>();
|
||||
params.put(new CategoryPath(DIMENSIONS[0]), new CategoryListParams(DIMENSIONS[0]));
|
||||
params.put(new CategoryPath(DIMENSIONS[1]), new CategoryListParams(DIMENSIONS[1]));
|
||||
FacetIndexingParams fip = new PerDimensionIndexingParams(params) {
|
||||
@Override
|
||||
public int getPartitionSize() {
|
||||
return partitionSize;
|
||||
}
|
||||
};
|
||||
|
||||
HashMap<String,Integer> expectedCounts = createIndex(indexDir, taxoDir, fip);
|
||||
migrateIndex(indexDir, fip);
|
||||
verifyMigratedIndex(indexDir, taxoDir, expectedCounts, fip);
|
||||
|
||||
IOUtils.close(indexDir, taxoDir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMigration() throws Exception {
|
||||
doTestMigration(Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMigrationWithPartitions() throws Exception {
|
||||
doTestMigration(2);
|
||||
}
|
||||
|
||||
}
|
|
@ -1,10 +1,12 @@
|
|||
package org.apache.lucene.facet.index.params;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.facet.index.params.CategoryListParams;
|
||||
import org.apache.lucene.util.encoding.DGapVInt8IntEncoder;
|
||||
import org.apache.lucene.util.encoding.IntDecoder;
|
||||
import org.apache.lucene.util.encoding.IntEncoder;
|
||||
import org.apache.lucene.util.encoding.SortingIntEncoder;
|
||||
import org.apache.lucene.util.encoding.UniqueValuesIntEncoder;
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -28,9 +30,11 @@ public class CategoryListParamsTest extends LuceneTestCase {
|
|||
@Test
|
||||
public void testDefaultSettings() {
|
||||
CategoryListParams clp = new CategoryListParams();
|
||||
assertEquals("wrong default term", new Term("$facets", "$fulltree$"), clp.getTerm());
|
||||
assertEquals("unexpected default encoder", "Sorting (Unique (DGap (VInt8)))", clp.createEncoder().toString());
|
||||
assertEquals("unexpected default decoder", "DGap (VInt8)", clp.createEncoder().createMatchingDecoder().toString());
|
||||
assertEquals("wrong default field", "$facets", clp.field);
|
||||
IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapVInt8IntEncoder()));
|
||||
IntDecoder decoder = encoder.createMatchingDecoder();
|
||||
assertEquals("unexpected default encoder", encoder.toString(), clp.createEncoder().toString());
|
||||
assertEquals("unexpected default decoder", decoder.toString(), clp.createEncoder().createMatchingDecoder().toString());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -64,8 +68,8 @@ public class CategoryListParamsTest extends LuceneTestCase {
|
|||
clParams1.hashCode(), clParams2.hashCode());
|
||||
|
||||
// Test 2 CategoryListParams with the same specified Term
|
||||
clParams1 = new CategoryListParams(new Term("test"));
|
||||
clParams2 = new CategoryListParams(new Term("test"));
|
||||
clParams1 = new CategoryListParams("test");
|
||||
clParams2 = new CategoryListParams("test");
|
||||
assertEquals(
|
||||
"2 CategoryListParams with the same term should equal each other.",
|
||||
clParams1, clParams2);
|
||||
|
@ -73,8 +77,8 @@ public class CategoryListParamsTest extends LuceneTestCase {
|
|||
clParams1.hashCode(), clParams2.hashCode());
|
||||
|
||||
// Test 2 CategoryListParams with DIFFERENT terms
|
||||
clParams1 = new CategoryListParams(new Term("test1"));
|
||||
clParams2 = new CategoryListParams(new Term("test2"));
|
||||
clParams1 = new CategoryListParams("test1");
|
||||
clParams2 = new CategoryListParams("test2");
|
||||
assertFalse(
|
||||
"2 CategoryListParams with the different terms should NOT equal each other.",
|
||||
clParams1.equals(clParams2));
|
||||
|
|
|
@ -35,8 +35,7 @@ public class FacetIndexingParamsTest extends LuceneTestCase {
|
|||
assertNotNull("Missing default category list", dfip.getAllCategoryListParams());
|
||||
assertEquals("all categories have the same CategoryListParams by default",
|
||||
dfip.getCategoryListParams(null), dfip.getCategoryListParams(new CategoryPath("a")));
|
||||
assertEquals("Expected default category list term is $facets:$fulltree$",
|
||||
new Term("$facets", "$fulltree$"), dfip.getCategoryListParams(null).getTerm());
|
||||
assertEquals("Expected default category list field is $facets", "$facets", dfip.getCategoryListParams(null).field);
|
||||
String expectedDDText = "a"
|
||||
+ dfip.getFacetDelimChar() + "b";
|
||||
CategoryPath cp = new CategoryPath("a", "b");
|
||||
|
@ -48,13 +47,13 @@ public class FacetIndexingParamsTest extends LuceneTestCase {
|
|||
assertEquals("wrong drill-down term text", expectedDDText, new String(
|
||||
buf, 0, numchars));
|
||||
CategoryListParams clParams = dfip.getCategoryListParams(null);
|
||||
assertEquals("partition for all ordinals is the first", "$fulltree$",
|
||||
PartitionsUtils.partitionNameByOrdinal(dfip, clParams , 250));
|
||||
assertEquals("partition for all ordinals is the first", "",
|
||||
PartitionsUtils.partitionNameByOrdinal(dfip, 250));
|
||||
assertEquals("for partition 0, the same name should be returned",
|
||||
"$fulltree$", PartitionsUtils.partitionName(clParams, 0));
|
||||
"", PartitionsUtils.partitionName(0));
|
||||
assertEquals(
|
||||
"for any other, it's the concatenation of name + partition",
|
||||
"$fulltree$1", PartitionsUtils.partitionName(clParams, 1));
|
||||
PartitionsUtils.PART_NAME_PREFIX + "1", PartitionsUtils.partitionName(1));
|
||||
assertEquals("default partition number is always 0", 0,
|
||||
PartitionsUtils.partitionNumber(dfip,100));
|
||||
assertEquals("default partition size is unbounded", Integer.MAX_VALUE,
|
||||
|
@ -63,11 +62,9 @@ public class FacetIndexingParamsTest extends LuceneTestCase {
|
|||
|
||||
@Test
|
||||
public void testCategoryListParamsWithDefaultIndexingParams() {
|
||||
CategoryListParams clp = new CategoryListParams(
|
||||
new Term("clp", "value"));
|
||||
CategoryListParams clp = new CategoryListParams("clp");
|
||||
FacetIndexingParams dfip = new FacetIndexingParams(clp);
|
||||
assertEquals("Expected default category list term is " + clp.getTerm(),
|
||||
clp.getTerm(), dfip.getCategoryListParams(null).getTerm());
|
||||
assertEquals("Expected default category list field is " + clp.field, clp.field, dfip.getCategoryListParams(null).field);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -32,44 +32,31 @@ public class PerDimensionIndexingParamsTest extends LuceneTestCase {
|
|||
public void testTopLevelSettings() {
|
||||
FacetIndexingParams ifip = new PerDimensionIndexingParams(Collections.<CategoryPath, CategoryListParams>emptyMap());
|
||||
assertNotNull("Missing default category list", ifip.getAllCategoryListParams());
|
||||
assertEquals(
|
||||
"Expected default category list term is $facets:$fulltree$",
|
||||
new Term("$facets", "$fulltree$"), ifip.getCategoryListParams(
|
||||
null).getTerm());
|
||||
String expectedDDText = "a"
|
||||
+ ifip.getFacetDelimChar() + "b";
|
||||
assertEquals("Expected default category list field is $facets", "$facets", ifip.getCategoryListParams(null).field);
|
||||
String expectedDDText = "a" + ifip.getFacetDelimChar() + "b";
|
||||
CategoryPath cp = new CategoryPath("a", "b");
|
||||
assertEquals("wrong drill-down term", new Term("$facets",
|
||||
expectedDDText), DrillDown.term(ifip,cp));
|
||||
assertEquals("wrong drill-down term", new Term("$facets", expectedDDText), DrillDown.term(ifip,cp));
|
||||
char[] buf = new char[20];
|
||||
int numchars = ifip.drillDownTermText(cp, buf);
|
||||
assertEquals("3 characters should be written", 3, numchars);
|
||||
assertEquals("wrong drill-down term text", expectedDDText, new String(
|
||||
buf, 0, numchars));
|
||||
assertEquals("wrong drill-down term text", expectedDDText, new String(buf, 0, numchars));
|
||||
|
||||
CategoryListParams clParams = ifip.getCategoryListParams(null);
|
||||
assertEquals("partition for all ordinals is the first", "$fulltree$",
|
||||
PartitionsUtils.partitionNameByOrdinal(ifip, clParams , 250));
|
||||
assertEquals("for partition 0, the same name should be returned",
|
||||
"$fulltree$", PartitionsUtils.partitionName(clParams, 0));
|
||||
assertEquals(
|
||||
"for any other, it's the concatenation of name + partition",
|
||||
"$fulltree$1", PartitionsUtils.partitionName(clParams, 1));
|
||||
assertEquals("default partition number is always 0", 0,
|
||||
PartitionsUtils.partitionNumber(ifip,100));
|
||||
|
||||
assertEquals("default partition size is unbounded", Integer.MAX_VALUE,
|
||||
ifip.getPartitionSize());
|
||||
assertEquals("partition for all ordinals is the first", "", PartitionsUtils.partitionNameByOrdinal(ifip, 250));
|
||||
assertEquals("for partition 0, the same name should be returned", "", PartitionsUtils.partitionName(0));
|
||||
assertEquals("for any other, it's the concatenation of name + partition", PartitionsUtils.PART_NAME_PREFIX + "1", PartitionsUtils.partitionName(1));
|
||||
assertEquals("default partition number is always 0", 0, PartitionsUtils.partitionNumber(ifip,100));
|
||||
assertEquals("default partition size is unbounded", Integer.MAX_VALUE, ifip.getPartitionSize());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCategoryListParamsAddition() {
|
||||
CategoryListParams clp = new CategoryListParams(new Term("clp", "value"));
|
||||
CategoryListParams clp = new CategoryListParams("clp");
|
||||
PerDimensionIndexingParams tlfip = new PerDimensionIndexingParams(
|
||||
Collections.<CategoryPath,CategoryListParams> singletonMap(new CategoryPath("a"), clp));
|
||||
assertEquals("Expected category list term is " + clp.getTerm(),
|
||||
clp.getTerm(), tlfip.getCategoryListParams(new CategoryPath("a")).getTerm());
|
||||
assertNotSame("Unexpected default category list " + clp.getTerm(), clp, tlfip.getCategoryListParams(null));
|
||||
assertEquals("Expected category list field is " + clp.field,
|
||||
clp.field, tlfip.getCategoryListParams(new CategoryPath("a")).field);
|
||||
assertNotSame("Unexpected default category list " + clp.field, clp, tlfip.getCategoryListParams(null));
|
||||
}
|
||||
|
||||
}
|
|
@ -1,23 +1,15 @@
|
|||
package org.apache.lucene.facet.search;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.analysis.MockTokenizer;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.document.StraightBytesDocValuesField;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IntsRef;
|
||||
|
@ -48,42 +40,6 @@ import org.junit.Test;
|
|||
|
||||
public class CategoryListIteratorTest extends LuceneTestCase {
|
||||
|
||||
private static final class DataTokenStream extends TokenStream {
|
||||
|
||||
private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
|
||||
private final BytesRef buf;
|
||||
private final IntEncoder encoder;
|
||||
private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
|
||||
|
||||
private int idx;
|
||||
private boolean exhausted = false;
|
||||
|
||||
public DataTokenStream(String text, IntEncoder encoder) {
|
||||
this.encoder = encoder;
|
||||
term.setEmpty().append(text);
|
||||
buf = new BytesRef();
|
||||
payload.setPayload(buf);
|
||||
}
|
||||
|
||||
public void setIdx(int idx) {
|
||||
this.idx = idx;
|
||||
exhausted = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean incrementToken() throws IOException {
|
||||
if (exhausted) {
|
||||
return false;
|
||||
}
|
||||
|
||||
// must copy because encoders may change the buffer
|
||||
encoder.encode(IntsRef.deepCopyOf(data[idx]), buf);
|
||||
exhausted = true;
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static final IntsRef[] data = new IntsRef[] {
|
||||
new IntsRef(new int[] { 1, 2 }, 0, 2),
|
||||
new IntsRef(new int[] { 3, 4 }, 0, 2),
|
||||
|
@ -95,13 +51,13 @@ public class CategoryListIteratorTest extends LuceneTestCase {
|
|||
public void testPayloadCategoryListIteraor() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
|
||||
DataTokenStream dts = new DataTokenStream("1",encoder);
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
|
||||
BytesRef buf = new BytesRef();
|
||||
for (int i = 0; i < data.length; i++) {
|
||||
dts.setIdx(i);
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("f", dts));
|
||||
encoder.encode(IntsRef.deepCopyOf(data[i]), buf);
|
||||
doc.add(new StraightBytesDocValuesField("f", buf));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
IndexReader reader = writer.getReader();
|
||||
|
@ -109,9 +65,9 @@ public class CategoryListIteratorTest extends LuceneTestCase {
|
|||
|
||||
int totalCategories = 0;
|
||||
IntsRef ordinals = new IntsRef();
|
||||
CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
|
||||
CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
cli.setNextReader(context);
|
||||
assertTrue("failed to initalize iterator", cli.setNextReader(context));
|
||||
int maxDoc = context.reader().maxDoc();
|
||||
int dataIdx = context.docBase;
|
||||
for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
|
||||
|
@ -136,24 +92,17 @@ public class CategoryListIteratorTest extends LuceneTestCase {
|
|||
public void testPayloadIteratorWithInvalidDoc() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
|
||||
DataTokenStream dts = new DataTokenStream("1", encoder);
|
||||
// this test requires that no payloads ever be randomly present!
|
||||
final Analyzer noPayloadsAnalyzer = new Analyzer() {
|
||||
@Override
|
||||
public TokenStreamComponents createComponents(String fieldName, Reader reader) {
|
||||
return new TokenStreamComponents(new MockTokenizer(reader, MockTokenizer.KEYWORD, false));
|
||||
}
|
||||
};
|
||||
// NOTE: test is wired to LogMP... because test relies on certain docids having payloads
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, noPayloadsAnalyzer).setMergePolicy(newLogMergePolicy()));
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
|
||||
for (int i = 0; i < data.length; i++) {
|
||||
Document doc = new Document();
|
||||
if (i == 0) {
|
||||
dts.setIdx(i);
|
||||
doc.add(new TextField("f", dts)); // only doc 0 has payloads!
|
||||
BytesRef buf = new BytesRef();
|
||||
encoder.encode(IntsRef.deepCopyOf(data[i]), buf );
|
||||
doc.add(new StraightBytesDocValuesField("f", buf));
|
||||
} else {
|
||||
doc.add(new TextField("f", "1", Field.Store.NO));
|
||||
doc.add(new StraightBytesDocValuesField("f", new BytesRef()));
|
||||
}
|
||||
writer.addDocument(doc);
|
||||
writer.commit();
|
||||
|
@ -164,9 +113,9 @@ public class CategoryListIteratorTest extends LuceneTestCase {
|
|||
|
||||
int totalCategories = 0;
|
||||
IntsRef ordinals = new IntsRef();
|
||||
CategoryListIterator cli = new PayloadCategoryListIteraor(new Term("f","1"), encoder.createMatchingDecoder());
|
||||
CategoryListIterator cli = new DocValuesCategoryListIterator("f", encoder.createMatchingDecoder());
|
||||
for (AtomicReaderContext context : reader.leaves()) {
|
||||
cli.setNextReader(context);
|
||||
assertTrue("failed to initalize iterator", cli.setNextReader(context));
|
||||
int maxDoc = context.reader().maxDoc();
|
||||
int dataIdx = context.docBase;
|
||||
for (int doc = 0; doc < maxDoc; doc++, dataIdx++) {
|
||||
|
@ -176,13 +125,13 @@ public class CategoryListIteratorTest extends LuceneTestCase {
|
|||
}
|
||||
cli.getOrdinals(doc, ordinals);
|
||||
if (dataIdx == 0) {
|
||||
assertTrue("document 0 must have a payload", ordinals.length > 0);
|
||||
assertTrue("document 0 must have ordinals", ordinals.length > 0);
|
||||
for (int j = 0; j < ordinals.length; j++) {
|
||||
assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
|
||||
}
|
||||
totalCategories += ordinals.length;
|
||||
} else {
|
||||
assertTrue("only document 0 should have a payload", ordinals.length == 0);
|
||||
assertTrue("only document 0 should have ordinals", ordinals.length == 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue