Simplify the Text API.

We have the Text API, which is essentially a wrapper around a String and a
BytesReference and then we have 3 implementations depending on whether the
String view should be cached, the BytesReference view should be cached, or both
should be cached.

This commit merges everything into a single Text that is essentially the old
StringAndBytesText impl.

Long term we should look into whether this API has any performance benefit or
if we could just use plain strings. This would greatly simplify all our other
APIs that currently use Text.
This commit is contained in:
Adrien Grand 2015-12-17 14:54:57 +01:00
parent b2b7b30d10
commit 6ea16671f4
26 changed files with 140 additions and 380 deletions

View File

@ -20,7 +20,6 @@
package org.elasticsearch.action.percolate;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.IndicesRequest;
@ -37,8 +36,6 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.threadpool.ThreadPool;

View File

@ -50,7 +50,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
@ -357,7 +357,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
timeInQueue = 0;
}
pendingClusterTasks.add(new PendingClusterTask(pending.insertionOrder, pending.priority, new StringText(source), timeInQueue, pending.executing));
pendingClusterTasks.add(new PendingClusterTask(pending.insertionOrder, pending.priority, new Text(source), timeInQueue, pending.executing));
}
return pendingClusterTasks;
}

View File

@ -33,7 +33,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
@ -256,13 +256,13 @@ public abstract class StreamInput extends InputStream {
if (length == -1) {
return null;
}
return new StringAndBytesText(readBytesReference(length));
return new Text(readBytesReference(length));
}
public Text readText() throws IOException {
// use StringAndBytes so we can cache the string if its ever converted to it
int length = readInt();
return new StringAndBytesText(readBytesReference(length));
return new Text(readBytesReference(length));
}
@Nullable

View File

@ -1,82 +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.common.text;
import java.nio.charset.StandardCharsets;
import org.elasticsearch.common.bytes.BytesReference;
/**
* A {@link BytesReference} representation of the text, will always convert on the fly to a {@link String}.
*/
public class BytesText implements Text {
private BytesReference bytes;
private int hash;
public BytesText(BytesReference bytes) {
this.bytes = bytes;
}
@Override
public boolean hasBytes() {
return true;
}
@Override
public BytesReference bytes() {
return bytes;
}
@Override
public boolean hasString() {
return false;
}
@Override
public String string() {
// TODO: we can optimize the conversion based on the bytes reference API similar to UnicodeUtil
if (!bytes.hasArray()) {
bytes = bytes.toBytesArray();
}
return new String(bytes.array(), bytes.arrayOffset(), bytes.length(), StandardCharsets.UTF_8);
}
@Override
public String toString() {
return string();
}
@Override
public int hashCode() {
if (hash == 0) {
hash = bytes.hashCode();
}
return hash;
}
@Override
public boolean equals(Object obj) {
return bytes().equals(((Text) obj).bytes());
}
@Override
public int compareTo(Text text) {
return UTF8SortedAsUnicodeComparator.utf8SortedAsUnicodeSortOrder.compare(bytes(), text.bytes());
}
}

View File

@ -1,111 +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.common.text;
import java.nio.charset.StandardCharsets;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
/**
* Both {@link String} and {@link BytesReference} representation of the text. Starts with one of those, and if
* the other is requests, caches the other one in a local reference so no additional conversion will be needed.
*/
public class StringAndBytesText implements Text {
public static final Text[] EMPTY_ARRAY = new Text[0];
public static Text[] convertFromStringArray(String[] strings) {
if (strings.length == 0) {
return EMPTY_ARRAY;
}
Text[] texts = new Text[strings.length];
for (int i = 0; i < strings.length; i++) {
texts[i] = new StringAndBytesText(strings[i]);
}
return texts;
}
private BytesReference bytes;
private String text;
private int hash;
public StringAndBytesText(BytesReference bytes) {
this.bytes = bytes;
}
public StringAndBytesText(String text) {
this.text = text;
}
@Override
public boolean hasBytes() {
return bytes != null;
}
@Override
public BytesReference bytes() {
if (bytes == null) {
bytes = new BytesArray(text.getBytes(StandardCharsets.UTF_8));
}
return bytes;
}
@Override
public boolean hasString() {
return text != null;
}
@Override
public String string() {
// TODO: we can optimize the conversion based on the bytes reference API similar to UnicodeUtil
if (text == null) {
if (!bytes.hasArray()) {
bytes = bytes.toBytesArray();
}
text = new String(bytes.array(), bytes.arrayOffset(), bytes.length(), StandardCharsets.UTF_8);
}
return text;
}
@Override
public String toString() {
return string();
}
@Override
public int hashCode() {
if (hash == 0) {
hash = bytes().hashCode();
}
return hash;
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
return bytes().equals(((Text) obj).bytes());
}
@Override
public int compareTo(Text text) {
return UTF8SortedAsUnicodeComparator.utf8SortedAsUnicodeSortOrder.compare(bytes(), text.bytes());
}
}

