Skip to content

Commit

Permalink
Term Vectors API: Computes term vectors on the fly if not stored in t…
Browse files Browse the repository at this point in the history
…he index.

Adds the ability to the Term Vector API to generate term vectors for some
chosen fields, even though they haven't been explicitely stored in the index.

Relates to elastic#5184
Closes elastic#6567
  • Loading branch information
alexksikes committed Jul 17, 2014
1 parent 6a25d9b commit f22f3db
Show file tree
Hide file tree
Showing 4 changed files with 399 additions and 75 deletions.
27 changes: 24 additions & 3 deletions docs/reference/docs/termvectors.asciidoc
Expand Up @@ -19,7 +19,7 @@ retrieved either with a parameter in the url
curl -XGET 'http://localhost:9200/twitter/tweet/1/_termvector?fields=text,...'
--------------------------------------------------

or adding by adding the requested fields in the request body (see
or by adding the requested fields in the request body (see
example below).

[float]
Expand All @@ -38,9 +38,11 @@ statistics are returned for all fields but no term statistics.
* term payloads (`payloads` : true), as base64 encoded bytes

If the requested information wasn't stored in the index, it will be
omitted without further warning. See <<mapping-types,type mapping>>
computed on the fly if possible. See <<mapping-types,type mapping>>
for how to configure your index to store term vectors.

coming[1.4.0,The ability to computed term vectors on the fly is only available from 1.4.0 onwards (see below)]

[WARNING]
======
Start and end offsets assume UTF-16 encoding is being used. If you want to use
Expand Down Expand Up @@ -84,7 +86,7 @@ are therefore only useful as relative measures whereas the absolute
numbers have no meaning in this context.

[float]
=== Example
=== Example 1

First, we create an index that stores term vectors, payloads etc. :

Expand Down Expand Up @@ -222,3 +224,22 @@ Response:
}
}
--------------------------------------------------

[float]
=== Example 2 coming[1.4.0]

Additionally, term vectors which are not explicitly stored in the index are automatically
computed on the fly. The following request returns all information and statistics for the
fields in document `1`, even though the terms haven't been explicitly stored in the index.
Note that for the field `text`, the terms are not re-generated.

