浏览代码

Remove script access to term statistics (#19462)

In scripts (at least some of the languages), the terms dictionary and
postings can be access with the special _index variable. This is for
very advanced use cases which want to do their own scoring. The problem
is segment level statistics must be recomputed for every document.
Additionally, this is not friendly to the terms index caching as the
order of looking up terms should be controlled by lucene.

This change removes _index from scripts. Anyone using it can and should
instead write a Similarity plugin, which is explicitly designed to allow
doing the calculations needed for a relevance score.

closes #19359
Ryan Ernst 8 年之前
父节点
当前提交
97d2657e18

+ 0 - 8
core/src/main/java/org/elasticsearch/script/AbstractSearchScript.java

@@ -23,7 +23,6 @@ import org.apache.lucene.search.Scorer;
 import org.elasticsearch.index.fielddata.ScriptDocValues;
 import org.elasticsearch.search.lookup.LeafDocLookup;
 import org.elasticsearch.search.lookup.LeafFieldsLookup;
-import org.elasticsearch.search.lookup.LeafIndexLookup;
 import org.elasticsearch.search.lookup.LeafSearchLookup;
 import org.elasticsearch.search.lookup.SourceLookup;
 
@@ -87,13 +86,6 @@ public abstract class AbstractSearchScript extends AbstractExecutableScript impl
         return lookup.source();
     }
 
-    /**
-     * Allows to access statistics on terms and fields.
-     */
-    protected final LeafIndexLookup indexLookup() {
-        return lookup.indexLookup();
-    }
-
     /**
      * Allows to access the *stored* fields.
      */

+ 0 - 132
core/src/main/java/org/elasticsearch/search/lookup/CachedPositionIterator.java

@@ -1,132 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.IntsRefBuilder;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/*
- * Can iterate over the positions of a term an arbitrary number of times.
- * */
-public class CachedPositionIterator extends PositionIterator {
-
-    public CachedPositionIterator(IndexFieldTerm indexFieldTerm) {
-        super(indexFieldTerm);
-    }
-
-    // all payloads of the term in the current document in one bytes array.
-    // payloadStarts and payloadLength mark the start and end of one payload.
-    final BytesRefBuilder payloads = new BytesRefBuilder();
-
-    final IntsRefBuilder payloadsLengths = new IntsRefBuilder();
-
-    final IntsRefBuilder payloadsStarts = new IntsRefBuilder();
-
-    final IntsRefBuilder positions = new IntsRefBuilder();
-
-    final IntsRefBuilder startOffsets = new IntsRefBuilder();
-
-    final IntsRefBuilder endOffsets = new IntsRefBuilder();
-
-    final BytesRef payload = new BytesRef();
-
-    @Override
-    public Iterator<TermPosition> reset() {
-        return new Iterator<TermPosition>() {
-            private int pos = 0;
-            private final TermPosition termPosition = new TermPosition();
-
-            @Override
-            public boolean hasNext() {
-                return pos < freq;
-            }
-
-            @Override
-            public TermPosition next() {
-                termPosition.position = positions.intAt(pos);
-                termPosition.startOffset = startOffsets.intAt(pos);
-                termPosition.endOffset = endOffsets.intAt(pos);
-                termPosition.payload = payload;
-                payload.bytes = payloads.bytes();
-                payload.offset = payloadsStarts.intAt(pos);
-                payload.length = payloadsLengths.intAt(pos);
-                pos++;
-                return termPosition;
-            }
-
-            @Override
-            public void remove() {
-            }
-        };
-    }
-
-
-    private void record() throws IOException {
-        TermPosition termPosition;
-        for (int i = 0; i < freq; i++) {
-            termPosition = super.next();
-            positions.setIntAt(i, termPosition.position);
-            addPayload(i, termPosition.payload);
-            startOffsets.setIntAt(i, termPosition.startOffset);
-            endOffsets.setIntAt(i, termPosition.endOffset);
-        }
-    }
-    private void ensureSize(int freq) {
-        if (freq == 0) {
-            return;
-        }
-        startOffsets.grow(freq);
-        endOffsets.grow(freq);
-        positions.grow(freq);
-        payloadsLengths.grow(freq);
-        payloadsStarts.grow(freq);
-        payloads.grow(freq * 8);// this is just a guess....
-
-    }
-
-    private void addPayload(int i, BytesRef currPayload) {
-        if (currPayload != null) {
-            payloadsLengths.setIntAt(i, currPayload.length);
-            payloadsStarts.setIntAt(i, i == 0 ? 0 : payloadsStarts.intAt(i - 1) + payloadsLengths.intAt(i - 1));
-            payloads.grow(payloadsStarts.intAt(i) + currPayload.length);
-            System.arraycopy(currPayload.bytes, currPayload.offset, payloads.bytes(), payloadsStarts.intAt(i), currPayload.length);
-        } else {
-            payloadsLengths.setIntAt(i, 0);
-            payloadsStarts.setIntAt(i, i == 0 ? 0 : payloadsStarts.intAt(i - 1) + payloadsLengths.intAt(i - 1));
-        }
-    }
-
-
-    @Override
-    public void nextDoc() throws IOException {
-        super.nextDoc();
-        ensureSize(freq);
-        record();
-    }
-
-    @Override
-    public TermPosition next() {
-        throw new UnsupportedOperationException();
-    }
-}

+ 0 - 128
core/src/main/java/org/elasticsearch/search/lookup/IndexField.java

@@ -1,128 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.search.CollectionStatistics;
-import org.elasticsearch.common.util.MinimalMap;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Script interface to all information regarding a field.
- * */
-public class IndexField extends MinimalMap<String, IndexFieldTerm> {
-
-    /*
-     * TermsInfo Objects that represent the Terms are stored in this map when
-     * requested. Information such as frequency, doc frequency and positions
-     * information can be retrieved from the TermInfo objects in this map.
-     */
-    private final Map<String, IndexFieldTerm> terms = new HashMap<>();
-
-    // the name of this field
-    private final String fieldName;
-
-    /*
-     * The holds the current reader. We need it to populate the field
-     * statistics. We just delegate all requests there
-     */
-    private final LeafIndexLookup indexLookup;
-
-    /*
-     * General field statistics such as number of documents containing the
-     * field.
-     */
-    private final CollectionStatistics fieldStats;
-
-    /*
-     * Represents a field in a document. Can be used to return information on
-     * statistics of this field. Information on specific terms in this field can
-     * be accessed by calling get(String term).
-     */
-    public IndexField(String fieldName, LeafIndexLookup indexLookup) throws IOException {
-
-        assert fieldName != null;
-        this.fieldName = fieldName;
-
-        assert indexLookup != null;
-        this.indexLookup = indexLookup;
-
-        fieldStats = this.indexLookup.getIndexSearcher().collectionStatistics(fieldName);
-    }
-
-    /* get number of documents containing the field */
-    public long docCount() throws IOException {
-        return fieldStats.docCount();
-    }
-
-    /* get sum of the number of words over all documents that were indexed */
-    public long sumttf() throws IOException {
-        return fieldStats.sumTotalTermFreq();
-    }
-
-    /*
-     * get the sum of doc frequencies over all words that appear in any document
-     * that has the field.
-     */
-    public long sumdf() throws IOException {
-        return fieldStats.sumDocFreq();
-    }
-
-    // TODO: might be good to get the field lengths here somewhere?
-
-    /*
-     * Returns a TermInfo object that can be used to access information on
-     * specific terms. flags can be set as described in TermInfo.
-     *
-     * TODO: here might be potential for running time improvement? If we knew in
-     * advance which terms are requested, we could provide an array which the
-     * user could then iterate over.
-     */
-    public IndexFieldTerm get(Object key, int flags) {
-        String termString = (String) key;
-        IndexFieldTerm indexFieldTerm = terms.get(termString);
-        // see if we initialized already...
-        if (indexFieldTerm == null) {
-            indexFieldTerm = new IndexFieldTerm(termString, fieldName, indexLookup, flags);
-            terms.put(termString, indexFieldTerm);
-        }
-        indexFieldTerm.validateFlags(flags);
-        return indexFieldTerm;
-    }
-
-    /*
-     * Returns a TermInfo object that can be used to access information on
-     * specific terms. flags can be set as described in TermInfo.
-     */
-    @Override
-    public IndexFieldTerm get(Object key) {
-        // per default, do not initialize any positions info
-        return get(key, IndexLookup.FLAG_FREQUENCIES);
-    }
-
-    public void setDocIdInTerms(int docId) {
-        for (IndexFieldTerm ti : terms.values()) {
-            ti.setDocument(docId);
-        }
-    }
-
-}

+ 0 - 298
core/src/main/java/org/elasticsearch/search/lookup/IndexFieldTerm.java