View File

@ -1,94 +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.common.text;
import java.nio.charset.StandardCharsets;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
/**
* A {@link String} only representation of the text. Will always convert to bytes on the fly.
*/
public class StringText implements Text {
public static final Text[] EMPTY_ARRAY = new Text[0];
public static Text[] convertFromStringArray(String[] strings) {
if (strings.length == 0) {
return EMPTY_ARRAY;
}
Text[] texts = new Text[strings.length];
for (int i = 0; i < strings.length; i++) {
texts[i] = new StringText(strings[i]);
}
return texts;
}
private final String text;
private int hash;
public StringText(String text) {
this.text = text;
}
@Override
public boolean hasBytes() {
return false;
}
@Override
public BytesReference bytes() {
return new BytesArray(text.getBytes(StandardCharsets.UTF_8));
}
@Override
public boolean hasString() {
return true;
}
@Override
public String string() {
return text;
}
@Override
public String toString() {
return string();
}
@Override
public int hashCode() {
// we use bytes here so we can be consistent with other text implementations
if (hash == 0) {
hash = bytes().hashCode();
}
return hash;
}
@Override
public boolean equals(Object obj) {
// we use bytes here so we can be consistent with other text implementations
return bytes().equals(((Text) obj).bytes());
}
@Override
public int compareTo(Text text) {
return UTF8SortedAsUnicodeComparator.utf8SortedAsUnicodeSortOrder.compare(bytes(), text.bytes());
}
}

View File