[source,js]
--------------------------------------------------
curl -XGET 'http://localhost:9200/twitter/tweet/1/_termvector?pretty=true' -d '{
"fields" : ["text", "some_field_without_term_vectors"],
"offsets" : true,
"positions" : true,
"term_statistics" : true,
"field_statistics" : true
}'
--------------------------------------------------
Expand Up @@ -197,23 +197,23 @@ private void startTerm(BytesRef term) throws IOException {

private void writeTermStatistics(TermsEnum topLevelIterator) throws IOException {
int docFreq = topLevelIterator.docFreq();
assert (docFreq >= 0);
assert (docFreq >= -1);
writePotentiallyNegativeVInt(docFreq);
long ttf = topLevelIterator.totalTermFreq();
assert (ttf >= 0);
assert (ttf >= -1);
writePotentiallyNegativeVLong(ttf);

}

private void writeFieldStatistics(Terms topLevelTerms) throws IOException {
long sttf = topLevelTerms.getSumTotalTermFreq();
assert (sttf >= 0);
assert (sttf >= -1);
writePotentiallyNegativeVLong(sttf);
long sdf = topLevelTerms.getSumDocFreq();
assert (sdf >= 0);
assert (sdf >= -1);
writePotentiallyNegativeVLong(sdf);
int dc = topLevelTerms.getDocCount();
assert (dc >= 0);
assert (dc >= -1);
writePotentiallyNegativeVInt(dc);

}
Expand Down
Expand Up @@ -19,35 +19,40 @@

package org.elasticsearch.index.termvectors;

import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.Term;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.*;
import org.apache.lucene.index.memory.MemoryIndex;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.termvector.TermVectorRequest;
import org.elasticsearch.action.termvector.TermVectorResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.get.GetField;
import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard;

import java.io.IOException;
import java.util.*;

/**
*/

public class ShardTermVectorService extends AbstractIndexShardComponent {

private IndexShard indexShard;
private MapperService mapperService;

@Inject
public ShardTermVectorService(ShardId shardId, @IndexSettings Settings indexSettings, MapperService mapperService) {
public ShardTermVectorService(ShardId shardId, @IndexSettings Settings indexSettings) {
super(shardId, indexSettings);
}

Expand All @@ -66,8 +71,11 @@ public TermVectorResponse getTermVector(TermVectorRequest request) {
Fields topLevelFields = MultiFields.getFields(topLevelReader);
Versions.DocIdAndVersion docIdAndVersion = Versions.loadDocIdAndVersion(topLevelReader, uidTerm);
if (docIdAndVersion != null) {

Fields termVectorsByField = docIdAndVersion.context.reader().getTermVectors(docIdAndVersion.docId);
/* generate term vectors if not available */
if (request.selectedFields() != null) {
termVectorsByField = generateTermVectorsIfNeeded(termVectorsByField, request, uidTerm, false);
}
termVectorResponse.setFields(termVectorsByField, request.selectedFields(), request.getFlags(), topLevelFields);
termVectorResponse.setExists(true);
termVectorResponse.setDocVersion(docIdAndVersion.version);
Expand All @@ -81,4 +89,102 @@ public TermVectorResponse getTermVector(TermVectorRequest request) {
}
return termVectorResponse;
}

private Fields generateTermVectorsIfNeeded(Fields termVectorsByField, TermVectorRequest request, Term uidTerm, boolean realTime) throws IOException {
List<String> validFields = new ArrayList<>();
for (String field : request.selectedFields()) {
FieldMapper fieldMapper = indexShard.mapperService().smartNameFieldMapper(field);
if (!(fieldMapper instanceof StringFieldMapper)) {
continue;
}
if (fieldMapper.fieldType().storeTermVectors()) {
continue;
}
// only disallow fields which are not indexed
if (!fieldMapper.fieldType().indexed()) {
continue;
}
validFields.add(field);
}
if (validFields.isEmpty()) {
return termVectorsByField;
}

Engine.GetResult get = indexShard.get(new Engine.Get(realTime, uidTerm));
Fields generatedTermVectors;
try {
if (!get.exists()) {
return termVectorsByField;
}
// TODO: support for fetchSourceContext?
GetResult getResult = indexShard.getService().get(
get, request.id(), request.type(), validFields.toArray(Strings.EMPTY_ARRAY), null);
generatedTermVectors = generateTermVectors(getResult.getFields().values(), request.offsets());
} finally {
get.release();
}
if (termVectorsByField == null) {
return generatedTermVectors;
} else {
return mergeFields(request.selectedFields().toArray(Strings.EMPTY_ARRAY), termVectorsByField, generatedTermVectors);
}
}

private Fields generateTermVectors(Collection<GetField> getFields, boolean withOffsets) throws IOException {
// store document in memory index
MemoryIndex index = new MemoryIndex(withOffsets);
for (GetField getField : getFields) {
String field = getField.getName();
Analyzer analyzer = indexShard.mapperService().smartNameFieldMapper(field).indexAnalyzer();
if (analyzer == null) {
analyzer = indexShard.mapperService().analysisService().defaultIndexAnalyzer();
}
for (Object text : getField.getValues()) {
index.addField(field, text.toString(), analyzer);
}
}
// and read vectors from it
return MultiFields.getFields(index.createSearcher().getIndexReader());
}

private Fields mergeFields(String[] fieldNames, Fields... fieldsObject) throws IOException {
ParallelFields parallelFields = new ParallelFields();
for (Fields fieldObject : fieldsObject) {
assert fieldObject != null;
for (String fieldName : fieldNames) {
Terms terms = fieldObject.terms(fieldName);
if (terms != null) {
parallelFields.addField(fieldName, terms);
}
}
}
return parallelFields;
}

// Poached from Lucene ParallelAtomicReader
private static final class ParallelFields extends Fields {
final Map<String,Terms> fields = new TreeMap<>();

ParallelFields() {
}

void addField(String fieldName, Terms terms) {
fields.put(fieldName, terms);
}

@Override
public Iterator<String> iterator() {
return Collections.unmodifiableSet(fields.keySet()).iterator();
}

@Override
public Terms terms(String field) {
return fields.get(field);
}

@Override
public int size() {
return fields.size();
}
}
}

0 comments on commit f22f3db

Please sign in to comment.