@@ -1,298 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FilterLeafReader.FilterPostingsEnum;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TermContext;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.TermStatistics;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.ElasticsearchException;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Holds all information on a particular term in a field.
- * */
-public class IndexFieldTerm implements Iterable<TermPosition> {
-
-    // The posting list for this term. Is null if the term or field does not
-    // exist.
-    PostingsEnum postings;
-
-    // Stores if positions, offsets and payloads are requested.
-    private final int flags;
-
-    private final String fieldName;
-
-    private final String term;
-
-    private final PositionIterator iterator;
-
-    // for lucene calls
-    private final Term identifier;
-
-    private final TermStatistics termStats;
-
-    // get the document frequency of the term
-    public long df() throws IOException {
-        return termStats.docFreq();
-    }
-
-    // get the total term frequency of the term, that is, how often does the
-    // term appear in any document?
-    public long ttf() throws IOException {
-        return termStats.totalTermFreq();
-    }
-
-    // when the reader changes, we have to get the posting list for this term
-    // and reader
-    private void setReader(LeafReader reader) {
-        try {
-            postings = getPostings(convertToLuceneFlags(flags), reader);
-
-            if (postings == null) {
-                // no term or field for this segment, fake out the postings...
-                final DocIdSetIterator empty = DocIdSetIterator.empty();
-                postings = new PostingsEnum() {
-                    @Override
-                    public int docID() {
-                        return empty.docID();
-                    }
-
-                    @Override
-                    public int nextDoc() throws IOException {
-                        return empty.nextDoc();
-                    }
-
-                    @Override
-                    public int advance(int target) throws IOException {
-                        return empty.advance(target);
-                    }
-
-                    @Override
-                    public long cost() {
-                        return empty.cost();
-                    }
-
-                    @Override
-                    public int freq() throws IOException {
-                        return 1;
-                    }
-
-                    @Override
-                    public int nextPosition() throws IOException {
-                        return -1;
-                    }
-
-                    @Override
-                    public int startOffset() throws IOException {
-                        return -1;
-                    }
-
-                    @Override
-                    public int endOffset() throws IOException {
-                        return -1;
-                    }
-
-                    @Override
-                    public BytesRef getPayload() throws IOException {
-                        return null;
-                    }
-                };
-            }
-        } catch (IOException e) {
-            throw new ElasticsearchException("Unable to get postings for field " + fieldName + " and term " + term, e);
-        }
-
-    }
-
-    private int convertToLuceneFlags(int flags) {
-        int lucenePositionsFlags = PostingsEnum.NONE;
-        lucenePositionsFlags |= (flags & IndexLookup.FLAG_FREQUENCIES) > 0 ? PostingsEnum.FREQS : 0x0;
-        lucenePositionsFlags |= (flags & IndexLookup.FLAG_POSITIONS) > 0 ? PostingsEnum.POSITIONS : 0x0;
-        lucenePositionsFlags |= (flags & IndexLookup.FLAG_PAYLOADS) > 0 ? PostingsEnum.PAYLOADS : 0x0;
-        lucenePositionsFlags |= (flags & IndexLookup.FLAG_OFFSETS) > 0 ? PostingsEnum.OFFSETS : 0x0;
-        return lucenePositionsFlags;
-    }
-
-    private PostingsEnum getPostings(int luceneFlags, LeafReader reader) throws IOException {
-        assert identifier.field() != null;
-        assert identifier.bytes() != null;
-        final Fields fields = reader.fields();
-        PostingsEnum newPostings = null;
-        if (fields != null) {
-            final Terms terms = fields.terms(identifier.field());
-            if (terms != null) {
-                TermsEnum termsEnum = terms.iterator();
-                if (termsEnum.seekExact(identifier.bytes())) {
-                    newPostings = termsEnum.postings(postings, luceneFlags);
-                    final Bits liveDocs = reader.getLiveDocs();
-                    if (liveDocs != null) {
-                        newPostings = new FilterPostingsEnum(newPostings) {
-                            private int doNext(int d) throws IOException {
-                                for (; d != NO_MORE_DOCS; d = super.nextDoc()) {
-                                    if (liveDocs.get(d)) {
-                                        return d;
-                                    }
-                                }
-                                return NO_MORE_DOCS;
-                            }
-                            @Override
-                            public int nextDoc() throws IOException {
-                                return doNext(super.nextDoc());
-                            }
-                            @Override
-                            public int advance(int target) throws IOException {
-                                return doNext(super.advance(target));
-                            }
-                        };
-                    }
-                }
-            }
-        }
-        return newPostings;
-    }
-
-    private int freq = 0;
-
-    public void setDocument(int docId) {
-        assert (postings != null);
-        try {
-            // we try to advance to the current document.
-            int currentDocPos = postings.docID();
-            if (currentDocPos < docId) {
-                currentDocPos = postings.advance(docId);
-            }
-            if (currentDocPos == docId) {
-                freq = postings.freq();
-            } else {
-                freq = 0;
-            }
-            iterator.nextDoc();
-        } catch (IOException e) {
-            throw new ElasticsearchException("While trying to initialize term positions in IndexFieldTerm.setNextDoc() ", e);
-        }
-    }
-
-    public IndexFieldTerm(String term, String fieldName, LeafIndexLookup indexLookup, int flags) {
-        assert fieldName != null;
-        this.fieldName = fieldName;
-        assert term != null;
-        this.term = term;
-        assert indexLookup != null;
-        identifier = new Term(fieldName, (String) term);
-        this.flags = flags;
-        boolean doRecord = ((flags & IndexLookup.FLAG_CACHE) > 0);
-        if (!doRecord) {
-            iterator = new PositionIterator(this);
-        } else {
-            iterator = new CachedPositionIterator(this);
-        }
-        setReader(indexLookup.getReader());
-        setDocument(indexLookup.getDocId());
-        try {
-            termStats = indexLookup.getIndexSearcher().termStatistics(identifier,
-                    TermContext.build(indexLookup.getReaderContext(), identifier));
-        } catch (IOException e) {
-            throw new ElasticsearchException("Cannot get term statistics: ", e);
-        }
-    }
-
-    public int tf() throws IOException {
-        return freq;
-    }
-
-    @Override
-    public Iterator<TermPosition> iterator() {
-        return iterator.reset();
-    }
-
-    /*
-     * A user might decide inside a script to call get with _POSITIONS and then
-     * a second time with _PAYLOADS. If the positions were recorded but the
-     * payloads were not, the user will not have access to them. Therefore, throw
-     * exception here explaining how to call get().
-     */
-    public void validateFlags(int flags2) {
-        if ((this.flags & flags2) < flags2) {
-            throw new ElasticsearchException("You must call get with all required flags! Instead of " + getCalledStatement(flags2)
-                    + "call " + getCallStatement(flags2 | this.flags) + " once");
-        }
-    }
-
-    private String getCalledStatement(int flags2) {
-        String calledFlagsCall1 = getFlagsString(flags);
-        String calledFlagsCall2 = getFlagsString(flags2);
-        String callStatement1 = getCallStatement(calledFlagsCall1);
-        String callStatement2 = getCallStatement(calledFlagsCall2);
-        return " " + callStatement1 + " and " + callStatement2 + " ";
-    }
-
-    private String getCallStatement(String calledFlags) {
-        return "_index['" + this.fieldName + "'].get('" + this.term + "', " + calledFlags + ")";
-    }
-
-    private String getFlagsString(int flags2) {
-        String flagsString = null;
-        if ((flags2 & IndexLookup.FLAG_FREQUENCIES) != 0) {
-            flagsString = anddToFlagsString(flagsString, "_FREQUENCIES");
-        }
-        if ((flags2 & IndexLookup.FLAG_POSITIONS) != 0) {
-            flagsString = anddToFlagsString(flagsString, "_POSITIONS");
-        }
-        if ((flags2 & IndexLookup.FLAG_OFFSETS) != 0) {
-            flagsString = anddToFlagsString(flagsString, "_OFFSETS");
-        }
-        if ((flags2 & IndexLookup.FLAG_PAYLOADS) != 0) {
-            flagsString = anddToFlagsString(flagsString, "_PAYLOADS");
-        }
-        if ((flags2 & IndexLookup.FLAG_CACHE) != 0) {
-            flagsString = anddToFlagsString(flagsString, "_CACHE");
-        }
-        return flagsString;
-    }
-
-    private String anddToFlagsString(String flagsString, String flag) {
-        if (flagsString != null) {
-            flagsString += " | ";
-        } else {
-            flagsString = "";
-        }
-        flagsString += flag;
-        return flagsString;
-    }
-
-    private String getCallStatement(int flags2) {
-        String calledFlags = getFlagsString(flags2);
-        String callStatement = getCallStatement(calledFlags);
-        return " " + callStatement + " ";
-
-    }
-
-}

+ 0 - 74
core/src/main/java/org/elasticsearch/search/lookup/IndexLookup.java

@@ -1,74 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.index.LeafReaderContext;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static java.util.Collections.unmodifiableMap;
-
-public class IndexLookup {
-    public static final Map<String, Object> NAMES;
-    static {
-        Map<String, Object> names = new HashMap<>();
-        names.put("_FREQUENCIES", IndexLookup.FLAG_FREQUENCIES);
-        names.put("_POSITIONS", IndexLookup.FLAG_POSITIONS);
-        names.put("_OFFSETS", IndexLookup.FLAG_OFFSETS);
-        names.put("_PAYLOADS", IndexLookup.FLAG_PAYLOADS);
-        names.put("_CACHE", IndexLookup.FLAG_CACHE);
-        NAMES = unmodifiableMap(names);
-    }
-    /**
-     * Flag to pass to {@link IndexField#get(Object, int)} if you require
-     * offsets in the returned {@link IndexFieldTerm}.
-     */
-    public static final int FLAG_OFFSETS = 2;
-
-    /**
-     * Flag to pass to {@link IndexField#get(Object, int)} if you require
-     * payloads in the returned {@link IndexFieldTerm}.
-     */
-    public static final int FLAG_PAYLOADS = 4;
-
-    /**
-     * Flag to pass to {@link IndexField#get(Object, int)} if you require
-     * frequencies in the returned {@link IndexFieldTerm}. Frequencies might be
-     * returned anyway for some lucene codecs even if this flag is no set.
-     */
-    public static final int FLAG_FREQUENCIES = 8;
-
-    /**
-     * Flag to pass to {@link IndexField#get(Object, int)} if you require
-     * positions in the returned {@link IndexFieldTerm}.
-     */
-    public static final int FLAG_POSITIONS = 16;
-
-    /**
-     * Flag to pass to {@link IndexField#get(Object, int)} if you require
-     * positions in the returned {@link IndexFieldTerm}.
-     */
-    public static final int FLAG_CACHE = 32;
-
-    public static LeafIndexLookup getLeafIndexLookup(LeafReaderContext context) {
-        return new LeafIndexLookup(context);
-    }
-
-}