@ -18,39 +18,101 @@
*/
package org.elasticsearch.common.text;
import java.nio.charset.StandardCharsets;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
/**
* Text represents a (usually) long text data. We use this abstraction instead of {@link String}
* so we can represent it in a more optimized manner in memory as well as serializing it over the
* network as well as converting it to json format.
* Both {@link String} and {@link BytesReference} representation of the text. Starts with one of those, and if
* the other is requests, caches the other one in a local reference so no additional conversion will be needed.
*/
public interface Text extends Comparable<Text> {
public final class Text implements Comparable<Text> {
public static final Text[] EMPTY_ARRAY = new Text[0];
public static Text[] convertFromStringArray(String[] strings) {
if (strings.length == 0) {
return EMPTY_ARRAY;
}
Text[] texts = new Text[strings.length];
for (int i = 0; i < strings.length; i++) {
texts[i] = new Text(strings[i]);
}
return texts;
}
private BytesReference bytes;
private String text;
private int hash;
public Text(BytesReference bytes) {
this.bytes = bytes;
}
public Text(String text) {
this.text = text;
}
/**
* Are bytes available without the need to be converted into bytes when calling {@link #bytes()}.
* Whether a {@link BytesReference} view of the data is already materialized.
*/
boolean hasBytes();
public boolean hasBytes() {
return bytes != null;
}
/**
* The UTF8 bytes representing the the text, might be converted on the fly, see {@link #hasBytes()}
* Returns a {@link BytesReference} view of the data.
*/
BytesReference bytes();
public BytesReference bytes() {
if (bytes == null) {
bytes = new BytesArray(text.getBytes(StandardCharsets.UTF_8));
}
return bytes;
}
/**
* Is there a {@link String} representation of the text. If not, then it {@link #hasBytes()}.
* Whether a {@link String} view of the data is already materialized.
*/
boolean hasString();
public boolean hasString() {
return text != null;
}
/**
* Returns the string representation of the text, might be converted to a string on the fly.
* Returns a {@link String} view of the data.
*/
String string();
public String string() {
if (text == null) {
if (!bytes.hasArray()) {
bytes = bytes.toBytesArray();
}
text = new String(bytes.array(), bytes.arrayOffset(), bytes.length(), StandardCharsets.UTF_8);
}
return text;
}
/**
* Returns the string representation of the text, might be converted to a string on the fly.
*/
@Override
String toString();
public String toString() {
return string();
}
@Override
public int hashCode() {
if (hash == 0) {
hash = bytes().hashCode();
}
return hash;
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
return bytes().equals(((Text) obj).bytes());
}
@Override
public int compareTo(Text text) {
return UTF8SortedAsUnicodeComparator.utf8SortedAsUnicodeSortOrder.compare(bytes(), text.bytes());
}
}

View File

@ -29,7 +29,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.common.xcontent.ToXContent;
@ -114,7 +114,7 @@ public class DocumentMapper implements ToXContent {
private final MapperService mapperService;
private final String type;
private final StringAndBytesText typeText;
private final Text typeText;
private volatile CompressedXContent mappingSource;
@ -138,7 +138,7 @@ public class DocumentMapper implements ToXContent {
ReentrantReadWriteLock mappingLock) {
this.mapperService = mapperService;
this.type = rootObjectMapper.name();
this.typeText = new StringAndBytesText(this.type);
this.typeText = new Text(this.type);
this.mapping = new Mapping(
Version.indexCreated(indexSettings),
rootObjectMapper,

View File

@ -38,7 +38,7 @@ import org.elasticsearch.common.HasHeaders;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.analysis.AnalysisService;
@ -164,7 +164,7 @@ public class PercolateContext extends SearchContext {
fields.put(field.name(), new InternalSearchHitField(field.name(), Collections.emptyList()));
}
hitContext().reset(
new InternalSearchHit(0, "unknown", new StringText(parsedDocument.type()), fields),
new InternalSearchHit(0, "unknown", new Text(parsedDocument.type()), fields),
atomicReaderContext, 0, docSearcher.searcher()
);
}

View File

@ -52,8 +52,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.BytesText;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
@ -533,10 +531,10 @@ public class PercolatorService extends AbstractComponent {
List<PercolateResponse.Match> finalMatches = new ArrayList<>(requestedSize == 0 ? numMatches : requestedSize);
outer:
for (PercolateShardResponse response : shardResults) {
Text index = new StringText(response.getIndex());
Text index = new Text(response.getIndex());
for (int i = 0; i < response.matches().length; i++) {
float score = response.scores().length == 0 ? NO_SCORE : response.scores()[i];
Text match = new BytesText(new BytesArray(response.matches()[i]));
Text match = new Text(new BytesArray(response.matches()[i]));
Map<String, HighlightField> hl = response.hls().isEmpty() ? null : response.hls().get(i);
finalMatches.add(new PercolateResponse.Match(index, match, score, hl));
if (requestedSize != 0 && finalMatches.size() == requestedSize) {
@ -686,10 +684,10 @@ public class PercolatorService extends AbstractComponent {
List<PercolateResponse.Match> finalMatches = new ArrayList<>(requestedSize);
if (nonEmptyResponses == 1) {
PercolateShardResponse response = shardResults.get(firstNonEmptyIndex);
Text index = new StringText(response.getIndex());
Text index = new Text(response.getIndex());
for (int i = 0; i < response.matches().length; i++) {
float score = response.scores().length == 0 ? Float.NaN : response.scores()[i];
Text match = new BytesText(new BytesArray(response.matches()[i]));
Text match = new Text(new BytesArray(response.matches()[i]));
if (!response.hls().isEmpty()) {
Map<String, HighlightField> hl = response.hls().get(i);
finalMatches.add(new PercolateResponse.Match(index, match, score, hl));
@ -728,8 +726,8 @@ public class PercolatorService extends AbstractComponent {
slots[requestIndex]++;
PercolateShardResponse shardResponse = shardResults.get(requestIndex);
Text index = new StringText(shardResponse.getIndex());
Text match = new BytesText(new BytesArray(shardResponse.matches()[itemIndex]));
Text index = new Text(shardResponse.getIndex());
Text match = new Text(new BytesArray(shardResponse.matches()[itemIndex]));
float score = shardResponse.scores()[itemIndex];
if (!shardResponse.hls().isEmpty()) {
Map<String, HighlightField> hl = shardResponse.hls().get(itemIndex);

View File

@ -23,7 +23,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import java.io.IOException;
@ -42,8 +42,8 @@ public class SearchShardTarget implements Streamable, Comparable<SearchShardTarg
}
public SearchShardTarget(String nodeId, String index, int shardId) {
this.nodeId = nodeId == null ? null : new StringAndBytesText(nodeId);
this.index = new StringAndBytesText(index);
this.nodeId = nodeId == null ? null : new Text(nodeId);
this.index = new Text(index);
this.shardId = shardId;
}

View File

@ -23,7 +23,6 @@ import org.apache.lucene.util.PriorityQueue;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
@ -151,7 +150,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (formatter != ValueFormatter.RAW) {
Text keyTxt = new StringText(formatter.format(key));
Text keyTxt = new Text(formatter.format(key));
if (keyed) {
builder.startObject(keyTxt.string());
} else {

View File

@ -31,7 +31,7 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
@ -198,7 +198,7 @@ public class FetchPhase implements SearchPhase {
DocumentMapper documentMapper = context.mapperService().documentMapper(fieldsVisitor.uid().type());
Text typeText;
if (documentMapper == null) {
typeText = new StringAndBytesText(fieldsVisitor.uid().type());
typeText = new Text(fieldsVisitor.uid().type());
} else {
typeText = documentMapper.typeText();
}

View File

@ -33,7 +33,7 @@ import org.apache.lucene.search.vectorhighlight.SimpleFieldFragList;
import org.apache.lucene.search.vectorhighlight.SimpleFragListBuilder;
import org.apache.lucene.search.vectorhighlight.SingleFragListBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
@ -159,7 +159,7 @@ public class FastVectorHighlighter implements Highlighter {
}
if (fragments != null && fragments.length > 0) {
return new HighlightField(highlighterContext.fieldName, StringText.convertFromStringArray(fragments));
return new HighlightField(highlighterContext.fieldName, Text.convertFromStringArray(fragments));
}
int noMatchSize = highlighterContext.field.fieldOptions().noMatchSize();
@ -170,7 +170,7 @@ public class FastVectorHighlighter implements Highlighter {
fragments = entry.fragmentsBuilder.createFragments(hitContext.reader(), hitContext.docId(), mapper.fieldType().names().indexName(),
fieldFragList, 1, field.fieldOptions().preTags(), field.fieldOptions().postTags(), encoder);
if (fragments != null && fragments.length > 0) {
return new HighlightField(highlighterContext.fieldName, StringText.convertFromStringArray(fragments));
return new HighlightField(highlighterContext.fieldName, Text.convertFromStringArray(fragments));
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.highlight;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import java.io.IOException;
@ -90,7 +89,7 @@ public class HighlightField implements Streamable {
if (in.readBoolean()) {
int size = in.readVInt();
if (size == 0) {
fragments = StringText.EMPTY_ARRAY;
fragments = Text.EMPTY_ARRAY;
} else {
fragments = new Text[size];
for (int i = 0; i < size; i++) {

View File

@ -33,9 +33,7 @@ import org.apache.lucene.search.highlight.SimpleSpanFragmenter;
import org.apache.lucene.search.highlight.TextFragment;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
@ -158,7 +156,7 @@ public class PlainHighlighter implements Highlighter {
}
if (fragments.length > 0) {
return new HighlightField(highlighterContext.fieldName, StringText.convertFromStringArray(fragments));
return new HighlightField(highlighterContext.fieldName, Text.convertFromStringArray(fragments));
}
int noMatchSize = highlighterContext.field.fieldOptions().noMatchSize();
@ -172,7 +170,7 @@ public class PlainHighlighter implements Highlighter {
throw new FetchPhaseExecutionException(context, "Failed to highlight field [" + highlighterContext.fieldName + "]", e);
}
if (end > 0) {
return new HighlightField(highlighterContext.fieldName, new Text[] { new StringText(fieldContents.substring(0, end)) });
return new HighlightField(highlighterContext.fieldName, new Text[] { new Text(fieldContents.substring(0, end)) });
}
}
return null;

View File

@ -28,7 +28,7 @@ import org.apache.lucene.search.postingshighlight.CustomSeparatorBreakIterator;
import org.apache.lucene.search.postingshighlight.Snippet;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.search.fetch.FetchPhaseExecutionException;
import org.elasticsearch.search.fetch.FetchSubPhase;
@ -122,7 +122,7 @@ public class PostingsHighlighter implements Highlighter {
}
if (fragments.length > 0) {
return new HighlightField(highlighterContext.fieldName, StringText.convertFromStringArray(fragments));
return new HighlightField(highlighterContext.fieldName, Text.convertFromStringArray(fragments));
}
return null;

View File

@ -30,7 +30,7 @@ import org.elasticsearch.common.compress.CompressorFactory;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -104,14 +104,14 @@ public class InternalSearchHit implements SearchHit {
public InternalSearchHit(int docId, String id, Text type, Map<String, SearchHitField> fields) {
this.docId = docId;
this.id = new StringAndBytesText(id);
this.id = new Text(id);
this.type = type;
this.fields = fields;
}
public InternalSearchHit(int nestedTopDocId, String id, Text type, InternalNestedIdentity nestedIdentity, Map<String, SearchHitField> fields) {
this.docId = nestedTopDocId;
this.id = new StringAndBytesText(id);
this.id = new Text(id);
this.type = type;
this.nestedIdentity = nestedIdentity;
this.fields = fields;
@ -339,7 +339,7 @@ public class InternalSearchHit implements SearchHit {
if (sortValues != null) {
for (int i = 0; i < sortValues.length; i++) {
if (sortValues[i] instanceof BytesRef) {
sortValuesCopy[i] = new StringAndBytesText(new BytesArray((BytesRef) sortValues[i]));
sortValuesCopy[i] = new Text(new BytesArray((BytesRef) sortValues[i]));
}
}
}
@ -783,7 +783,7 @@ public class InternalSearchHit implements SearchHit {
private InternalNestedIdentity child;
public InternalNestedIdentity(String field, int offset, InternalNestedIdentity child) {
this.field = new StringAndBytesText(field);
this.field = new Text(field);
this.offset = offset;
this.child = child;
}

View File

@ -30,7 +30,7 @@ import org.apache.lucene.search.suggest.document.TopSuggestDocs;
import org.apache.lucene.search.suggest.document.TopSuggestDocsCollector;
import org.apache.lucene.util.*;
import org.apache.lucene.util.PriorityQueue;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.mapper.MappedFieldType;
@ -57,7 +57,7 @@ public class CompletionSuggester extends Suggester<CompletionSuggestionContext>
}
CompletionSuggestion completionSuggestion = new CompletionSuggestion(name, suggestionContext.getSize());
spare.copyUTF8Bytes(suggestionContext.getText());
CompletionSuggestion.Entry completionSuggestEntry = new CompletionSuggestion.Entry(new StringText(spare.toString()), 0, spare.length());
CompletionSuggestion.Entry completionSuggestEntry = new CompletionSuggestion.Entry(new Text(spare.toString()), 0, spare.length());
completionSuggestion.addTerm(completionSuggestEntry);
TopSuggestDocsCollector collector = new TopDocumentsCollector(suggestionContext.getSize());
suggest(searcher, suggestionContext.toQuery(), collector);
@ -91,7 +91,7 @@ public class CompletionSuggester extends Suggester<CompletionSuggestionContext>
}
if (numResult++ < suggestionContext.getSize()) {
CompletionSuggestion.Entry.Option option = new CompletionSuggestion.Entry.Option(
new StringText(suggestDoc.key.toString()), suggestDoc.score, contexts, payload);
new Text(suggestDoc.key.toString()), suggestDoc.score, contexts, payload);
completionSuggestEntry.addOption(option);
} else {
break;

View File

@ -30,7 +30,6 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.ParsedQuery;
@ -127,11 +126,11 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
if (!collateMatch && !collatePrune) {
continue;
}
Text phrase = new StringText(spare.toString());
Text phrase = new Text(spare.toString());
Text highlighted = null;
if (suggestion.getPreTag() != null) {
spare.copyUTF8Bytes(correction.join(SEPARATOR, byteSpare, suggestion.getPreTag(), suggestion.getPostTag()));
highlighted = new StringText(spare.toString());
highlighted = new Text(spare.toString());
}
if (collatePrune) {
resultEntry.addOption(new Suggestion.Entry.Option(phrase, highlighted, (float) (correction.score), collateMatch));
@ -147,7 +146,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
private PhraseSuggestion.Entry buildResultEntry(PhraseSuggestionContext suggestion, CharsRefBuilder spare, double cutoffScore) {
spare.copyUTF8Bytes(suggestion.getText());
return new PhraseSuggestion.Entry(new StringText(spare.toString()), 0, spare.length(), cutoffScore);
return new PhraseSuggestion.Entry(new Text(spare.toString()), 0, spare.length(), cutoffScore);
}
ScriptService scriptService() {

View File

@ -27,8 +27,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.text.BytesText;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.search.suggest.SuggestContextParser;
import org.elasticsearch.search.suggest.SuggestUtils;
@ -54,10 +52,10 @@ public final class TermSuggester extends Suggester<TermSuggestionContext> {
SuggestWord[] suggestedWords = directSpellChecker.suggestSimilar(
token.term, suggestion.getShardSize(), indexReader, suggestion.getDirectSpellCheckerSettings().suggestMode()
);
Text key = new BytesText(new BytesArray(token.term.bytes()));
Text key = new Text(new BytesArray(token.term.bytes()));
TermSuggestion.Entry resultEntry = new TermSuggestion.Entry(key, token.startOffset, token.endOffset - token.startOffset);
for (SuggestWord suggestWord : suggestedWords) {
Text word = new StringText(suggestWord.string);
Text word = new Text(suggestWord.string);
resultEntry.addOption(new TermSuggestion.Entry.Option(word, suggestWord.freq, suggestWord.score));
}
response.addTerm(resultEntry);

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.search.highlight;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.mapper.FieldMapper;
@ -52,12 +51,12 @@ public class CustomHighlighter implements Highlighter {
}
List<Text> responses = new ArrayList<>();
responses.add(new StringText(String.format(Locale.ENGLISH, "standard response for %s at position %s", field.field(),
responses.add(new Text(String.format(Locale.ENGLISH, "standard response for %s at position %s", field.field(),
cacheEntry.position)));
if (field.fieldOptions().options() != null) {
for (Map.Entry<String, Object> entry : field.fieldOptions().options().entrySet()) {
responses.add(new StringText("field:" + entry.getKey() + ":" + entry.getValue()));
responses.add(new Text("field:" + entry.getKey() + ":" + entry.getValue()));
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.internal;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.test.ESTestCase;
@ -39,25 +39,25 @@ public class InternalSearchHitTests extends ESTestCase {
SearchShardTarget target = new SearchShardTarget("_node_id", "_index", 0);
Map<String, InternalSearchHits> innerHits = new HashMap<>();
InternalSearchHit innerHit1 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit innerHit1 = new InternalSearchHit(0, "_id", new Text("_type"), null);
innerHit1.shardTarget(target);
InternalSearchHit innerInnerHit2 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit innerInnerHit2 = new InternalSearchHit(0, "_id", new Text("_type"), null);
innerInnerHit2.shardTarget(target);
innerHits.put("1", new InternalSearchHits(new InternalSearchHit[]{innerInnerHit2}, 1, 1f));
innerHit1.setInnerHits(innerHits);
InternalSearchHit innerHit2 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit innerHit2 = new InternalSearchHit(0, "_id", new Text("_type"), null);
innerHit2.shardTarget(target);
InternalSearchHit innerHit3 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit innerHit3 = new InternalSearchHit(0, "_id", new Text("_type"), null);
innerHit3.shardTarget(target);
innerHits = new HashMap<>();
InternalSearchHit hit1 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit hit1 = new InternalSearchHit(0, "_id", new Text("_type"), null);
innerHits.put("1", new InternalSearchHits(new InternalSearchHit[]{innerHit1, innerHit2}, 1, 1f));
innerHits.put("2", new InternalSearchHits(new InternalSearchHit[]{innerHit3}, 1, 1f));
hit1.shardTarget(target);
hit1.setInnerHits(innerHits);
InternalSearchHit hit2 = new InternalSearchHit(0, "_id", new StringText("_type"), null);
InternalSearchHit hit2 = new InternalSearchHit(0, "_id", new Text("_type"), null);
hit2.shardTarget(target);
InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[]{hit1, hit2}, 2, 1f);

View File

@ -20,7 +20,7 @@ package org.elasticsearch.search.suggest;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.CharsRefBuilder;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import java.io.IOException;
import java.util.Locale;
@ -42,11 +42,11 @@ public class CustomSuggester extends Suggester<CustomSuggester.CustomSuggestions
Suggest.Suggestion<Suggest.Suggestion.Entry<Suggest.Suggestion.Entry.Option>> response = new Suggest.Suggestion<>(name, suggestion.getSize());
String firstSuggestion = String.format(Locale.ROOT, "%s-%s-%s-%s", text, suggestion.getField(), suggestion.options.get("suffix"), "12");
Suggest.Suggestion.Entry<Suggest.Suggestion.Entry.Option> resultEntry12 = new Suggest.Suggestion.Entry<>(new StringText(firstSuggestion), 0, text.length() + 2);
Suggest.Suggestion.Entry<Suggest.Suggestion.Entry.Option> resultEntry12 = new Suggest.Suggestion.Entry<>(new Text(firstSuggestion), 0, text.length() + 2);
response.addTerm(resultEntry12);
String secondSuggestion = String.format(Locale.ROOT, "%s-%s-%s-%s", text, suggestion.getField(), suggestion.options.get("suffix"), "123");
Suggest.Suggestion.Entry<Suggest.Suggestion.Entry.Option> resultEntry123 = new Suggest.Suggestion.Entry<>(new StringText(secondSuggestion), 0, text.length() + 3);
Suggest.Suggestion.Entry<Suggest.Suggestion.Entry.Option> resultEntry123 = new Suggest.Suggestion.Entry<>(new Text(secondSuggestion), 0, text.length() + 3);
response.addTerm(resultEntry123);
return response;

View File

@ -34,7 +34,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.StringAndBytesText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -1633,8 +1633,7 @@ public class SimpleSortTests extends ESIntegTestCase {
assertThat(hits[i].getSortValues().length, is(1));
Object o = hits[i].getSortValues()[0];
assertThat(o, notNullValue());
assertThat(o instanceof StringAndBytesText, is(true));
StringAndBytesText text = (StringAndBytesText) o;
Text text = (Text) o;
assertThat(text.string(), is("bar"));
}
@ -1650,8 +1649,7 @@ public class SimpleSortTests extends ESIntegTestCase {
assertThat(hits[i].getSortValues().length, is(1));
Object o = hits[i].getSortValues()[0];
assertThat(o, notNullValue());
assertThat(o instanceof StringAndBytesText, is(true));
StringAndBytesText text = (StringAndBytesText) o;
Text text = (Text) o;
assertThat(text.string(), is("bar bar"));
}
}
@ -1925,7 +1923,7 @@ public class SimpleSortTests extends ESIntegTestCase {
.addSort(fieldSort("str_field2").order(SortOrder.DESC).unmappedType("string")).get();
assertSortValues(resp,
new Object[] {new StringAndBytesText("bcd"), null},
new Object[] {new Text("bcd"), null},
new Object[] {null, null});
resp = client().prepareSearch("test1", "test2")

View File

@ -28,7 +28,7 @@ import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.util.concurrent.CountDown;
@ -339,7 +339,7 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase {
final int nbDocs = randomIntBetween(0, 20);
SearchHit[] docs = new SearchHit[nbDocs];
for (int i = 0; i < nbDocs; i++) {
InternalSearchHit doc = new InternalSearchHit(randomInt(), String.valueOf(i), new StringText("type"), null);
InternalSearchHit doc = new InternalSearchHit(randomInt(), String.valueOf(i), new Text("type"), null);
doc.shard(new SearchShardTarget("node", "test", randomInt()));
docs[i] = doc;
}