+ 0 - 199
core/src/main/java/org/elasticsearch/search/lookup/LeafIndexLookup.java

@@ -1,199 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-package org.elasticsearch.search.lookup;
-
-import org.apache.logging.log4j.Logger;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReaderContext;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.search.IndexSearcher;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.common.logging.DeprecationLogger;
-import org.elasticsearch.common.logging.Loggers;
-import org.elasticsearch.common.util.MinimalMap;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-public class LeafIndexLookup extends MinimalMap<String, IndexField> {
-
-    // Current reader from which we can get the term vectors. No info on term
-    // and field statistics.
-    private final LeafReader reader;
-
-    // The parent reader from which we can get proper field and term
-    // statistics
-    private final IndexReader parentReader;
-
-    // we need this later to get the field and term statistics of the shard
-    private final IndexSearcher indexSearcher;
-
-    // current docId
-    private int docId = -1;
-
-    // stores the objects that are used in the script. we maintain this map
-    // because we do not want to re-initialize the objects each time a field is
-    // accessed
-    private final Map<String, IndexField> indexFields = new HashMap<>();
-
-    // number of documents per shard. cached here because the computation is
-    // expensive
-    private int numDocs = -1;
-
-    // the maximum doc number of the shard.
-    private int maxDoc = -1;
-
-    // number of deleted documents per shard. cached here because the
-    // computation is expensive
-    private int numDeletedDocs = -1;
-
-    private boolean deprecationEmitted = false;
-
-    private void logDeprecation() {
-        if (deprecationEmitted == false) {
-            Logger logger = Loggers.getLogger(getClass());
-            DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
-            deprecationLogger.deprecated("Using _index is deprecated. Create a custom ScriptEngine to access index internals.");
-            deprecationEmitted = true;
-        }
-    }
-
-    public int numDocs() {
-        logDeprecation();
-        if (numDocs == -1) {
-            numDocs = parentReader.numDocs();
-        }
-        return numDocs;
-    }
-
-    public int maxDoc() {
-        logDeprecation();
-        if (maxDoc == -1) {
-            maxDoc = parentReader.maxDoc();
-        }
-        return maxDoc;
-    }
-
-    public int numDeletedDocs() {
-        logDeprecation();
-        if (numDeletedDocs == -1) {
-            numDeletedDocs = parentReader.numDeletedDocs();
-        }
-        return numDeletedDocs;
-    }
-
-    public LeafIndexLookup(LeafReaderContext ctx) {
-        reader = ctx.reader();
-        parentReader = ReaderUtil.getTopLevelContext(ctx).reader();
-        indexSearcher = new IndexSearcher(parentReader);
-        indexSearcher.setQueryCache(null);
-    }
-
-    public void setDocument(int docId) {
-        if (this.docId == docId) { // if we are called with the same docId,
-                                   // nothing to do
-            return;
-        }
-        // We assume that docs are processed in ascending order of id. If this
-        // is not the case, we would have to re initialize all posting lists in
-        // IndexFieldTerm. TODO: Instead of assert we could also call
-        // setReaderInFields(); here?
-        if (this.docId > docId) {
-            // This might happen if the same SearchLookup is used in different
-            // phases, such as score and fetch phase.
-            // In this case we do not want to re initialize posting list etc.
-            // because we do not even know if term and field statistics will be
-            // needed in this new phase.
-            // Therefore we just remove all IndexFieldTerms.
-            indexFields.clear();
-        }
-        this.docId = docId;
-        setNextDocIdInFields();
-    }
-
-    protected void setNextDocIdInFields() {
-        for (IndexField stat : indexFields.values()) {
-            stat.setDocIdInTerms(this.docId);
-        }
-    }
-
-    /*
-     * TODO: here might be potential for running time improvement? If we knew in
-     * advance which terms are requested, we could provide an array which the
-     * user could then iterate over.
-     */
-    @Override
-    public IndexField get(Object key) {
-        logDeprecation();
-        String stringField = (String) key;
-        IndexField indexField = indexFields.get(key);
-        if (indexField == null) {
-            try {
-                indexField = new IndexField(stringField, this);
-                indexFields.put(stringField, indexField);
-            } catch (IOException e) {
-                throw new ElasticsearchException(e);
-            }
-        }
-        return indexField;
-    }
-
-    /*
-     * Get the lucene term vectors. See
-     * https://lucene.apache.org/core/4_0_0/core/org/apache/lucene/index/Fields.html
-     * *
-     */
-    public Fields termVectors() throws IOException {
-        logDeprecation();
-        assert reader != null;
-        return reader.getTermVectors(docId);
-    }
-
-    LeafReader getReader() {
-        logDeprecation();
-        return reader;
-    }
-
-    public int getDocId() {
-        logDeprecation();
-        return docId;
-    }
-
-    public IndexReader getParentReader() {
-        logDeprecation();
-        if (parentReader == null) {
-            return reader;
-        }
-        return parentReader;
-    }
-
-    public IndexSearcher getIndexSearcher() {
-        logDeprecation();
-        return indexSearcher;
-    }
-
-    public IndexReaderContext getReaderContext() {
-        logDeprecation();
-        return getParentReader().getContext();
-    }
-}

+ 2 - 11
core/src/main/java/org/elasticsearch/search/lookup/LeafSearchLookup.java

@@ -35,24 +35,20 @@ public class LeafSearchLookup {
     final LeafDocLookup docMap;
     final SourceLookup sourceLookup;
     final LeafFieldsLookup fieldsLookup;
-    final LeafIndexLookup indexLookup;
     final Map<String, Object> asMap;
 
     public LeafSearchLookup(LeafReaderContext ctx, LeafDocLookup docMap, SourceLookup sourceLookup,
-            LeafFieldsLookup fieldsLookup, LeafIndexLookup indexLookup, Map<String, Object> topLevelMap) {
+            LeafFieldsLookup fieldsLookup) {
         this.ctx = ctx;
         this.docMap = docMap;
         this.sourceLookup = sourceLookup;
         this.fieldsLookup = fieldsLookup;
-        this.indexLookup = indexLookup;
 
-        Map<String, Object> asMap = new HashMap<>(topLevelMap.size() + 5);
-        asMap.putAll(topLevelMap);
+        Map<String, Object> asMap = new HashMap<>(4);
         asMap.put("doc", docMap);
         asMap.put("_doc", docMap);
         asMap.put("_source", sourceLookup);
         asMap.put("_fields", fieldsLookup);
-        asMap.put("_index", indexLookup);
         this.asMap = unmodifiableMap(asMap);
     }
 
@@ -64,10 +60,6 @@ public class LeafSearchLookup {
         return this.sourceLookup;
     }
 
-    public LeafIndexLookup indexLookup() {
-        return this.indexLookup;
-    }
-
     public LeafFieldsLookup fields() {
         return this.fieldsLookup;
     }
@@ -80,6 +72,5 @@ public class LeafSearchLookup {
         docMap.setDocument(docId);
         sourceLookup.setSegmentAndDocument(ctx, docId);
         fieldsLookup.setDocument(docId);
-        indexLookup.setDocument(docId);
     }
 }

+ 0 - 87
core/src/main/java/org/elasticsearch/search/lookup/PositionIterator.java

@@ -1,87 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.index.PostingsEnum;
-import org.elasticsearch.ElasticsearchException;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-public class PositionIterator implements Iterator<TermPosition> {
-    
-    private boolean resetted = false;
-
-    protected IndexFieldTerm indexFieldTerm;
-
-    protected int freq = -1;
-
-    // current position of iterator
-    private int currentPos;
-
-    protected final TermPosition termPosition = new TermPosition();
-
-    private PostingsEnum postings;
-
-    public PositionIterator(IndexFieldTerm indexFieldTerm) {
-        this.indexFieldTerm = indexFieldTerm;
-    }
-
-    @Override
-    public void remove() {
-        throw new UnsupportedOperationException("Cannot remove anything from TermPosition iterator.");
-    }
-
-    @Override
-    public boolean hasNext() {
-        return currentPos < freq;
-    }
-
-
-    @Override
-    public TermPosition next() {
-        try {
-            termPosition.position = postings.nextPosition();
-            termPosition.startOffset = postings.startOffset();
-            termPosition.endOffset = postings.endOffset();
-            termPosition.payload = postings.getPayload();
-        } catch (IOException ex) {
-            throw new ElasticsearchException("can not advance iterator", ex);
-        }
-        currentPos++;
-        return termPosition;
-    }
-
-    public void nextDoc() throws IOException {
-        resetted = false;
-        currentPos = 0;
-        freq = indexFieldTerm.tf();
-        postings = indexFieldTerm.postings;
-    }
-
-    public Iterator<TermPosition> reset() {
-        if (resetted) {
-            throw new ElasticsearchException(
-                    "Cannot iterate twice! If you want to iterate more that once, add _CACHE explicitly.");
-        }
-        resetted = true;
-        return this;
-    }
-}

+ 1 - 3
core/src/main/java/org/elasticsearch/search/lookup/SearchLookup.java

@@ -42,9 +42,7 @@ public class SearchLookup {
         return new LeafSearchLookup(context,
                 docMap.getLeafDocLookup(context),
                 sourceLookup,
-                fieldsLookup.getLeafFieldsLookup(context),
-                IndexLookup.getLeafIndexLookup(context),
-                IndexLookup.NAMES);
+                fieldsLookup.getLeafFieldsLookup(context));
     }
 
     public DocLookup doc() {

+ 0 - 58
core/src/main/java/org/elasticsearch/search/lookup/TermPosition.java

@@ -1,58 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.search.lookup;
-
-import org.apache.lucene.analysis.payloads.PayloadHelper;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.CharsRefBuilder;
-
-public class TermPosition {
-
-    public int position = -1;
-    public int startOffset = -1;
-    public int endOffset = -1;
-    public BytesRef payload;
-    private CharsRefBuilder spare = new CharsRefBuilder(); 
-
-    public String payloadAsString() {
-        if (payload != null && payload.length != 0) {
-            spare.copyUTF8Bytes(payload);
-            return spare.toString();
-        } else {
-            return null;
-        }
-    }
-
-    public float payloadAsFloat(float defaultMissing) {
-        if (payload != null && payload.length != 0) {
-            return PayloadHelper.decodeFloat(payload.bytes, payload.offset);
-        } else {
-            return defaultMissing;
-        }
-    }
-
-    public int payloadAsInt(int defaultMissing) {
-        if (payload != null && payload.length != 0) {
-            return PayloadHelper.decodeInt(payload.bytes, payload.offset);
-        } else {
-            return defaultMissing;
-        }
-    }
-}

+ 0 - 1029
core/src/test/java/org/elasticsearch/script/IndexLookupIT.java

@@ -1,1029 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch 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.
- */
-
-package org.elasticsearch.script;
-
-import org.elasticsearch.action.search.SearchPhaseExecutionException;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.search.ShardSearchFailure;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.index.query.QueryBuilders;
-import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders;
-import org.elasticsearch.plugins.Plugin;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.lookup.IndexField;
-import org.elasticsearch.search.lookup.IndexFieldTerm;
-import org.elasticsearch.search.lookup.IndexLookup;
-import org.elasticsearch.search.lookup.LeafIndexLookup;
-import org.elasticsearch.search.lookup.TermPosition;
-import org.elasticsearch.test.ESIntegTestCase;
-import org.hamcrest.Matchers;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.function.Function;
-
-import static java.util.Collections.emptyList;
-
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-
-public class IndexLookupIT extends ESIntegTestCase {
-
-    private static final String INCLUDE_ALL = "_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS|_CACHE";
-    private static final int ALL_FLAGS = IndexLookup.FLAG_FREQUENCIES
-            | IndexLookup.FLAG_OFFSETS
-            | IndexLookup.FLAG_PAYLOADS
-            | IndexLookup.FLAG_POSITIONS
-            | IndexLookup.FLAG_CACHE;
-
-    private static final String INCLUDE_ALL_BUT_CACHE = "_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS";
-    private static final int ALL_FLAGS_WITHOUT_CACHE = IndexLookup.FLAG_FREQUENCIES
-            | IndexLookup.FLAG_OFFSETS
-            | IndexLookup.FLAG_PAYLOADS
-            | IndexLookup.FLAG_POSITIONS;
-
-    private HashMap<String, List<Object>> expectedEndOffsetsArray;
-    private HashMap<String, List<Object>> expectedPayloadsArray;
-    private HashMap<String, List<Object>> expectedPositionsArray;
-    private HashMap<String, List<Object>> expectedStartOffsetsArray;
-
-    @Override
-    protected Collection<Class<? extends Plugin>> nodePlugins() {
-        return Collections.singleton(CustomScriptPlugin.class);
-    }
-
-    public static class CustomScriptPlugin extends MockScriptPlugin {
-
-        @Override
-        @SuppressWarnings("unchecked")
-        protected Map<String, Function<Map<String, Object>, Object>> pluginScripts() {
-            Map<String, Function<Map<String, Object>, Object>> scripts = new HashMap<>();
-
-            scripts.put("term = _index['int_payload_field']['c']; term.tf()", vars -> tf(vars, "int_payload_field", "c"));
-            scripts.put("term = _index['int_payload_field']['b']; term.tf()", vars -> tf(vars, "int_payload_field", "b"));
-
-            scripts.put("Sum the payloads of [float_payload_field][b]", vars -> payloadSum(vars, "float_payload_field", "b"));
-            scripts.put("Sum the payloads of [int_payload_field][b]", vars -> payloadSum(vars, "int_payload_field", "b"));
-
-            scripts.put("createPositionsArrayScriptIterateTwice[b," + INCLUDE_ALL + ",position]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS, p -> p.position));
-            scripts.put("createPositionsArrayScriptIterateTwice[b," + INCLUDE_ALL + ",startOffset]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS, p -> p.startOffset));
-            scripts.put("createPositionsArrayScriptIterateTwice[b," + INCLUDE_ALL + ",endOffset]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS, p -> p.endOffset));
-            scripts.put("createPositionsArrayScriptIterateTwice[b," + INCLUDE_ALL + ",payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScriptIterateTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,position]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.position));
-            scripts.put("createPositionsArrayScriptIterateTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,startOffset]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.startOffset));
-            scripts.put("createPositionsArrayScriptIterateTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,endOffset]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.endOffset));
-            scripts.put("createPositionsArrayScriptIterateTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScriptIterateTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScriptGetInfoObjectTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,position]",
-                    vars -> createPositionsArrayScriptGetInfoObjectTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.position));
-            scripts.put("createPositionsArrayScriptGetInfoObjectTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,startOffset]",
-                    vars -> createPositionsArrayScriptGetInfoObjectTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.startOffset));
-            scripts.put("createPositionsArrayScriptGetInfoObjectTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,endOffset]",
-                    vars -> createPositionsArrayScriptGetInfoObjectTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.endOffset));
-            scripts.put("createPositionsArrayScriptGetInfoObjectTwice[b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScriptGetInfoObjectTwice(vars, "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_POSITIONS,position]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_POSITIONS, p -> p.position));
-
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_OFFSETS,position]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_OFFSETS, p -> p.position));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_OFFSETS,startOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_OFFSETS, p -> p.startOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_OFFSETS,endOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_OFFSETS, p -> p.endOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_OFFSETS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_OFFSETS, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_PAYLOADS,position]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_PAYLOADS, p -> p.position));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_PAYLOADS,startOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_PAYLOADS, p -> p.startOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_PAYLOADS,endOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_PAYLOADS, p -> p.endOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_PAYLOADS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", IndexLookup.FLAG_PAYLOADS, p -> p.payloadAsInt(-1)));
-
-            int posoffpay = IndexLookup.FLAG_POSITIONS|IndexLookup.FLAG_OFFSETS|IndexLookup.FLAG_PAYLOADS;
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_POSITIONS|_OFFSETS|_PAYLOADS,position]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", posoffpay, p -> p.position));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_POSITIONS|_OFFSETS|_PAYLOADS,startOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", posoffpay, p -> p.startOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_POSITIONS|_OFFSETS|_PAYLOADS,endOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", posoffpay, p -> p.endOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_POSITIONS|_OFFSETS|_PAYLOADS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", posoffpay, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,position]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.position));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,startOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.startOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,endOffset]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.endOffset));
-            scripts.put("createPositionsArrayScript[int_payload_field,b,_FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS,payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScript(vars, "int_payload_field", "b", ALL_FLAGS_WITHOUT_CACHE, p -> p.payloadAsInt(-1)));
-
-            scripts.put("createPositionsArrayScript" +
-                            "[float_payload_field,b," + INCLUDE_ALL + ",payloadAsFloat(-1)]",
-                    vars -> createPositionsArrayScript(vars,"float_payload_field", "b", ALL_FLAGS, p -> p.payloadAsFloat(-1)));
-            scripts.put("createPositionsArrayScript" +
-                            "[string_payload_field,b," + INCLUDE_ALL + ",payloadAsString()]",
-                    vars -> createPositionsArrayScript(vars,"string_payload_field", "b", ALL_FLAGS, TermPosition::payloadAsString));
-            scripts.put("createPositionsArrayScript" +
-                            "[int_payload_field,c," + INCLUDE_ALL + ",payloadAsInt(-1)]",
-                    vars -> createPositionsArrayScript(vars,"int_payload_field", "c", ALL_FLAGS, p -> p.payloadAsInt(-1)));
-
-            // Call with different flags twice, equivalent to:
-            //      term = _index['int_payload_field']['b']; return _index['int_payload_field'].get('b', _POSITIONS).tf();
-            scripts.put("Call with different flags twice", vars -> {
-                LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-                IndexField indexField = leafIndexLookup.get("int_payload_field");
-
-                // 1st call
-                indexField.get("b");
-                try {
-                    // 2nd call, must throws an exception
-                    return indexField.get("b", IndexLookup.FLAG_POSITIONS).tf();
-                } catch (IOException e) {
-                    throw new ScriptException(e.getMessage(), e, emptyList(), "Call with different flags twice", CustomScriptPlugin.NAME);
-                }
-            });
-
-            // Call with same flags twice: equivalent to:
-            //      term = _index['int_payload_field'].get('b', _POSITIONS | _FREQUENCIES);return _index['int_payload_field']['b'].tf();
-            scripts.put("Call with same flags twice", vars -> {
-                LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-                IndexField indexField = leafIndexLookup.get("int_payload_field");
-
-                // 1st call
-                indexField.get("b", IndexLookup.FLAG_POSITIONS | IndexLookup.FLAG_FREQUENCIES);
-                try {
-                    // 2nd call, must throws an exception
-                    return indexField.get("b").tf();
-                } catch (IOException e) {
-                    throw new ScriptException(e.getMessage(), e, emptyList(), "Call with same flags twice", CustomScriptPlugin.NAME);
-                }
-            });
-
-            // get the number of all docs
-            scripts.put("_index.numDocs()",
-                    vars -> ((LeafIndexLookup) vars.get("_index")).numDocs());
-
-            // get the number of docs with field float_payload_field
-            scripts.put("_index['float_payload_field'].docCount()",
-                    vars -> indexFieldScript(vars, "float_payload_field", indexField -> {
-                        try {
-                            return indexField.docCount();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "docCount()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // corner case: what if the field does not exist?
-            scripts.put("_index['non_existent_field'].docCount()",
-                    vars -> indexFieldScript(vars, "non_existent_field", indexField -> {
-                        try {
-                            return indexField.docCount();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "docCount()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // get the number of all tokens in all docs
-            scripts.put("_index['float_payload_field'].sumttf()",
-                    vars -> indexFieldScript(vars, "float_payload_field", indexField -> {
-                        try {
-                            return indexField.sumttf();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "sumttf()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // corner case get the number of all tokens in all docs for non existent
-            // field
-            scripts.put("_index['non_existent_field'].sumttf()",
-                    vars -> indexFieldScript(vars, "non_existent_field", indexField -> {
-                        try {
-                            return indexField.sumttf();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "sumttf()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // get the sum of doc freqs in all docs
-            scripts.put("_index['float_payload_field'].sumdf()",
-                    vars -> indexFieldScript(vars, "float_payload_field", indexField -> {
-                        try {
-                            return indexField.sumdf();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "sumdf()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // get the sum of doc freqs in all docs for non existent field
-            scripts.put("_index['non_existent_field'].sumdf()",
-                    vars -> indexFieldScript(vars, "non_existent_field", indexField -> {
-                        try {
-                            return indexField.sumdf();
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "sumdf()", CustomScriptPlugin.NAME);
-                        }
-                    }));
-
-            // check term frequencies for 'a'
-            scripts.put("term = _index['float_payload_field']['a']; if (term != null) {term.tf()}",
-                    vars -> indexFieldTermScript(vars, "float_payload_field", "a", indexFieldTerm -> {
-                        try {
-                            if (indexFieldTerm != null) {
-                                return indexFieldTerm.tf();
-                            }
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "term.tf()", CustomScriptPlugin.NAME);
-                        }
-                        return null;
-                    }));
-
-            // check doc frequencies for 'c'
-            scripts.put("term = _index['float_payload_field']['c']; if (term != null) {term.df()}",
-                    vars -> indexFieldTermScript(vars, "float_payload_field", "c", indexFieldTerm -> {
-                        try {
-                            if (indexFieldTerm != null) {
-                                return indexFieldTerm.df();
-                            }
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "term.df()", CustomScriptPlugin.NAME);
-                        }
-                        return null;
-                    }));
-
-            // check doc frequencies for term that does not exist
-            scripts.put("term = _index['float_payload_field']['non_existent_term']; if (term != null) {term.df()}",
-                    vars -> indexFieldTermScript(vars, "float_payload_field", "non_existent_term", indexFieldTerm -> {
-                        try {
-                            if (indexFieldTerm != null) {
-                                return indexFieldTerm.df();
-                            }
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "term.df()", CustomScriptPlugin.NAME);
-                        }
-                        return null;
-                    }));
-
-            // check doc frequencies for term that does not exist
-            scripts.put("term = _index['non_existent_field']['non_existent_term']; if (term != null) {term.tf()}",
-                    vars -> indexFieldTermScript(vars, "non_existent_field", "non_existent_term", indexFieldTerm -> {
-                        try {
-                            if (indexFieldTerm != null) {
-                                return indexFieldTerm.tf();
-                            }
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "term.tf()", CustomScriptPlugin.NAME);
-                        }
-                        return null;
-                    }));
-
-            // check total term frequencies for 'a'
-            scripts.put("term = _index['float_payload_field']['a']; if (term != null) {term.ttf()}",
-                    vars -> indexFieldTermScript(vars, "float_payload_field", "a", indexFieldTerm -> {
-                        try {
-                            if (indexFieldTerm != null) {
-                                return indexFieldTerm.ttf();
-                            }
-                        } catch (IOException e) {
-                            throw new ScriptException(e.getMessage(), e, emptyList(), "term.ttf()", CustomScriptPlugin.NAME);
-                        }
-                        return null;
-                    }));
-
-            return scripts;
-        }
-
-        @SuppressWarnings("unchecked")
-        static Object indexFieldScript(Map<String, Object> vars, String fieldName, Function<IndexField, Object> fn) {
-            LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-            return fn.apply(leafIndexLookup.get(fieldName));
-        }
-
-        @SuppressWarnings("unchecked")
-        static Object indexFieldTermScript(Map<String, Object> vars, String fieldName, String term, Function<IndexFieldTerm, Object> fn) {
-            return indexFieldScript(vars, fieldName, indexField -> fn.apply(indexField.get(term)));
-        }
-
-        @SuppressWarnings("unchecked")
-        static Object tf(Map<String, Object> vars, String fieldName, String term) {
-            return indexFieldTermScript(vars, fieldName, term, indexFieldTerm -> {
-                try {
-                    return indexFieldTerm.tf();
-                } catch (IOException e) {
-                    throw new RuntimeException("Mocked script error when retrieving TF for [" + fieldName + "][" + term + "]");
-                }
-            });
-        }
-
-        // Sum the payloads for a given field term, equivalent to:
-        //      term = _index['float_payload_field'].get('b', _FREQUENCIES|_OFFSETS|_PAYLOADS|_POSITIONS|_CACHE);
-        //      payloadSum=0;
-        //      for (pos in term) {
-        //          payloadSum += pos.payloadAsInt(0)
-        //      };
-        //      return payloadSum;
-        @SuppressWarnings("unchecked")
-        static Object payloadSum(Map<String, Object> vars, String fieldName, String term) {
-            return indexFieldScript(vars, fieldName, indexField -> {
-                IndexFieldTerm indexFieldTerm = indexField.get(term, IndexLookup.FLAG_FREQUENCIES
-                        | IndexLookup.FLAG_OFFSETS
-                        | IndexLookup.FLAG_PAYLOADS
-                        | IndexLookup.FLAG_POSITIONS
-                        | IndexLookup.FLAG_CACHE);
-                int payloadSum = 0;
-                for (TermPosition position : indexFieldTerm) {
-                    payloadSum += position.payloadAsInt(0);
-                }
-                return payloadSum;
-            });
-        }
-
-        @SuppressWarnings("unchecked")
-        static List<Object> createPositionsArrayScriptGetInfoObjectTwice(Map<String, Object> vars, String term, int flags,
-                                                                         Function<TermPosition, Object> fn) {
-            LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-            IndexField indexField = leafIndexLookup.get("int_payload_field");
-
-            // 1st call
-            IndexFieldTerm indexFieldTerm = indexField.get(term, flags);
-
-            List<Object> array = new ArrayList<>();
-            for (TermPosition position : indexFieldTerm) {
-                array.add(fn.apply(position));
-            }
-
-            // 2nd call
-            indexField.get(term, flags);
-
-            array = new ArrayList<>();
-            for (TermPosition position : indexFieldTerm) {
-                array.add(fn.apply(position));
-            }
-
-            return array;
-        }
-
-        @SuppressWarnings("unchecked")
-        static List<Object> createPositionsArrayScriptIterateTwice(Map<String, Object> vars, String term, int flags,
-                                                                   Function<TermPosition, Object> fn) {
-            LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-            IndexField indexField = leafIndexLookup.get("int_payload_field");
-
-            IndexFieldTerm indexFieldTerm = indexField.get(term, flags);
-
-            // 1st iteration
-            List<Object> array = new ArrayList<>();
-            for (TermPosition position : indexFieldTerm) {
-                array.add(fn.apply(position));
-            }
-
-            // 2nd iteration
-            array = new ArrayList<>();
-            for (TermPosition position : indexFieldTerm) {
-                array.add(fn.apply(position));
-            }
-
-            return array;
-        }
-
-        @SuppressWarnings("unchecked")
-        static List<Object> createPositionsArrayScript(Map<String, Object> vars, String field, String term, int flags,
-                                                       Function<TermPosition, Object> fn) {
-
-            LeafIndexLookup leafIndexLookup = (LeafIndexLookup) vars.get("_index");
-            IndexField indexField = leafIndexLookup.get(field);
-
-            IndexFieldTerm indexFieldTerm = indexField.get(term, flags);
-            List<Object> array = new ArrayList<>();
-            for (TermPosition position : indexFieldTerm) {
-                array.add(fn.apply(position));
-            }
-            return array;
-        }
-    }
-
-    void initTestData() throws InterruptedException, ExecutionException, IOException {
-        HashMap<String, List<Object>> emptyArray = new HashMap<>();
-        List<Object> empty1 = new ArrayList<>();
-        empty1.add(-1);
-        empty1.add(-1);
-        emptyArray.put("1", empty1);
-        List<Object> empty2 = new ArrayList<>();
-        empty2.add(-1);
-        empty2.add(-1);
-        emptyArray.put("2", empty2);
-        List<Object> empty3 = new ArrayList<>();
-        empty3.add(-1);
-        empty3.add(-1);
-        emptyArray.put("3", empty3);
-
-        expectedPositionsArray = new HashMap<>();
-
-        List<Object> pos1 = new ArrayList<>();
-        pos1.add(1);
-        pos1.add(2);
-        expectedPositionsArray.put("1", pos1);
-        List<Object> pos2 = new ArrayList<>();
-        pos2.add(0);
-        pos2.add(1);
-        expectedPositionsArray.put("2", pos2);
-        List<Object> pos3 = new ArrayList<>();
-        pos3.add(0);
-        pos3.add(4);
-        expectedPositionsArray.put("3", pos3);
-
-        expectedPayloadsArray = new HashMap<>();
-        List<Object> pay1 = new ArrayList<>();
-        pay1.add(2);
-        pay1.add(3);
-        expectedPayloadsArray.put("1", pay1);
-        List<Object> pay2 = new ArrayList<>();
-        pay2.add(1);
-        pay2.add(2);
-        expectedPayloadsArray.put("2", pay2);
-        List<Object> pay3 = new ArrayList<>();
-        pay3.add(1);
-        pay3.add(-1);
-        expectedPayloadsArray.put("3", pay3);
-        /*
-         * "a|1 b|2 b|3 c|4 d " "b|1 b|2 c|3 d|4 a " "b|1 c|2 d|3 a|4 b "
-         */
-        expectedStartOffsetsArray = new HashMap<>();
-        List<Object> starts1 = new ArrayList<>();
-        starts1.add(4);
-        starts1.add(8);
-        expectedStartOffsetsArray.put("1", starts1);
-        List<Object> starts2 = new ArrayList<>();
-        starts2.add(0);
-        starts2.add(4);
-        expectedStartOffsetsArray.put("2", starts2);
-        List<Object> starts3 = new ArrayList<>();
-        starts3.add(0);
-        starts3.add(16);
-        expectedStartOffsetsArray.put("3", starts3);
-
-        expectedEndOffsetsArray = new HashMap<>();
-        List<Object> ends1 = new ArrayList<>();
-        ends1.add(7);
-        ends1.add(11);
-        expectedEndOffsetsArray.put("1", ends1);
-        List<Object> ends2 = new ArrayList<>();
-        ends2.add(3);
-        ends2.add(7);
-        expectedEndOffsetsArray.put("2", ends2);
-        List<Object> ends3 = new ArrayList<>();
-        ends3.add(3);
-        ends3.add(17);
-        expectedEndOffsetsArray.put("3", ends3);
-
-        XContentBuilder mapping = XContentFactory.jsonBuilder()
-                .startObject()
-                    .startObject("type1")
-                        .startObject("properties")
-                            .startObject("int_payload_field")
-                                .field("type", "text")
-                                .field("index_options", "offsets")
-                                .field("analyzer", "payload_int")
-                            .endObject()
-                        .endObject()
-                    .endObject()
-                .endObject();
-
-        assertAcked(prepareCreate("test").addMapping("type1", mapping).setSettings(
-                Settings.builder()
-                        .put(indexSettings())
-                        .put("index.analysis.analyzer.payload_int.tokenizer", "whitespace")
-                        .putArray("index.analysis.analyzer.payload_int.filter", "delimited_int")
-                        .put("index.analysis.filter.delimited_int.delimiter", "|")
-                        .put("index.analysis.filter.delimited_int.encoding", "int")
-                        .put("index.analysis.filter.delimited_int.type", "delimited_payload_filter")));
-        indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("int_payload_field", "a|1 b|2 b|3 c|4 d "), client()
-                        .prepareIndex("test", "type1", "2").setSource("int_payload_field", "b|1 b|2 c|3 d|4 a "),
-                client().prepareIndex("test", "type1", "3").setSource("int_payload_field", "b|1 c|2 d|3 a|4 b "));
-        ensureGreen();
-    }
-
-    public void testTwoScripts() throws Exception {
-        initTestData();
-
-        Script scriptFieldScript = createScript("term = _index['int_payload_field']['c']; term.tf()");
-        Script scoreScript = createScript("term = _index['int_payload_field']['b']; term.tf()");
-        Map<String, Object> expectedResultsField = new HashMap<>();
-        expectedResultsField.put("1", 1);
-        expectedResultsField.put("2", 1);
-        expectedResultsField.put("3", 1);
-        Map<String, Object> expectedResultsScore = new HashMap<>();
-        expectedResultsScore.put("1", 2f);
-        expectedResultsScore.put("2", 2f);
-        expectedResultsScore.put("3", 2f);
-        checkOnlyFunctionScore(scoreScript, expectedResultsScore, 3);
-        checkValueInEachDocWithFunctionScore(scriptFieldScript, expectedResultsField, scoreScript, expectedResultsScore, 3);
-
-    }
-
-    public void testCallWithDifferentFlagsFails() throws Exception {
-        initTestData();
-        final int numPrimaries = getNumShards("test").numPrimaries;
-        final String expectedError = "You must call get with all required flags! " +
-                "Instead of  _index['int_payload_field'].get('b', _FREQUENCIES) and _index['int_payload_field'].get('b', _POSITIONS)" +
-                " call  _index['int_payload_field'].get('b', _FREQUENCIES | _POSITIONS)  once]";
-
-        // should throw an exception, we cannot call with different flags twice
-        // if the flags of the second call were not included in the first call.
-        Script script = createScript("Call with different flags twice");
-        try {
-            SearchResponse response = client().prepareSearch("test")
-                    .setQuery(QueryBuilders.matchAllQuery())
-                    .addScriptField("tvtest", script)
-                    .get();
-
-            // (partial) success when at least one shard succeeds
-            assertThat(numPrimaries, greaterThan(response.getShardFailures().length));
-            assertThat(response.getFailedShards(), greaterThanOrEqualTo(1));
-
-            for (ShardSearchFailure failure : response.getShardFailures()) {
-                assertThat(failure.reason(), containsString(expectedError));
-            }
-        } catch (SearchPhaseExecutionException e) {
-            // Exception thrown when *all* shards fail
-            assertThat(numPrimaries, equalTo(e.shardFailures().length));
-            for (ShardSearchFailure failure : e.shardFailures()) {
-                assertThat(failure.reason(), containsString(expectedError));
-            }
-        }
-
-        // Should not throw an exception this way round
-        script = createScript("Call with same flags twice");
-        assertThat(client().prepareSearch("test")
-                .setQuery(QueryBuilders.matchAllQuery())
-                .addScriptField("tvtest", script)
-                .get().getHits().getTotalHits(), greaterThan(0L));
-    }
-
-    private void checkOnlyFunctionScore(Script scoreScript, Map<String, Object> expectedScore, int numExpectedDocs) {
-        SearchResponse sr = client().prepareSearch("test")
-                .setQuery(QueryBuilders.functionScoreQuery(ScoreFunctionBuilders.scriptFunction(scoreScript))).execute()
-                .actionGet();
-        assertHitCount(sr, numExpectedDocs);
-        for (SearchHit hit : sr.getHits().getHits()) {
-            assertThat("for doc " + hit.getId(), ((Float) expectedScore.get(hit.getId())).doubleValue(),
-                    Matchers.closeTo(hit.getScore(), 1.e-4));
-        }
-    }
-
-    public void testDocumentationExample() throws Exception {
-        initTestData();
-
-        Script script = createScript("Sum the payloads of [float_payload_field][b]");
-
-        // non existing field: sum should be 0
-        HashMap<String, Object> zeroArray = new HashMap<>();
-        zeroArray.put("1", 0);
-        zeroArray.put("2", 0);
-        zeroArray.put("3", 0);
-        checkValueInEachDoc(script, zeroArray, 3);
-
-        script = createScript("Sum the payloads of [int_payload_field][b]");
-
-        // existing field: sums should be as here:
-        zeroArray.put("1", 5);
-        zeroArray.put("2", 3);
-        zeroArray.put("3", 1);
-        checkValueInEachDoc(script, zeroArray, 3);
-    }
-
-    public void testIteratorAndRecording() throws Exception {
-        initTestData();
-
-        // call twice with record: should work as expected
-        Script script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL, "position");
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL, "startOffset");
-        checkArrayValsInEachDoc(script, expectedStartOffsetsArray, 3);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL, "endOffset");
-        checkArrayValsInEachDoc(script, expectedEndOffsetsArray, 3);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL, "payloadAsInt(-1)");
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 3);
-
-        // no record and get iterator twice: should fail
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL_BUT_CACHE, "position");
-        checkExceptions(script);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL_BUT_CACHE, "startOffset");
-        checkExceptions(script);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL_BUT_CACHE, "endOffset");
-        checkExceptions(script);
-        script = createPositionsArrayScriptIterateTwice("b", INCLUDE_ALL_BUT_CACHE, "payloadAsInt(-1)");
-        checkExceptions(script);
-
-        // no record and get termObject twice and iterate: should fail
-        script = createPositionsArrayScriptGetInfoObjectTwice("b", INCLUDE_ALL_BUT_CACHE, "position");
-        checkExceptions(script);
-        script = createPositionsArrayScriptGetInfoObjectTwice("b", INCLUDE_ALL_BUT_CACHE, "startOffset");
-        checkExceptions(script);
-        script = createPositionsArrayScriptGetInfoObjectTwice("b", INCLUDE_ALL_BUT_CACHE, "endOffset");
-        checkExceptions(script);
-        script = createPositionsArrayScriptGetInfoObjectTwice("b", INCLUDE_ALL_BUT_CACHE, "payloadAsInt(-1)");
-        checkExceptions(script);
-
-    }
-
-    private Script createPositionsArrayScriptGetInfoObjectTwice(String term, String flags, String what) {
-        return createScript("createPositionsArrayScriptGetInfoObjectTwice[" + term + "," + flags + "," + what + "]");
-    }
-
-    private Script createPositionsArrayScriptIterateTwice(String term, String flags, String what) {
-        return createScript("createPositionsArrayScriptIterateTwice[" + term + "," + flags + "," + what + "]");
-    }
-
-    private Script createPositionsArrayScript(String field, String term, String flags, String what) {
-        return createScript("createPositionsArrayScript[" + field + ","  + term + "," + flags + "," + what + "]");
-    }
-
-    private Script createPositionsArrayScriptDefaultGet(String field, String term, String what) {
-        return createScript("createPositionsArrayScriptDefaultGet[" + field + ","  + term + "," + what + "]");
-    }
-
-    private Script createScript(String script) {
-        return new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, script, Collections.emptyMap());
-    }
-
-    public void testFlags() throws Exception {
-        initTestData();
-
-        // check default flag
-        Script script = createPositionsArrayScriptDefaultGet("int_payload_field", "b", "position");
-        // there should be no positions
-        /* TODO: the following tests fail with the new postings enum apis because of a bogus assert in BlockDocsEnum
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScriptDefaultGet("int_payload_field", "b", "startOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScriptDefaultGet("int_payload_field", "b", "endOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScriptDefaultGet("int_payload_field", "b", "payloadAsInt(-1)");
-        // there should be no payload
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-
-        // check FLAG_FREQUENCIES flag
-        script = createPositionsArrayScript("int_payload_field", "b", "_FREQUENCIES", "position");
-        // there should be no positions
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_FREQUENCIES", "startOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_FREQUENCIES", "endOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_FREQUENCIES", "payloadAsInt(-1)");
-        // there should be no payloads
-        checkArrayValsInEachDoc(script, emptyArray, 3);*/
-
-        // check FLAG_POSITIONS flag
-        script = createPositionsArrayScript("int_payload_field", "b", "_POSITIONS", "position");
-        // there should be positions
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        /* TODO: these tests make a bogus assumption that asking for positions will return only positions
-        script = createPositionsArrayScript("int_payload_field", "b", "_POSITIONS", "startOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_POSITIONS", "endOffset");
-        // there should be no offsets
-        checkArrayValsInEachDoc(script, emptyArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_POSITIONS", "payloadAsInt(-1)");
-        // there should be no payloads
-        checkArrayValsInEachDoc(script, emptyArray, 3);*/
-
-        // check FLAG_OFFSETS flag
-        script = createPositionsArrayScript("int_payload_field", "b", "_OFFSETS", "position");
-        // there should be positions and s forth ...
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_OFFSETS", "startOffset");
-        checkArrayValsInEachDoc(script, expectedStartOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_OFFSETS", "endOffset");
-        checkArrayValsInEachDoc(script, expectedEndOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_OFFSETS", "payloadAsInt(-1)");
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 3);
-
-        // check FLAG_PAYLOADS flag
-        script = createPositionsArrayScript("int_payload_field", "b", "_PAYLOADS", "position");
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_PAYLOADS", "startOffset");
-        checkArrayValsInEachDoc(script, expectedStartOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_PAYLOADS", "endOffset");
-        checkArrayValsInEachDoc(script, expectedEndOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", "_PAYLOADS", "payloadAsInt(-1)");
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 3);
-
-        // check all flags
-        String allFlags = "_POSITIONS|_OFFSETS|_PAYLOADS";
-        script = createPositionsArrayScript("int_payload_field", "b", allFlags, "position");
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", allFlags, "startOffset");
-        checkArrayValsInEachDoc(script, expectedStartOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", allFlags, "endOffset");
-        checkArrayValsInEachDoc(script, expectedEndOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", allFlags, "payloadAsInt(-1)");
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 3);
-
-        // check all flags without record
-        script = createPositionsArrayScript("int_payload_field", "b", INCLUDE_ALL_BUT_CACHE, "position");
-        checkArrayValsInEachDoc(script, expectedPositionsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", INCLUDE_ALL_BUT_CACHE, "startOffset");
-        checkArrayValsInEachDoc(script, expectedStartOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", INCLUDE_ALL_BUT_CACHE, "endOffset");
-        checkArrayValsInEachDoc(script, expectedEndOffsetsArray, 3);
-        script = createPositionsArrayScript("int_payload_field", "b", INCLUDE_ALL_BUT_CACHE, "payloadAsInt(-1)");
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 3);
-
-    }
-
-    private void checkArrayValsInEachDoc(Script script, HashMap<String, List<Object>> expectedArray, int expectedHitSize) {
-        SearchResponse sr = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).addScriptField("tvtest", script)
-                .execute().actionGet();
-        assertHitCount(sr, expectedHitSize);
-        int nullCounter = 0;
-        for (SearchHit hit : sr.getHits().getHits()) {
-            Object result = hit.getFields().get("tvtest").getValues();
-            Object expectedResult = expectedArray.get(hit.getId());
-            assertThat("for doc " + hit.getId(), result, equalTo(expectedResult));
-            if (expectedResult != null) {
-                nullCounter++;
-            }
-        }
-        assertThat(nullCounter, equalTo(expectedArray.size()));
-    }
-
-    public void testAllExceptPosAndOffset() throws Exception {
-        XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
-                .startObject("float_payload_field").field("type", "text").field("index_options", "offsets").field("term_vector", "no")
-                .field("analyzer", "payload_float").endObject().startObject("string_payload_field").field("type", "text")
-                .field("index_options", "offsets").field("term_vector", "no").field("analyzer", "payload_string").endObject()
-                .startObject("int_payload_field").field("type", "text").field("index_options", "offsets")
-                .field("analyzer", "payload_int").endObject().endObject().endObject().endObject();
-        assertAcked(prepareCreate("test").addMapping("type1", mapping).setSettings(
-                Settings.builder()
-                        .put(indexSettings())
-                        .put("index.analysis.analyzer.payload_float.tokenizer", "whitespace")
-                        .putArray("index.analysis.analyzer.payload_float.filter", "delimited_float")
-                        .put("index.analysis.filter.delimited_float.delimiter", "|")
-                        .put("index.analysis.filter.delimited_float.encoding", "float")
-                        .put("index.analysis.filter.delimited_float.type", "delimited_payload_filter")
-                        .put("index.analysis.analyzer.payload_string.tokenizer", "whitespace")
-                        .putArray("index.analysis.analyzer.payload_string.filter", "delimited_string")
-                        .put("index.analysis.filter.delimited_string.delimiter", "|")
-                        .put("index.analysis.filter.delimited_string.encoding", "identity")
-                        .put("index.analysis.filter.delimited_string.type", "delimited_payload_filter")
-                        .put("index.analysis.analyzer.payload_int.tokenizer", "whitespace")
-                        .putArray("index.analysis.analyzer.payload_int.filter", "delimited_int")
-                        .put("index.analysis.filter.delimited_int.delimiter", "|")
-                        .put("index.analysis.filter.delimited_int.encoding", "int")
-                        .put("index.analysis.filter.delimited_int.type", "delimited_payload_filter")
-                        .put("index.number_of_shards", 1)));
-        indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("float_payload_field", "a|1 b|2 a|3 b "), client()
-                        .prepareIndex("test", "type1", "2").setSource("string_payload_field", "a|a b|b a|a b "),
-                client().prepareIndex("test", "type1", "3").setSource("float_payload_field", "a|4 b|5 a|6 b "),
-                client().prepareIndex("test", "type1", "4").setSource("string_payload_field", "a|b b|a a|b b "),
-                client().prepareIndex("test", "type1", "5").setSource("float_payload_field", "c "),
-                client().prepareIndex("test", "type1", "6").setSource("int_payload_field", "c|1"));
-
-        // get the number of all docs
-        Script script = createScript("_index.numDocs()");
-        checkValueInEachDoc(6, script, 6);
-
-        // get the number of docs with field float_payload_field
-        script = createScript("_index['float_payload_field'].docCount()");
-        checkValueInEachDoc(3, script, 6);
-
-        // corner case: what if the field does not exist?
-        script = createScript("_index['non_existent_field'].docCount()");
-        checkValueInEachDoc(0, script, 6);
-
-        // get the number of all tokens in all docs
-        script = createScript("_index['float_payload_field'].sumttf()");
-        checkValueInEachDoc(9, script, 6);
-
-        // corner case get the number of all tokens in all docs for non existent
-        // field
-        script = createScript("_index['non_existent_field'].sumttf()");
-        checkValueInEachDoc(0, script, 6);
-
-        // get the sum of doc freqs in all docs
-        script = createScript("_index['float_payload_field'].sumdf()");
-        checkValueInEachDoc(5, script, 6);
-
-        // get the sum of doc freqs in all docs for non existent field
-        script = createScript("_index['non_existent_field'].sumdf()");
-        checkValueInEachDoc(0, script, 6);
-
-        // check term frequencies for 'a'
-        script = createScript("term = _index['float_payload_field']['a']; if (term != null) {term.tf()}");
-        Map<String, Object> expectedResults = new HashMap<>();
-        expectedResults.put("1", 2);
-        expectedResults.put("2", 0);
-        expectedResults.put("3", 2);
-        expectedResults.put("4", 0);
-        expectedResults.put("5", 0);
-        expectedResults.put("6", 0);
-        checkValueInEachDoc(script, expectedResults, 6);
-        expectedResults.clear();
-
-        // check doc frequencies for 'c'
-        script = createScript("term = _index['float_payload_field']['c']; if (term != null) {term.df()}");
-        expectedResults.put("1", 1L);
-        expectedResults.put("2", 1L);
-        expectedResults.put("3", 1L);
-        expectedResults.put("4", 1L);
-        expectedResults.put("5", 1L);
-        expectedResults.put("6", 1L);
-        checkValueInEachDoc(script, expectedResults, 6);
-        expectedResults.clear();
-
-        // check doc frequencies for term that does not exist
-        script = createScript("term = _index['float_payload_field']['non_existent_term']; if (term != null) {term.df()}");
-        expectedResults.put("1", 0L);
-        expectedResults.put("2", 0L);
-        expectedResults.put("3", 0L);
-        expectedResults.put("4", 0L);
-        expectedResults.put("5", 0L);
-        expectedResults.put("6", 0L);
-        checkValueInEachDoc(script, expectedResults, 6);
-        expectedResults.clear();
-
-        // check doc frequencies for term that does not exist
-        script = createScript("term = _index['non_existent_field']['non_existent_term']; if (term != null) {term.tf()}");
-        expectedResults.put("1", 0);
-        expectedResults.put("2", 0);
-        expectedResults.put("3", 0);
-        expectedResults.put("4", 0);
-        expectedResults.put("5", 0);
-        expectedResults.put("6", 0);
-        checkValueInEachDoc(script, expectedResults, 6);
-        expectedResults.clear();
-
-        // check total term frequencies for 'a'
-        script = createScript("term = _index['float_payload_field']['a']; if (term != null) {term.ttf()}");
-        expectedResults.put("1", 4L);
-        expectedResults.put("2", 4L);
-        expectedResults.put("3", 4L);
-        expectedResults.put("4", 4L);
-        expectedResults.put("5", 4L);
-        expectedResults.put("6", 4L);
-        checkValueInEachDoc(script, expectedResults, 6);
-        expectedResults.clear();
-
-        // check float payload for 'b'
-        HashMap<String, List<Object>> expectedPayloadsArray = new HashMap<>();
-        script = createPositionsArrayScript("float_payload_field", "b", INCLUDE_ALL, "payloadAsFloat(-1)");
-        float missingValue = -1;
-        List<Object> payloadsFor1 = new ArrayList<>();
-        payloadsFor1.add(2f);
-        payloadsFor1.add(missingValue);
-        expectedPayloadsArray.put("1", payloadsFor1);
-        List<Object> payloadsFor2 = new ArrayList<>();
-        payloadsFor2.add(5f);
-        payloadsFor2.add(missingValue);
-        expectedPayloadsArray.put("3", payloadsFor2);
-        expectedPayloadsArray.put("6", new ArrayList<>());
-        expectedPayloadsArray.put("5", new ArrayList<>());
-        expectedPayloadsArray.put("4", new ArrayList<>());
-        expectedPayloadsArray.put("2", new ArrayList<>());
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 6);
-
-        // check string payload for 'b'
-        expectedPayloadsArray.clear();
-        payloadsFor1.clear();
-        payloadsFor2.clear();
-        script = createPositionsArrayScript("string_payload_field", "b", INCLUDE_ALL, "payloadAsString()");
-        payloadsFor1.add("b");
-        payloadsFor1.add(null);
-        expectedPayloadsArray.put("2", payloadsFor1);
-        payloadsFor2.add("a");
-        payloadsFor2.add(null);
-        expectedPayloadsArray.put("4", payloadsFor2);
-        expectedPayloadsArray.put("6", new ArrayList<>());
-        expectedPayloadsArray.put("5", new ArrayList<>());
-        expectedPayloadsArray.put("3", new ArrayList<>());
-        expectedPayloadsArray.put("1", new ArrayList<>());
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 6);
-
-        // check int payload for 'c'
-        expectedPayloadsArray.clear();
-        payloadsFor1.clear();
-        payloadsFor2.clear();
-        script = createPositionsArrayScript("int_payload_field", "c", INCLUDE_ALL, "payloadAsInt(-1)");
-        payloadsFor1 = new ArrayList<>();
-        payloadsFor1.add(1);
-        expectedPayloadsArray.put("6", payloadsFor1);
-        expectedPayloadsArray.put("5", new ArrayList<>());
-        expectedPayloadsArray.put("4", new ArrayList<>());
-        expectedPayloadsArray.put("3", new ArrayList<>());
-        expectedPayloadsArray.put("2", new ArrayList<>());
-        expectedPayloadsArray.put("1", new ArrayList<>());
-        checkArrayValsInEachDoc(script, expectedPayloadsArray, 6);
-
-    }
-
-    private void checkExceptions(Script script) {
-        try {
-            SearchResponse sr = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).addScriptField("tvtest", script)
-                    .execute().actionGet();
-            assertThat(sr.getHits().getHits().length, equalTo(0));
-            ShardSearchFailure[] shardFails = sr.getShardFailures();
-            for (ShardSearchFailure fail : shardFails) {
-                assertThat(fail.reason().indexOf("Cannot iterate twice! If you want to iterate more that once, add _CACHE explicitly."),
-                        Matchers.greaterThan(-1));
-            }
-        } catch (SearchPhaseExecutionException ex) {
-            assertThat(
-                    "got " + ex.toString(),
-                    ex.toString().indexOf("Cannot iterate twice! If you want to iterate more that once, add _CACHE explicitly."),
-                    Matchers.greaterThan(-1));
-        }
-    }
-
-    private void checkValueInEachDocWithFunctionScore(Script fieldScript, Map<String, Object> expectedFieldVals, Script scoreScript,
-                                                      Map<String, Object> expectedScore, int numExpectedDocs) {
-        SearchResponse sr = client().prepareSearch("test")
-                .setQuery(QueryBuilders.functionScoreQuery(ScoreFunctionBuilders.scriptFunction(scoreScript)))
-                .addScriptField("tvtest", fieldScript).execute().actionGet();
-        assertHitCount(sr, numExpectedDocs);
-        for (SearchHit hit : sr.getHits().getHits()) {
-            Object result = hit.getFields().get("tvtest").getValues().get(0);
-            Object expectedResult = expectedFieldVals.get(hit.getId());
-            assertThat("for doc " + hit.getId(), result, equalTo(expectedResult));
-            assertThat("for doc " + hit.getId(), ((Float) expectedScore.get(hit.getId())).doubleValue(),
-                    Matchers.closeTo(hit.getScore(), 1.e-4));
-        }
-    }
-
-    private void checkValueInEachDoc(Script script, Map<String, Object> expectedResults, int numExpectedDocs) {
-        SearchResponse sr = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).addScriptField("tvtest", script)
-                .execute().actionGet();
-        assertHitCount(sr, numExpectedDocs);
-        for (SearchHit hit : sr.getHits().getHits()) {
-            Object result = hit.getFields().get("tvtest").getValues().get(0);
-            Object expectedResult = expectedResults.get(hit.getId());
-            assertThat("for doc " + hit.getId(), result, equalTo(expectedResult));
-        }
-    }
-
-    private void checkValueInEachDoc(int value, Script script, int numExpectedDocs) {
-        SearchResponse sr = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()).addScriptField("tvtest", script)
-                .execute().actionGet();
-        assertHitCount(sr, numExpectedDocs);
-        for (SearchHit hit : sr.getHits().getHits()) {
-            Object result = hit.getFields().get("tvtest").getValues().get(0);
-            if (result instanceof Integer) {
-                assertThat(result, equalTo(value));
-            } else if (result instanceof Long) {
-                assertThat(((Long) result).intValue(), equalTo(value));
-            } else {
-                fail();
-            }
-        }
-    }
-}

+ 5 - 1
docs/reference/migration/migrate_6_0/scripting.asciidoc

@@ -13,7 +13,11 @@ milliseconds since epoch as a `long`. The same is true for
 `doc.some_date_field[some_number]`. Use `doc.some_date_field.value.millis` to
 fetch the milliseconds since epoch if you need it.
 
+==== Removed access to index internal via the _index variable
+
+The `_index` variable has been removed. If you used it for advanced scoring, consider writing a `Similarity` plugin.
+
 ==== Script Settings
 
 All of the existing scripting security settings have been deprecated.  Instead
-they are replaced with `script.allowed_types` and `script.allowed_contexts`.
+they are replaced with `script.allowed_types` and `script.allowed_contexts`.

+ 0 - 4
docs/reference/modules/scripting/fields.asciidoc

@@ -29,10 +29,6 @@ Field values can be accessed from a script using
 <<modules-scripting-doc-vals,doc-values>>, or
 <<modules-scripting-stored,stored fields or `_source` field>>, which are explained below.
 
-Scripts may also have access to the document's relevance
-<<scripting-score,`_score`>> and, via the experimental `_index` variable,
-to term statistics for <<modules-advanced-scripting,advanced text scoring>>.
-
 [[scripting-score]]
 [float]
 === Accessing the score of a document within a script