Merge branch 'fielddata'

This commit is contained in:
Shay Banon 2013-01-22 16:17:15 +01:00
commit 1185d4eb19
268 changed files with 22382 additions and 15558 deletions

View File

@ -127,10 +127,10 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastOperatio
if (request.fieldDataCache()) {
clearedAtLeastOne = true;
if (request.fields() == null || request.fields().length == 0) {
service.cache().fieldData().clear("api");
service.fieldData().clear();
} else {
for (String field : request.fields()) {
service.cache().fieldData().clear("api", field);
service.fieldData().clearField(field);
}
}
}
@ -142,7 +142,7 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastOperatio
if (request.fields() != null && request.fields().length > 0) {
// only clear caches relating to the specified fields
for (String field : request.fields()) {
service.cache().fieldData().clear("api", field);
service.fieldData().clearField(field);
}
} else {
service.cache().clear("api");

View File

@ -0,0 +1,86 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.lucene;
import org.apache.lucene.util.BytesRef;
/**
* A wrapped to {@link BytesRef} that also caches the hashCode for it.
*/
public class HashedBytesRef {
public BytesRef bytes;
public int hash;
public HashedBytesRef() {
}
public HashedBytesRef(String bytes) {
this(new BytesRef(bytes));
}
public HashedBytesRef(BytesRef bytes) {
this(bytes, bytes.hashCode());
}
public HashedBytesRef(BytesRef bytes, int hash) {
this.bytes = bytes;
this.hash = hash;
}
public HashedBytesRef resetHashCode() {
this.hash = bytes.hashCode();
return this;
}
public HashedBytesRef reset(BytesRef bytes, int hash) {
this.bytes = bytes;
this.hash = hash;
return this;
}
@Override
public int hashCode() {
return hash;
}
@Override
public boolean equals(Object other) {
if (other instanceof HashedBytesRef) {
return bytes.equals(((HashedBytesRef) other).bytes);
}
return false;
}
@Override
public String toString() {
return bytes.toString();
}
public HashedBytesRef deepCopy() {
return deepCopyOf(this);
}
public static HashedBytesRef deepCopyOf(HashedBytesRef other) {
BytesRef copy = new BytesRef();
copy.copyBytes(other.bytes);
return new HashedBytesRef(copy, other.hash);
}
}

View File

@ -32,7 +32,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.analysis.AnalyzerScope;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import java.io.IOException;
import java.lang.reflect.Field;
@ -215,7 +215,7 @@ public class Lucene {
out.writeString(sortField.getField());
}
if (sortField.getComparatorSource() != null) {
writeSortType(out, ((FieldDataType.ExtendedFieldComparatorSource) sortField.getComparatorSource()).reducedType());
writeSortType(out, ((IndexFieldData.XFieldComparatorSource) sortField.getComparatorSource()).reducedType());
} else {
writeSortType(out, sortField.getType());
}

View File

@ -34,42 +34,28 @@ import java.io.IOException;
*/
public class CacheStats implements Streamable, ToXContent {
long fieldEvictions;
long filterEvictions;
long filterCount;
long fieldSize;
long filterSize;
long idCacheSize;
public CacheStats() {
}
public CacheStats(long fieldEvictions, long filterEvictions, long fieldSize, long filterSize, long filterCount, long idCacheSize) {
this.fieldEvictions = fieldEvictions;
public CacheStats(long filterEvictions, long filterSize, long filterCount, long idCacheSize) {
this.filterEvictions = filterEvictions;
this.fieldSize = fieldSize;
this.filterSize = filterSize;
this.filterCount = filterCount;
this.idCacheSize = idCacheSize;
}
public void add(CacheStats stats) {
this.fieldEvictions += stats.fieldEvictions;
this.filterEvictions += stats.filterEvictions;
this.fieldSize += stats.fieldSize;
this.filterSize += stats.filterSize;
this.filterCount += stats.filterCount;
this.idCacheSize += stats.idCacheSize;
}
public long fieldEvictions() {
return this.fieldEvictions;
}
public long getFieldEvictions() {
return this.fieldEvictions();
}
public long filterEvictions() {
return this.filterEvictions;
}
@ -94,22 +80,6 @@ public class CacheStats implements Streamable, ToXContent {
return filterCount;
}
public long fieldSizeInBytes() {
return this.fieldSize;
}
public long getFieldSizeInBytes() {
return fieldSizeInBytes();
}
public ByteSizeValue fieldSize() {
return new ByteSizeValue(fieldSize);
}
public ByteSizeValue getFieldSize() {
return this.fieldSize();
}
public long filterSizeInBytes() {
return this.filterSize;
}
@ -145,9 +115,6 @@ public class CacheStats implements Streamable, ToXContent {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.CACHE);
builder.field(Fields.FIELD_EVICTIONS, fieldEvictions);
builder.field(Fields.FIELD_SIZE, fieldSize().toString());
builder.field(Fields.FIELD_SIZE_IN_BYTES, fieldSize);
builder.field(Fields.FILTER_COUNT, filterCount);
builder.field(Fields.FILTER_EVICTIONS, filterEvictions);
builder.field(Fields.FILTER_SIZE, filterSize().toString());
@ -160,9 +127,6 @@ public class CacheStats implements Streamable, ToXContent {
static final class Fields {
static final XContentBuilderString CACHE = new XContentBuilderString("cache");
static final XContentBuilderString FIELD_SIZE = new XContentBuilderString("field_size");
static final XContentBuilderString FIELD_SIZE_IN_BYTES = new XContentBuilderString("field_size_in_bytes");
static final XContentBuilderString FIELD_EVICTIONS = new XContentBuilderString("field_evictions");
static final XContentBuilderString FILTER_EVICTIONS = new XContentBuilderString("filter_evictions");
static final XContentBuilderString FILTER_COUNT = new XContentBuilderString("filter_count");
static final XContentBuilderString FILTER_SIZE = new XContentBuilderString("filter_size");
@ -179,9 +143,7 @@ public class CacheStats implements Streamable, ToXContent {
@Override
public void readFrom(StreamInput in) throws IOException {
fieldEvictions = in.readVLong();
filterEvictions = in.readVLong();
fieldSize = in.readVLong();
filterSize = in.readVLong();
filterCount = in.readVLong();
idCacheSize = in.readVLong();
@ -189,9 +151,7 @@ public class CacheStats implements Streamable, ToXContent {
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(fieldEvictions);
out.writeVLong(filterEvictions);
out.writeVLong(fieldSize);
out.writeVLong(filterSize);
out.writeVLong(filterCount);
out.writeVLong(idCacheSize);

View File

@ -31,7 +31,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.cache.filter.FilterCache;
import org.elasticsearch.index.cache.id.IdCache;
import org.elasticsearch.index.cache.query.parser.QueryParserCache;
@ -44,8 +43,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
private final FilterCache filterCache;
private final FieldDataCache fieldDataCache;
private final QueryParserCache queryParserCache;
private final IdCache idCache;
@ -58,11 +55,9 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
private CacheStats latestCacheStats;
@Inject
public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, FieldDataCache fieldDataCache,
QueryParserCache queryParserCache, IdCache idCache) {
public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, QueryParserCache queryParserCache, IdCache idCache) {
super(index, indexSettings);
this.filterCache = filterCache;
this.fieldDataCache = fieldDataCache;
this.queryParserCache = queryParserCache;
this.idCache = idCache;
@ -81,7 +76,7 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
public synchronized void invalidateCache() {
FilterCache.EntriesStats filterEntriesStats = filterCache.entriesStats();
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
latestCacheStats = new CacheStats(filterCache.evictions(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
latestCacheStatsTimestamp = System.currentTimeMillis();
}
@ -89,7 +84,7 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
long timestamp = System.currentTimeMillis();
if ((timestamp - latestCacheStatsTimestamp) > refreshInterval.millis()) {
FilterCache.EntriesStats filterEntriesStats = filterCache.entriesStats();
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
latestCacheStats = new CacheStats(filterCache.evictions(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
latestCacheStatsTimestamp = timestamp;
}
return latestCacheStats;
@ -99,10 +94,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
return filterCache;
}
public FieldDataCache fieldData() {
return fieldDataCache;
}
public IdCache idCache() {
return this.idCache;
}
@ -114,7 +105,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
@Override
public void close() throws ElasticSearchException {
filterCache.close();
fieldDataCache.close();
idCache.close();
queryParserCache.close();
if (clusterService != null) {
@ -124,13 +114,11 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
public void clear(IndexReader reader) {
filterCache.clear(reader);
fieldDataCache.clear(reader);
idCache.clear(reader);
}
public void clear(String reason) {
filterCache.clear(reason);
fieldDataCache.clear(reason);
idCache.clear();
queryParserCache.clear();
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.cache;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.field.data.FieldDataCacheModule;
import org.elasticsearch.index.cache.filter.FilterCacheModule;
import org.elasticsearch.index.cache.id.IdCacheModule;
import org.elasticsearch.index.cache.query.parser.QueryParserCacheModule;
@ -40,7 +39,6 @@ public class IndexCacheModule extends AbstractModule {
@Override
protected void configure() {
new FilterCacheModule(settings).configure(binder());
new FieldDataCacheModule(settings).configure(binder());
new IdCacheModule(settings).configure(binder());
new QueryParserCacheModule(settings).configure(binder());

View File

@ -1,51 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.cache.field.data;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.elasticsearch.common.component.CloseableComponent;
import org.elasticsearch.index.IndexComponent;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public interface FieldDataCache extends IndexComponent, CloseableComponent {
FieldData cache(FieldDataType type, AtomicReader reader, String fieldName) throws IOException;
String type();
void clear(String reason, String fieldName);
void clear(String reason);
void clear(IndexReader reader);
long evictions();
long sizeInBytes();
long sizeInBytes(String fieldName);
}

View File

@ -1,48 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.cache.field.data;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.field.data.resident.ResidentFieldDataCache;
/**
*
*/
public class FieldDataCacheModule extends AbstractModule {
public static final class FieldDataCacheSettings {
public static final String FIELD_DATA_CACHE_TYPE = "index.cache.field.type";
}
private final Settings settings;
public FieldDataCacheModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
bind(FieldDataCache.class)
.to(settings.getAsClass(FieldDataCacheSettings.FIELD_DATA_CACHE_TYPE, ResidentFieldDataCache.class, "org.elasticsearch.index.cache.field.data.", "FieldDataCache"))
.in(Scopes.SINGLETON);
}
}

View File

@ -1,88 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.cache.field.data.none;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.settings.IndexSettings;
import java.io.IOException;
/**
*
*/
public class NoneFieldDataCache extends AbstractIndexComponent implements FieldDataCache {
@Inject
public NoneFieldDataCache(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
logger.debug("Using no field cache");
}
@Override
public FieldData cache(FieldDataType type, AtomicReader reader, String fieldName) throws IOException {
return FieldData.load(type, reader, fieldName);
}
@Override
public String type() {
return "none";
}
@Override
public void clear(String reason, String fieldName) {
}
@Override
public void clear(String reason) {
}
@Override
public void clear(IndexReader reader) {
}
@Override
public void close() throws ElasticSearchException {
}
@Override
public long sizeInBytes() {
return 0;
}
@Override
public long sizeInBytes(String fieldName) {
return 0;
}
@Override
public long evictions() {
return 0;
}
}

View File

@ -1,130 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.cache.field.data.resident;
import com.google.common.base.Objects;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.support.AbstractConcurrentMapFieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import java.util.concurrent.TimeUnit;
/**
*
*/
public class ResidentFieldDataCache extends AbstractConcurrentMapFieldDataCache implements RemovalListener<String, FieldData> {
private final IndexSettingsService indexSettingsService;
private volatile int maxSize;
private volatile TimeValue expire;
private final CounterMetric evictions = new CounterMetric();
private final ApplySettings applySettings = new ApplySettings();
@Inject
public ResidentFieldDataCache(Index index, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService) {
super(index, indexSettings);
this.indexSettingsService = indexSettingsService;
this.maxSize = indexSettings.getAsInt("index.cache.field.max_size", componentSettings.getAsInt("max_size", -1));
this.expire = indexSettings.getAsTime("index.cache.field.expire", componentSettings.getAsTime("expire", null));
logger.debug("using [resident] field cache with max_size [{}], expire [{}]", maxSize, expire);
indexSettingsService.addListener(applySettings);
}
@Override
public void close() throws ElasticSearchException {
indexSettingsService.removeListener(applySettings);
super.close();
}
@Override
protected Cache<String, FieldData> buildFieldDataMap() {
CacheBuilder<String, FieldData> cacheBuilder = CacheBuilder.newBuilder().removalListener(this);
if (maxSize != -1) {
cacheBuilder.maximumSize(maxSize);
}
if (expire != null) {
cacheBuilder.expireAfterAccess(expire.nanos(), TimeUnit.NANOSECONDS);
}
return cacheBuilder.build();
}
@Override
public String type() {
return "resident";
}
@Override
public long evictions() {
return evictions.count();
}
@Override
public void onRemoval(RemovalNotification<String, FieldData> removalNotification) {
if (removalNotification.wasEvicted()) {
evictions.inc();
}
}
static {
IndexMetaData.addDynamicSettings(
"index.cache.field.max_size",
"index.cache.field.expire"
);
}
class ApplySettings implements IndexSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int maxSize = settings.getAsInt("index.cache.field.max_size", ResidentFieldDataCache.this.maxSize);
TimeValue expire = settings.getAsTime("index.cache.field.expire", ResidentFieldDataCache.this.expire);
boolean changed = false;
if (maxSize != ResidentFieldDataCache.this.maxSize) {
logger.info("updating index.cache.field.max_size from [{}] to [{}]", ResidentFieldDataCache.this.maxSize, maxSize);
changed = true;
ResidentFieldDataCache.this.maxSize = maxSize;
}
if (!Objects.equal(expire, ResidentFieldDataCache.this.expire)) {
logger.info("updating index.cache.field.expire from [{}] to [{}]", ResidentFieldDataCache.this.expire, expire);
changed = true;
ResidentFieldDataCache.this.expire = expire;
}
if (changed) {
clear("update_settings");
}
}
}
}

View File

@ -1,68 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.cache.field.data.soft;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.support.AbstractConcurrentMapFieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.settings.IndexSettings;
/**
*
*/
public class SoftFieldDataCache extends AbstractConcurrentMapFieldDataCache implements RemovalListener<String, FieldData> {
private final CounterMetric evictions = new CounterMetric();
@Inject
public SoftFieldDataCache(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
}
@Override
protected Cache<String, FieldData> buildFieldDataMap() {
CacheBuilder<String, FieldData> cacheBuilder = CacheBuilder.newBuilder().softValues().removalListener(this);
return cacheBuilder.build();
}
@Override
public long evictions() {
return evictions.count();
}
@Override
public String type() {
return "soft";
}
@Override
public void onRemoval(RemovalNotification<String, FieldData> removalNotification) {
if (removalNotification.wasEvicted()) {
evictions.inc();
}
}
}

View File

@ -1,150 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.cache.field.data.support;
import com.google.common.cache.Cache;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.SegmentReader;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.settings.IndexSettings;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
/**
*
*/
public abstract class AbstractConcurrentMapFieldDataCache extends AbstractIndexComponent implements FieldDataCache, SegmentReader.CoreClosedListener {
private final ConcurrentMap<Object, Cache<String, FieldData>> cache;
private final Object creationMutex = new Object();
protected AbstractConcurrentMapFieldDataCache(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
this.cache = ConcurrentCollections.newConcurrentMap();
}
@Override
public void close() throws ElasticSearchException {
clear("close");
}
@Override
public void clear(String reason, String fieldName) {
logger.debug("clearing field [{}] cache, reason [{}]", fieldName, reason);
for (Map.Entry<Object, Cache<String, FieldData>> entry : cache.entrySet()) {
entry.getValue().invalidate(fieldName);
}
}
@Override
public void clear(String reason) {
logger.debug("full cache clear, reason [{}]", reason);
cache.clear();
}
@Override
public void onClose(SegmentReader owner) {
clear(owner);
}
@Override
public void clear(IndexReader reader) {
cache.remove(reader.getCoreCacheKey());
}
@Override
public long sizeInBytes() {
// the overhead of the map is not really relevant...
long sizeInBytes = 0;
for (Cache<String, FieldData> map : cache.values()) {
for (FieldData fieldData : map.asMap().values()) {
sizeInBytes += fieldData.sizeInBytes();
}
}
return sizeInBytes;
}
@Override
public long sizeInBytes(String fieldName) {
long sizeInBytes = 0;
for (Cache<String, FieldData> map : cache.values()) {
FieldData fieldData = map.getIfPresent(fieldName);
if (fieldData != null) {
sizeInBytes += fieldData.sizeInBytes();
}
}
return sizeInBytes;
}
@Override
public FieldData cache(FieldDataType type, AtomicReader reader, String fieldName) throws IOException {
Cache<String, FieldData> fieldDataCache = cache.get(reader.getCoreCacheKey());
if (fieldDataCache == null) {
synchronized (creationMutex) {
fieldDataCache = cache.get(reader.getCoreCacheKey());
if (fieldDataCache == null) {
fieldDataCache = buildFieldDataMap();
if (reader instanceof SegmentReader) {
((SegmentReader) reader).addCoreClosedListener(this);
}
cache.put(reader.getCoreCacheKey(), fieldDataCache);
}
}
}
FieldData fieldData = fieldDataCache.getIfPresent(fieldName);
if (fieldData == null) {
synchronized (fieldDataCache) {
fieldData = fieldDataCache.getIfPresent(fieldName);
if (fieldData == null) {
try {
long time = System.nanoTime();
fieldData = FieldData.load(type, reader, fieldName);
fieldDataCache.put(fieldName, fieldData);
long took = System.nanoTime() - time;
if (logger.isTraceEnabled()) {
logger.trace("loaded field [{}] for reader [{}], took [{}], took_millis [{}]", fieldName, reader, TimeValue.timeValueNanos(took), TimeUnit.NANOSECONDS.toMillis(took));
}
} catch (OutOfMemoryError e) {
logger.warn("loading field [" + fieldName + "] caused out of memory failure", e);
final OutOfMemoryError outOfMemoryError = new OutOfMemoryError("loading field [" + fieldName + "] caused out of memory failure");
outOfMemoryError.initCause(e);
throw outOfMemoryError;
}
}
}
}
return fieldData;
}
protected abstract Cache<String, FieldData> buildFieldDataMap();
}

View File

@ -1,114 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import java.io.IOException;
/**
*
*/
// General TODOs on FieldData
// TODO Optimize the order (both int[] and int[][] when they are sparse, create an Order abstraction)
public abstract class FieldData<Doc extends DocFieldData> {
private final ThreadLocal<ThreadLocals.CleanableValue<Doc>> cachedDocFieldData = new ThreadLocal<ThreadLocals.CleanableValue<Doc>>() {
@Override
protected ThreadLocals.CleanableValue<Doc> initialValue() {
return new ThreadLocals.CleanableValue<Doc>(createFieldData());
}
};
private final String fieldName;
private long sizeInBytes = -1;
protected FieldData(String fieldName) {
this.fieldName = fieldName;
}
/**
* The field name of this field data.
*/
public final String fieldName() {
return fieldName;
}
public Doc docFieldData(int docId) {
Doc docFieldData = cachedDocFieldData.get().get();
docFieldData.setDocId(docId);
return docFieldData;
}
public long sizeInBytes() {
if (sizeInBytes == -1) {
sizeInBytes = computeSizeInBytes();
}
return sizeInBytes;
}
protected abstract long computeSizeInBytes();
protected abstract Doc createFieldData();
/**
* Is the field data a multi valued one (has multiple values / terms per document id) or not.
*/
public abstract boolean multiValued();
/**
* Is there a value associated with this document id.
*/
public abstract boolean hasValue(int docId);
public abstract BytesRef stringValue(int docId);
public abstract void forEachValue(StringValueProc proc);
public static interface StringValueProc {
void onValue(BytesRef value);
}
public abstract void forEachValueInDoc(int docId, StringValueInDocProc proc);
public static interface StringValueInDocProc {
void onValue(int docId, BytesRef value);
void onMissing(int docId);
}
public abstract void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc);
public static interface OrdinalInDocProc {
void onOrdinal(int docId, int ordinal);
}
/**
* The type of this field data.
*/
public abstract FieldDataType type();
public static FieldData load(FieldDataType type, AtomicReader reader, String fieldName) throws IOException {
return type.load(reader, fieldName);
}
}

View File

@ -1,63 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparatorSource;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.bytes.ByteFieldDataType;
import org.elasticsearch.index.field.data.doubles.DoubleFieldDataType;
import org.elasticsearch.index.field.data.floats.FloatFieldDataType;
import org.elasticsearch.index.field.data.ints.IntFieldDataType;
import org.elasticsearch.index.field.data.longs.LongFieldDataType;
import org.elasticsearch.index.field.data.shorts.ShortFieldDataType;
import org.elasticsearch.index.field.data.strings.StringFieldDataType;
import java.io.IOException;
/**
*
*/
public interface FieldDataType<T extends FieldData> {
public static final class DefaultTypes {
public static final StringFieldDataType STRING = new StringFieldDataType();
public static final ByteFieldDataType BYTE = new ByteFieldDataType();
public static final ShortFieldDataType SHORT = new ShortFieldDataType();
public static final IntFieldDataType INT = new IntFieldDataType();
public static final LongFieldDataType LONG = new LongFieldDataType();
public static final FloatFieldDataType FLOAT = new FloatFieldDataType();
public static final DoubleFieldDataType DOUBLE = new DoubleFieldDataType();
}
ExtendedFieldComparatorSource newFieldComparatorSource(FieldDataCache cache, @Nullable String missing);
T load(AtomicReader reader, String fieldName) throws IOException;
// we need this extended source we we have custom comparators to reuse our field data
// in this case, we need to reduce type that will be used when search results are reduced
// on another node (we don't have the custom source them...)
public abstract class ExtendedFieldComparatorSource extends FieldComparatorSource {
public abstract SortField.Type reducedType();
}
}

View File

@ -1,58 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data;
/**
*
*/
public class NumericDocFieldData<T extends NumericFieldData> extends DocFieldData<T> {
public NumericDocFieldData(T fieldData) {
super(fieldData);
}
public int getIntValue() {
return fieldData.intValue(docId);
}
public long getLongValue() {
return fieldData.longValue(docId);
}
public float getFloatValue() {
return fieldData.floatValue(docId);
}
public double getDoubleValue() {
return fieldData.doubleValue(docId);
}
public short getShortValue() {
return fieldData.shortValue(docId);
}
public byte getByteValue() {
return fieldData.byteValue(docId);
}
public double[] getDoubleValues() {
return fieldData.doubleValues(docId);
}
}

View File

@ -1,123 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data;
/**
*
*/
public abstract class NumericFieldData<Doc extends NumericDocFieldData> extends FieldData<Doc> {
protected NumericFieldData(String fieldName) {
super(fieldName);
}
/**
* Returns the value of the specified number as an <code>int</code>.
* This may involve rounding or truncation.
*
* @return the numeric value represented by this object after conversion
* to type <code>int</code>.
*/
public abstract int intValue(int docId);
/**
* Returns the value of the specified number as a <code>long</code>.
* This may involve rounding or truncation.
*
* @return the numeric value represented by this object after conversion
* to type <code>long</code>.
*/
public abstract long longValue(int docId);
/**
* Returns the value of the specified number as a <code>float</code>.
* This may involve rounding.
*
* @return the numeric value represented by this object after conversion
* to type <code>float</code>.
*/
public abstract float floatValue(int docId);
/**
* Returns the value of the specified number as a <code>double</code>.
* This may involve rounding.
*
* @return the numeric value represented by this object after conversion
* to type <code>double</code>.
*/
public abstract double doubleValue(int docId);
/**
* Returns the value of the specified number as a <code>byte</code>.
* This may involve rounding or truncation.
*
* @return the numeric value represented by this object after conversion
* to type <code>byte</code>.
*/
public byte byteValue(int docId) {
return (byte) intValue(docId);
}
/**
* Returns the value of the specified number as a <code>short</code>.
* This may involve rounding or truncation.
*
* @return the numeric value represented by this object after conversion
* to type <code>short</code>.
*/
public short shortValue(int docId) {
return (short) intValue(docId);
}
@Override
public Doc docFieldData(int docId) {
return super.docFieldData(docId);
}
public abstract double[] doubleValues(int docId);
public abstract void forEachValueInDoc(int docId, DoubleValueInDocProc proc);
public abstract void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc);
public abstract void forEachValueInDoc(int docId, LongValueInDocProc proc);
public abstract void forEachValueInDoc(int docId, MissingLongValueInDocProc proc);
public static interface DoubleValueInDocProc {
void onValue(int docId, double value);
}
public static interface LongValueInDocProc {
void onValue(int docId, long value);
}
public static interface MissingLongValueInDocProc {
void onValue(int docId, long value);
void onMissing(int docId);
}
public static interface MissingDoubleValueInDocProc {
void onValue(int docId, double value);
void onMissing(int docId);
}
}

View File

@ -1,165 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.field.data.bytes;
import gnu.trove.list.array.TByteArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
/**
*
*/
public abstract class ByteFieldData extends NumericFieldData<ByteDocFieldData> {
static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
protected final byte[] values;
protected ByteFieldData(String fieldName, byte[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return 1 * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final byte[] values() {
return this.values;
}
abstract public byte value(int docId);
abstract public byte[] values(int docId);
@Override
public ByteDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected ByteDocFieldData createFieldData() {
return new ByteDocFieldData(this);
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Byte.toString(values[i])));
}
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Byte.toString(value(docId)));
}
@Override
public byte byteValue(int docId) {
return value(docId);
}
@Override
public short shortValue(int docId) {
return value(docId);
}
@Override
public int intValue(int docId) {
return (int) value(docId);
}
@Override
public long longValue(int docId) {
return (long) value(docId);
}
@Override
public float floatValue(int docId) {
return (float) value(docId);
}
@Override
public double doubleValue(int docId) {
return (double) value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.BYTE;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(byte value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, byte value);
void onMissing(int docID);
}
public static ByteFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new ByteTypeLoader());
}
static class ByteTypeLoader extends FieldDataLoader.FreqsTypeLoader<ByteFieldData> {
private final TByteArrayList terms = new TByteArrayList();
ByteTypeLoader() {
super();
// the first one indicates null value
terms.add((byte) 0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add((byte) FieldCache.NUMERIC_UTILS_INT_PARSER.parseInt(term));
}
@Override
public ByteFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueByteFieldData(field, ordinals, terms.toArray());
}
@Override
public ByteFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueByteFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,77 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.field.data.bytes;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Short
public class ByteFieldDataComparator extends NumericFieldDataComparator<Byte> {
private final byte[] values;
private short bottom;
public ByteFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new byte[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.BYTE;
}
@Override
public int compare(int slot1, int slot2) {
return values[slot1] - values[slot2];
}
@Override
public int compareBottom(int doc) {
return bottom - currentFieldData.byteValue(doc);
}
@Override
public int compareDocToValue(int doc, Byte val2) throws IOException {
byte val1 = currentFieldData.byteValue(doc);
return val1 - val2;
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.byteValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Byte value(int slot) {
return values[slot];
}
}

View File

@ -1,90 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.bytes;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Short
public class ByteFieldDataMissingComparator extends NumericFieldDataComparator<Byte> {
private final byte[] values;
private short bottom;
private final byte missingValue;
public ByteFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, byte missingValue) {
super(fieldName, fieldDataCache);
values = new byte[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.BYTE;
}
@Override
public int compare(int slot1, int slot2) {
return values[slot1] - values[slot2];
}
@Override
public int compareBottom(int doc) {
byte value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.byteValue(doc);
}
return bottom - value;
}
@Override
public int compareDocToValue(int doc, Byte val2) throws IOException {
byte val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.byteValue(doc);
}
return val1 - val2;
}
@Override
public void copy(int slot, int doc) {
byte value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.byteValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Byte value(int slot) {
return values[slot];
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.field.data.bytes;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class ByteFieldDataType implements FieldDataType<ByteFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ByteFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.BYTE;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ByteFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Byte.MIN_VALUE : Byte.MAX_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.BYTE;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ByteFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Byte.MAX_VALUE : Byte.MIN_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.BYTE;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ByteFieldDataMissingComparator(numHits, fieldname, cache, Byte.parseByte(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.BYTE;
}
};
}
@Override
public ByteFieldData load(AtomicReader reader, String fieldName) throws IOException {
return ByteFieldData.load(reader, fieldName);
}
}

View File

@ -1,239 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.field.data.bytes;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class MultiValueByteFieldData extends ByteFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<byte[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<byte[][]>>() {
@Override
protected ThreadLocals.CleanableValue<byte[][]> initialValue() {
byte[][] value = new byte[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new byte[i];
}
return new ThreadLocals.CleanableValue<byte[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueByteFieldData(String fieldName, int[][] ordinals, byte[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, new BytesRef(Byte.toString(values[loc])));
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public double[] doubleValues(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = doublesValuesCache.get().get()[length];
} else {
doubles = new double[length];
}
for (int i = 0; i < length; i++) {
doubles[i] = values[ordinals[i][docId]];
}
return doubles;
}
@Override
public byte value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public byte[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_BYTE_ARRAY;
}
byte[] bytes;
if (length < VALUE_CACHE_SIZE) {
bytes = valuesCache.get().get()[length];
} else {
bytes = new byte[length];
}
for (int i = 0; i < length; i++) {
bytes[i] = values[ordinals[i][docId]];
}
return bytes;
}
}

View File

@ -1,159 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.index.field.data.bytes;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class SingleValueByteFieldData extends ByteFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<byte[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<byte[]>>() {
@Override
protected ThreadLocals.CleanableValue<byte[]> initialValue() {
return new ThreadLocals.CleanableValue<byte[]>(new byte[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueByteFieldData(String fieldName, int[] ordinals, byte[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Byte.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public byte value(int docId) {
return values[ordinals[docId]];
}
@Override
public double[] doubleValues(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] ret = doublesValuesCache.get().get();
ret[0] = values[loc];
return ret;
}
@Override
public byte[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_BYTE_ARRAY;
}
byte[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,165 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import gnu.trove.list.array.TDoubleArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
/**
*
*/
public abstract class DoubleFieldData extends NumericFieldData<DoubleDocFieldData> {
public static final double[] EMPTY_DOUBLE_ARRAY = new double[0];
protected final double[] values;
protected DoubleFieldData(String fieldName, double[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return RamUsage.NUM_BYTES_DOUBLE * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final double[] values() {
return this.values;
}
abstract public double value(int docId);
abstract public double[] values(int docId);
@Override
public DoubleDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected DoubleDocFieldData createFieldData() {
return new DoubleDocFieldData(this);
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Double.toString(value(docId)));
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Double.toString(values[i])));
}
}
@Override
public byte byteValue(int docId) {
return (byte) value(docId);
}
@Override
public short shortValue(int docId) {
return (short) value(docId);
}
@Override
public int intValue(int docId) {
return (int) value(docId);
}
@Override
public long longValue(int docId) {
return (long) value(docId);
}
@Override
public float floatValue(int docId) {
return (float) value(docId);
}
@Override
public double doubleValue(int docId) {
return value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.DOUBLE;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(double value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, double value);
void onMissing(int docId);
}
public static DoubleFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new DoubleTypeLoader());
}
static class DoubleTypeLoader extends FieldDataLoader.FreqsTypeLoader<DoubleFieldData> {
private final TDoubleArrayList terms = new TDoubleArrayList();
DoubleTypeLoader() {
super();
// the first one indicates null value
terms.add(0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add(FieldCache.NUMERIC_UTILS_DOUBLE_PARSER.parseDouble(term));
}
@Override
public DoubleFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueDoubleFieldData(field, ordinals, terms.toArray());
}
@Override
public DoubleFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueDoubleFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,92 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Double
public class DoubleFieldDataComparator extends NumericFieldDataComparator<Double> {
private final double[] values;
private double bottom;
public DoubleFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new double[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.DOUBLE;
}
@Override
public int compare(int slot1, int slot2) {
final double v1 = values[slot1];
final double v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
final double v2 = currentFieldData.doubleValue(doc);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Double val2) throws IOException {
double val1 = currentFieldData.doubleValue(doc);
return Double.compare(val1, val2);
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.doubleValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Double value(int slot) {
return values[slot];
}
}

View File

@ -1,104 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Double
public class DoubleFieldDataMissingComparator extends NumericFieldDataComparator<Double> {
private final double[] values;
private double bottom;
private final double missingValue;
public DoubleFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, double missingValue) {
super(fieldName, fieldDataCache);
values = new double[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.DOUBLE;
}
@Override
public int compare(int slot1, int slot2) {
final double v1 = values[slot1];
final double v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
double v2 = missingValue;
if (currentFieldData.hasValue(doc)) {
v2 = currentFieldData.doubleValue(doc);
}
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Double val2) throws IOException {
double val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.doubleValue(doc);
}
return Double.compare(val1, val2);
}
@Override
public void copy(int slot, int doc) {
double value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.doubleValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Double value(int slot) {
return values[slot];
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class DoubleFieldDataType implements FieldDataType<DoubleFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new DoubleFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new DoubleFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new DoubleFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new DoubleFieldDataMissingComparator(numHits, fieldname, cache, Double.parseDouble(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
};
}
@Override
public DoubleFieldData load(AtomicReader reader, String fieldName) throws IOException {
return DoubleFieldData.load(reader, fieldName);
}
}

View File

@ -1,208 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
/**
*
*/
public class MultiValueDoubleFieldData extends DoubleFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueDoubleFieldData(String fieldName, int[][] ordinals, double[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, new BytesRef(Double.toString(values[loc])));
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, (long) values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, (long) values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public double[] doubleValues(int docId) {
return values(docId);
}
@Override
public double value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public double[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = valuesCache.get().get()[length];
} else {
doubles = new double[length];
}
for (int i = 0; i < length; i++) {
doubles[i] = values[ordinals[i][docId]];
}
return doubles;
}
}

View File

@ -1,145 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.doubles;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
/**
*
*/
public class SingleValueDoubleFieldData extends DoubleFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueDoubleFieldData(String fieldName, int[] ordinals, double[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Double.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, (long) values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, (long) values[loc]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public double[] doubleValues(int docId) {
return values(docId);
}
@Override
public double value(int docId) {
return values[ordinals[docId]];
}
@Override
public double[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_DOUBLE_ARRAY;
}
double[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,165 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import gnu.trove.list.array.TFloatArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
/**
*
*/
public abstract class FloatFieldData extends NumericFieldData<FloatDocFieldData> {
static final float[] EMPTY_FLOAT_ARRAY = new float[0];
protected final float[] values;
protected FloatFieldData(String fieldName, float[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return RamUsage.NUM_BYTES_FLOAT * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final float[] values() {
return this.values;
}
abstract public float value(int docId);
abstract public float[] values(int docId);
@Override
public FloatDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected FloatDocFieldData createFieldData() {
return new FloatDocFieldData(this);
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Float.toString(value(docId)));
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Float.toString(values[i])));
}
}
@Override
public byte byteValue(int docId) {
return (byte) value(docId);
}
@Override
public short shortValue(int docId) {
return (short) value(docId);
}
@Override
public int intValue(int docId) {
return (int) value(docId);
}
@Override
public long longValue(int docId) {
return (long) value(docId);
}
@Override
public float floatValue(int docId) {
return value(docId);
}
@Override
public double doubleValue(int docId) {
return (double) value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.FLOAT;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(float value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, float value);
void onMissing(int docId);
}
public static FloatFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new FloatTypeLoader());
}
static class FloatTypeLoader extends FieldDataLoader.FreqsTypeLoader<FloatFieldData> {
private final TFloatArrayList terms = new TFloatArrayList();
FloatTypeLoader() {
super();
// the first one indicates null value
terms.add(0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add(FieldCache.NUMERIC_UTILS_FLOAT_PARSER.parseFloat(term));
}
@Override
public FloatFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueFloatFieldData(field, ordinals, terms.toArray());
}
@Override
public FloatFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueFloatFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,96 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Float
public class FloatFieldDataComparator extends NumericFieldDataComparator<Float> {
private final float[] values;
private float bottom;
public FloatFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new float[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.FLOAT;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: are there sneaky non-branch ways to compute
// sign of float?
final float v1 = values[slot1];
final float v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: are there sneaky non-branch ways to compute
// sign of float?
final float v2 = currentFieldData.floatValue(doc);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Float val2) throws IOException {
float val1 = currentFieldData.floatValue(doc);
return Float.compare(val1, val2);
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.floatValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Float value(int slot) {
return values[slot];
}
}

View File

@ -1,108 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Float
public class FloatFieldDataMissingComparator extends NumericFieldDataComparator<Float> {
private final float[] values;
private float bottom;
private final float missingValue;
public FloatFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, float missingValue) {
super(fieldName, fieldDataCache);
values = new float[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.FLOAT;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: are there sneaky non-branch ways to compute
// sign of float?
final float v1 = values[slot1];
final float v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: are there sneaky non-branch ways to compute
// sign of float?
float v2 = missingValue;
if (currentFieldData.hasValue(doc)) {
v2 = currentFieldData.floatValue(doc);
}
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Float val2) throws IOException {
float val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.floatValue(doc);
}
return Float.compare(val1, val2);
}
@Override
public void copy(int slot, int doc) {
float value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.floatValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Float value(int slot) {
return values[slot];
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class FloatFieldDataType implements FieldDataType<FloatFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new FloatFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.FLOAT;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new FloatFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.FLOAT;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new FloatFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Float.POSITIVE_INFINITY : Float.NEGATIVE_INFINITY);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.FLOAT;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new FloatFieldDataMissingComparator(numHits, fieldname, cache, Float.parseFloat(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.FLOAT;
}
};
}
@Override
public FloatFieldData load(AtomicReader reader, String fieldName) throws IOException {
return FloatFieldData.load(reader, fieldName);
}
}

View File

@ -1,239 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class MultiValueFloatFieldData extends FloatFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<float[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<float[][]>>() {
@Override
protected ThreadLocals.CleanableValue<float[][]> initialValue() {
float[][] value = new float[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new float[i];
}
return new ThreadLocals.CleanableValue<float[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueFloatFieldData(String fieldName, int[][] ordinals, float[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, new BytesRef(Double.toString(values[loc])));
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, (long) values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, (long) values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public double[] doubleValues(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = doublesValuesCache.get().get()[length];
} else {
doubles = new double[length];
}
for (int i = 0; i < length; i++) {
doubles[i] = values[ordinals[i][docId]];
}
return doubles;
}
@Override
public float value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public float[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_FLOAT_ARRAY;
}
float[] floats;
if (length < VALUE_CACHE_SIZE) {
floats = valuesCache.get().get()[length];
} else {
floats = new float[length];
}
for (int i = 0; i < length; i++) {
floats[i] = values[ordinals[i][docId]];
}
return floats;
}
}

View File

@ -1,159 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.floats;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class SingleValueFloatFieldData extends FloatFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<float[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<float[]>>() {
@Override
protected ThreadLocals.CleanableValue<float[]> initialValue() {
return new ThreadLocals.CleanableValue<float[]>(new float[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueFloatFieldData(String fieldName, int[] ordinals, float[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Float.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, (long) values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, (long) values[loc]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public double[] doubleValues(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] ret = doublesValuesCache.get().get();
ret[0] = values[loc];
return ret;
}
@Override
public float value(int docId) {
return values[ordinals[docId]];
}
@Override
public float[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_FLOAT_ARRAY;
}
float[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,165 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import gnu.trove.list.array.TIntArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
/**
*
*/
public abstract class IntFieldData extends NumericFieldData<IntDocFieldData> {
static final int[] EMPTY_INT_ARRAY = new int[0];
protected final int[] values;
protected IntFieldData(String fieldName, int[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return RamUsage.NUM_BYTES_INT * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final int[] values() {
return this.values;
}
abstract public int value(int docId);
abstract public int[] values(int docId);
@Override
public IntDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected IntDocFieldData createFieldData() {
return new IntDocFieldData(this);
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Integer.toString(value(docId)));
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Integer.toString(values[i])));
}
}
@Override
public byte byteValue(int docId) {
return (byte) value(docId);
}
@Override
public short shortValue(int docId) {
return (short) value(docId);
}
@Override
public int intValue(int docId) {
return value(docId);
}
@Override
public long longValue(int docId) {
return (long) value(docId);
}
@Override
public float floatValue(int docId) {
return (float) value(docId);
}
@Override
public double doubleValue(int docId) {
return (double) value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.INT;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(int value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, int value);
void onMissing(int docId);
}
public static IntFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new IntTypeLoader());
}
static class IntTypeLoader extends FieldDataLoader.FreqsTypeLoader<IntFieldData> {
private final TIntArrayList terms = new TIntArrayList();
IntTypeLoader() {
super();
// the first one indicates null value
terms.add(0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add(FieldCache.NUMERIC_UTILS_INT_PARSER.parseInt(term));
}
@Override
public IntFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueIntFieldData(field, ordinals, terms.toArray());
}
@Override
public IntFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueIntFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,102 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Int
public class IntFieldDataComparator extends NumericFieldDataComparator<Integer> {
private final int[] values;
private int bottom; // Value of bottom of queue
public IntFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new int[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.INT;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// Cannot return values[slot1] - values[slot2] because that
// may overflow
final int v1 = values[slot1];
final int v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// Cannot return bottom - values[slot2] because that
// may overflow
// final int v2 = currentReaderValues[doc];
final int v2 = currentFieldData.intValue(doc);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Integer val2) throws IOException {
int val1 = currentFieldData.intValue(doc);
return val1 - val2;
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.intValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Integer value(int slot) {
return values[slot];
}
}

View File

@ -1,114 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Int
public class IntFieldDataMissingComparator extends NumericFieldDataComparator<Integer> {
private final int[] values;
private int bottom; // Value of bottom of queue
private final int missingValue;
public IntFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, int missingValue) {
super(fieldName, fieldDataCache);
values = new int[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.INT;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// Cannot return values[slot1] - values[slot2] because that
// may overflow
final int v1 = values[slot1];
final int v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// Cannot return bottom - values[slot2] because that
// may overflow
// final int v2 = currentReaderValues[doc];
int v2 = missingValue;
if (currentFieldData.hasValue(doc)) {
v2 = currentFieldData.intValue(doc);
}
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Integer val2) throws IOException {
int val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.intValue(doc);
}
return val1 - val2;
}
@Override
public void copy(int slot, int doc) {
int value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.intValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Integer value(int slot) {
return values[slot];
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class IntFieldDataType implements FieldDataType<IntFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new IntFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.INT;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new IntFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Integer.MIN_VALUE : Integer.MAX_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.INT;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new IntFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Integer.MAX_VALUE : Integer.MIN_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.INT;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new IntFieldDataMissingComparator(numHits, fieldname, cache, Integer.parseInt(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.INT;
}
};
}
@Override
public IntFieldData load(AtomicReader reader, String fieldName) throws IOException {
return IntFieldData.load(reader, fieldName);
}
}

View File

@ -1,239 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class MultiValueIntFieldData extends IntFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<int[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<int[][]>>() {
@Override
protected ThreadLocals.CleanableValue<int[][]> initialValue() {
int[][] value = new int[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new int[i];
}
return new ThreadLocals.CleanableValue<int[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueIntFieldData(String fieldName, int[][] ordinals, int[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, new BytesRef(Integer.toString(values[loc])));
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public double[] doubleValues(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = doublesValuesCache.get().get()[length];
} else {
doubles = new double[length];
}
for (int i = 0; i < length; i++) {
doubles[i] = values[ordinals[i][docId]];
}
return doubles;
}
@Override
public int value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public int[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_INT_ARRAY;
}
int[] ints;
if (length < VALUE_CACHE_SIZE) {
ints = valuesCache.get().get()[length];
} else {
ints = new int[length];
}
for (int i = 0; i < length; i++) {
ints[i] = values[ordinals[i][docId]];
}
return ints;
}
}

View File

@ -1,159 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.ints;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class SingleValueIntFieldData extends IntFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<int[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<int[]>>() {
@Override
protected ThreadLocals.CleanableValue<int[]> initialValue() {
return new ThreadLocals.CleanableValue<int[]>(new int[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueIntFieldData(String fieldName, int[] ordinals, int[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Integer.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public double[] doubleValues(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] ret = doublesValuesCache.get().get();
ret[0] = values[loc];
return ret;
}
@Override
public int value(int docId) {
return values[ordinals[docId]];
}
@Override
public int[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_INT_ARRAY;
}
int[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,195 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import gnu.trove.list.array.TLongArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import org.joda.time.DateTimeZone;
import org.joda.time.MutableDateTime;
import java.io.IOException;
/**
*
*/
public abstract class LongFieldData extends NumericFieldData<LongDocFieldData> {
static final long[] EMPTY_LONG_ARRAY = new long[0];
static final MutableDateTime[] EMPTY_DATETIME_ARRAY = new MutableDateTime[0];
ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime>> dateTimeCache = new ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime>>() {
@Override
protected ThreadLocals.CleanableValue<MutableDateTime> initialValue() {
return new ThreadLocals.CleanableValue<MutableDateTime>(new MutableDateTime(DateTimeZone.UTC));
}
};
protected final long[] values;
protected LongFieldData(String fieldName, long[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return RamUsage.NUM_BYTES_LONG * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final long[] values() {
return this.values;
}
abstract public long value(int docId);
abstract public long[] values(int docId);
public MutableDateTime date(int docId) {
MutableDateTime dateTime = dateTimeCache.get().get();
dateTime.setMillis(value(docId));
return dateTime;
}
public void date(int docId, MutableDateTime dateTime) {
dateTime.setMillis(value(docId));
}
public abstract MutableDateTime[] dates(int docId);
@Override
public LongDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected LongDocFieldData createFieldData() {
return new LongDocFieldData(this);
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Long.toString(values[i])));
}
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Long.toString(value(docId)));
}
@Override
public byte byteValue(int docId) {
return (byte) value(docId);
}
@Override
public short shortValue(int docId) {
return (short) value(docId);
}
@Override
public int intValue(int docId) {
return (int) value(docId);
}
@Override
public long longValue(int docId) {
return value(docId);
}
@Override
public float floatValue(int docId) {
return (float) value(docId);
}
@Override
public double doubleValue(int docId) {
return (double) value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.LONG;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(long value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, long value);
void onMissing(int docId);
}
public abstract void forEachValueInDoc(int docId, DateValueInDocProc proc);
public abstract void forEachValueInDoc(int docId, MutableDateTime dateTime, DateValueInDocProc proc);
public static interface DateValueInDocProc {
void onValue(int docId, MutableDateTime dateTime);
}
public static LongFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new LongTypeLoader());
}
static class LongTypeLoader extends FieldDataLoader.FreqsTypeLoader<LongFieldData> {
private final TLongArrayList terms = new TLongArrayList();
LongTypeLoader() {
super();
// the first one indicates null value
terms.add(0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add(FieldCache.NUMERIC_UTILS_LONG_PARSER.parseLong(term));
}
@Override
public LongFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueLongFieldData(field, ordinals, terms.toArray());
}
@Override
public LongFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueLongFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,98 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Long
public class LongFieldDataComparator extends NumericFieldDataComparator<Long> {
private final long[] values;
private long bottom;
public LongFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new long[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.LONG;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
final long v1 = values[slot1];
final long v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// final long v2 = currentReaderValues[doc];
final long v2 = currentFieldData.longValue(doc);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Long val2) throws IOException {
long val1 = currentFieldData.longValue(doc);
return (int) (val1 - val2);
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.longValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Long value(int slot) {
return values[slot];
}
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR - Monitor against FieldComparator.Long
public class LongFieldDataMissingComparator extends NumericFieldDataComparator<Long> {
private final long[] values;
private long bottom;
private final long missingValue;
public LongFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, long missingValue) {
super(fieldName, fieldDataCache);
values = new long[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.LONG;
}
@Override
public int compare(int slot1, int slot2) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
final long v1 = values[slot1];
final long v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
// TODO: there are sneaky non-branch ways to compute
// -1/+1/0 sign
// final long v2 = currentReaderValues[doc];
long v2 = missingValue;
if (currentFieldData.hasValue(doc)) {
v2 = currentFieldData.longValue(doc);
}
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Long val2) throws IOException {
long val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.longValue(doc);
}
return (int) (val1 - val2);
}
@Override
public void copy(int slot, int doc) {
long value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.longValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Long value(int slot) {
return Long.valueOf(values[slot]);
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class LongFieldDataType implements FieldDataType<LongFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new LongFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.LONG;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new LongFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Long.MIN_VALUE : Long.MAX_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.LONG;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new LongFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Long.MAX_VALUE : Long.MIN_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.LONG;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new LongFieldDataMissingComparator(numHits, fieldname, cache, Long.parseLong(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.LONG;
}
};
}
@Override
public LongFieldData load(AtomicReader reader, String fieldName) throws IOException {
return LongFieldData.load(reader, fieldName);
}
}

View File

@ -1,311 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
import org.joda.time.DateTimeZone;
import org.joda.time.MutableDateTime;
/**
*
*/
public class MultiValueLongFieldData extends LongFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime[][]>> dateTimesCache = new ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime[][]>>() {
@Override
protected ThreadLocals.CleanableValue<MutableDateTime[][]> initialValue() {
MutableDateTime[][] value = new MutableDateTime[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new MutableDateTime[i];
for (int j = 0; j < i; j++) {
value[i][j] = new MutableDateTime(DateTimeZone.UTC);
}
}
return new ThreadLocals.CleanableValue<MutableDateTime[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<long[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<long[][]>>() {
@Override
protected ThreadLocals.CleanableValue<long[][]> initialValue() {
long[][] value = new long[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new long[i];
}
return new ThreadLocals.CleanableValue<long[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueLongFieldData(String fieldName, int[][] ordinals, long[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, new BytesRef(Long.toString(values[loc])));
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachValueInDoc(int docId, DateValueInDocProc proc) {
MutableDateTime dateTime = dateTimeCache.get().get();
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
dateTime.setMillis(values[loc]);
proc.onValue(docId, dateTime);
}
}
@Override
public void forEachValueInDoc(int docId, MutableDateTime dateTime, DateValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc == 0) {
break;
}
dateTime.setMillis(values[loc]);
proc.onValue(docId, dateTime);
}
}
@Override
public MutableDateTime[] dates(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
length++;
}
}
if (length == 0) {
return EMPTY_DATETIME_ARRAY;
}
MutableDateTime[] dates;
if (length < VALUE_CACHE_SIZE) {
dates = dateTimesCache.get().get()[length];
} else {
dates = new MutableDateTime[length];
for (int i = 0; i < dates.length; i++) {
dates[i] = new MutableDateTime();
}
}
int i = 0;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
dates[i++].setMillis(values[loc]);
}
}
return dates;
}
@Override
public double[] doubleValues(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = doublesValuesCache.get().get()[length];
} else {
doubles = new double[length];
}
for (int i = 0; i < length; i++) {
doubles[i] = values[ordinals[i][docId]];
}
return doubles;
}
@Override
public long value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public long[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_LONG_ARRAY;
}
long[] longs;
if (length < VALUE_CACHE_SIZE) {
longs = valuesCache.get().get()[length];
} else {
longs = new long[length];
}
for (int i = 0; i < length; i++) {
longs[i] = values[ordinals[i][docId]];
}
return longs;
}
}

View File

@ -1,202 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.longs;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
import org.joda.time.DateTimeZone;
import org.joda.time.MutableDateTime;
/**
*
*/
public class SingleValueLongFieldData extends LongFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime[]>> datesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<MutableDateTime[]>>() {
@Override
protected ThreadLocals.CleanableValue<MutableDateTime[]> initialValue() {
MutableDateTime[] date = new MutableDateTime[1];
date[0] = new MutableDateTime(DateTimeZone.UTC);
return new ThreadLocals.CleanableValue<MutableDateTime[]>(date);
}
};
private ThreadLocal<long[]> valuesCache = new ThreadLocal<long[]>() {
@Override
protected long[] initialValue() {
return new long[1];
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueLongFieldData(String fieldName, int[] ordinals, long[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Long.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, DateValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
MutableDateTime dateTime = dateTimeCache.get().get();
dateTime.setMillis(values[loc]);
proc.onValue(docId, dateTime);
}
@Override
public void forEachValueInDoc(int docId, MutableDateTime dateTime, DateValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
dateTime.setMillis(values[loc]);
proc.onValue(docId, dateTime);
}
@Override
public MutableDateTime[] dates(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_DATETIME_ARRAY;
}
MutableDateTime[] ret = datesValuesCache.get().get();
ret[0].setMillis(values[loc]);
return ret;
}
@Override
public double[] doubleValues(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] ret = doublesValuesCache.get().get();
ret[0] = values[loc];
return ret;
}
@Override
public long value(int docId) {
return values[ordinals[docId]];
}
@Override
public long[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_LONG_ARRAY;
}
long[] ret = valuesCache.get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,248 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class MultiValueShortFieldData extends ShortFieldData {
private static final int VALUE_CACHE_SIZE = 10;
private ThreadLocal<ThreadLocals.CleanableValue<double[][]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[][]>>() {
@Override
protected ThreadLocals.CleanableValue<double[][]> initialValue() {
double[][] value = new double[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new double[i];
}
return new ThreadLocals.CleanableValue<double[][]>(value);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<short[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<short[][]>>() {
@Override
protected ThreadLocals.CleanableValue<short[][]> initialValue() {
short[][] value = new short[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new short[i];
}
return new ThreadLocals.CleanableValue<short[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueShortFieldData(String fieldName, int[][] ordinals, short[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
boolean found = false;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
found = true;
proc.onValue(docId, new BytesRef(Short.toString(values[loc])));
}
}
if (!found) {
proc.onMissing(docId);
}
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
proc.onValue(docId, values[loc]);
}
}
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
proc.onValue(docId, values[loc]);
}
}
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
boolean found = false;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
found = true;
proc.onValue(docId, values[loc]);
}
}
if (!found) {
proc.onMissing(docId);
}
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
boolean found = false;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
found = true;
proc.onValue(docId, values[loc]);
}
}
if (!found) {
proc.onMissing(docId);
}
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
boolean found = false;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
found = true;
proc.onValue(docId, values[loc]);
}
}
if (!found) {
proc.onMissing(docId);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
boolean found = false;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
found = true;
proc.onOrdinal(docId, loc);
}
}
if (!found) {
proc.onOrdinal(docId, 0);
}
}
@Override
public double[] doubleValues(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
length++;
}
}
if (length == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] doubles;
if (length < VALUE_CACHE_SIZE) {
doubles = doublesValuesCache.get().get()[length];
} else {
doubles = new double[length];
}
int i = 0;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
doubles[i++] = values[loc];
}
}
return doubles;
}
@Override
public short value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return 0;
}
@Override
public short[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
length++;
}
}
if (length == 0) {
return EMPTY_SHORT_ARRAY;
}
short[] shorts;
if (length < VALUE_CACHE_SIZE) {
shorts = valuesCache.get().get()[length];
} else {
shorts = new short[length];
}
int i = 0;
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
shorts[i++] = values[loc];
}
}
return shorts;
}
}

View File

@ -1,164 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import gnu.trove.list.array.TShortArrayList;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
/**
*
*/
public abstract class ShortFieldData extends NumericFieldData<ShortDocFieldData> {
static final short[] EMPTY_SHORT_ARRAY = new short[0];
protected final short[] values;
protected ShortFieldData(String fieldName, short[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
return RamUsage.NUM_BYTES_SHORT * values.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
public final short[] values() {
return this.values;
}
abstract public short value(int docId);
abstract public short[] values(int docId);
@Override
public ShortDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
protected ShortDocFieldData createFieldData() {
return new ShortDocFieldData(this);
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(new BytesRef(Short.toString(values[i])));
}
}
@Override
public BytesRef stringValue(int docId) {
return new BytesRef(Short.toString(value(docId)));
}
@Override
public byte byteValue(int docId) {
return (byte) value(docId);
}
@Override
public short shortValue(int docId) {
return value(docId);
}
@Override
public int intValue(int docId) {
return (int) value(docId);
}
@Override
public long longValue(int docId) {
return (long) value(docId);
}
@Override
public float floatValue(int docId) {
return (float) value(docId);
}
@Override
public double doubleValue(int docId) {
return (double) value(docId);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.SHORT;
}
public void forEachValue(ValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static interface ValueProc {
void onValue(short value);
}
public abstract void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, short value);
void onMissing(int docId);
}
public static ShortFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new ShortTypeLoader());
}
static class ShortTypeLoader extends FieldDataLoader.FreqsTypeLoader<ShortFieldData> {
private final TShortArrayList terms = new TShortArrayList();
ShortTypeLoader() {
super();
// the first one indicates null value
terms.add((short) 0);
}
@Override
public void collectTerm(BytesRef term) {
terms.add((short) FieldCache.NUMERIC_UTILS_INT_PARSER.parseInt(term));
}
@Override
public ShortFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueShortFieldData(field, ordinals, terms.toArray());
}
@Override
public ShortFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueShortFieldData(field, ordinals, terms.toArray());
}
}
}

View File

@ -1,78 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Short
public class ShortFieldDataComparator extends NumericFieldDataComparator<Short> {
private final short[] values;
private short bottom;
public ShortFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
super(fieldName, fieldDataCache);
values = new short[numHits];
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.SHORT;
}
@Override
public int compare(int slot1, int slot2) {
return values[slot1] - values[slot2];
}
@Override
public int compareBottom(int doc) {
return bottom - currentFieldData.shortValue(doc);
}
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.shortValue(doc);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public int compareDocToValue(int doc, Short val2) throws IOException {
short val1 = currentFieldData.shortValue(doc);
return val1 - val2;
}
@Override
public Short value(int slot) {
return values[slot];
}
}

View File

@ -1,91 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.NumericFieldDataComparator;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Short
public class ShortFieldDataMissingComparator extends NumericFieldDataComparator<Short> {
private final short[] values;
private short bottom;
private final short missingValue;
public ShortFieldDataMissingComparator(int numHits, String fieldName, FieldDataCache fieldDataCache, short missingValue) {
super(fieldName, fieldDataCache);
values = new short[numHits];
this.missingValue = missingValue;
}
@Override
public FieldDataType fieldDataType() {
return FieldDataType.DefaultTypes.SHORT;
}
@Override
public int compare(int slot1, int slot2) {
return values[slot1] - values[slot2];
}
@Override
public int compareBottom(int doc) {
short value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.shortValue(doc);
}
return bottom - value;
}
@Override
public void copy(int slot, int doc) {
short value = missingValue;
if (currentFieldData.hasValue(doc)) {
value = currentFieldData.shortValue(doc);
}
values[slot] = value;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public int compareDocToValue(int doc, Short val2) throws IOException {
short val1 = missingValue;
if (currentFieldData.hasValue(doc)) {
val1 = currentFieldData.shortValue(doc);
}
return val1 - val2;
}
@Override
public Short value(int slot) {
return values[slot];
}
}

View File

@ -1,93 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class ShortFieldDataType implements FieldDataType<ShortFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing == null) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ShortFieldDataComparator(numHits, fieldname, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.SHORT;
}
};
}
if (missing.equals("_last")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ShortFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Short.MIN_VALUE : Short.MAX_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.SHORT;
}
};
}
if (missing.equals("_first")) {
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ShortFieldDataMissingComparator(numHits, fieldname, cache, reversed ? Short.MAX_VALUE : Short.MIN_VALUE);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.SHORT;
}
};
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new ShortFieldDataMissingComparator(numHits, fieldname, cache, Short.parseShort(missing));
}
@Override
public SortField.Type reducedType() {
return SortField.Type.SHORT;
}
};
}
@Override
public ShortFieldData load(AtomicReader reader, String fieldName) throws IOException {
return ShortFieldData.load(reader, fieldName);
}
}

View File

@ -1,159 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.shorts;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
import org.elasticsearch.index.field.data.doubles.DoubleFieldData;
/**
*
*/
public class SingleValueShortFieldData extends ShortFieldData {
private ThreadLocal<ThreadLocals.CleanableValue<double[]>> doublesValuesCache = new ThreadLocal<ThreadLocals.CleanableValue<double[]>>() {
@Override
protected ThreadLocals.CleanableValue<double[]> initialValue() {
return new ThreadLocals.CleanableValue<double[]>(new double[1]);
}
};
private ThreadLocal<ThreadLocals.CleanableValue<short[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<short[]>>() {
@Override
protected ThreadLocals.CleanableValue<short[]> initialValue() {
return new ThreadLocals.CleanableValue<short[]>(new short[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueShortFieldData(String fieldName, int[] ordinals, short[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, new BytesRef(Short.toString(values[loc])));
}
@Override
public void forEachValueInDoc(int docId, DoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, LongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingDoubleValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, MissingLongValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public short value(int docId) {
return values[ordinals[docId]];
}
@Override
public double[] doubleValues(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return DoubleFieldData.EMPTY_DOUBLE_ARRAY;
}
double[] ret = doublesValuesCache.get().get();
ret[0] = values[loc];
return ret;
}
@Override
public short[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_SHORT_ARRAY;
}
short[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,141 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
/**
*
*/
public class MultiValueStringFieldData extends StringFieldData {
private static final BytesRef[] EMPTY_ARRAY = new BytesRef[0];
private static final int VALUE_CACHE_SIZE = 100;
private static ThreadLocal<ThreadLocals.CleanableValue<BytesRef[][]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<BytesRef[][]>>() {
@Override
protected ThreadLocals.CleanableValue<BytesRef[][]> initialValue() {
BytesRef[][] value = new BytesRef[VALUE_CACHE_SIZE][];
for (int i = 0; i < value.length; i++) {
value[i] = new BytesRef[i];
}
return new ThreadLocals.CleanableValue<BytesRef[][]>(value);
}
};
// order with value 0 indicates no value
private final int[][] ordinals;
public MultiValueStringFieldData(String fieldName, int[][] ordinals, BytesRef[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
long size = super.computeSizeInBytes();
size += RamUsage.NUM_BYTES_ARRAY_HEADER; // for the top level array
for (int[] ordinal : ordinals) {
size += RamUsage.NUM_BYTES_INT * ordinal.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
return size;
}
@Override
public boolean multiValued() {
return true;
}
@Override
public boolean hasValue(int docId) {
for (int[] ordinal : ordinals) {
if (ordinal[docId] != 0) {
return true;
}
}
return false;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onMissing(docId);
}
break;
}
proc.onValue(docId, values[loc]);
}
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
for (int i = 0; i < ordinals.length; i++) {
int loc = ordinals[i][docId];
if (loc == 0) {
if (i == 0) {
proc.onOrdinal(docId, 0);
}
break;
}
proc.onOrdinal(docId, loc);
}
}
@Override
public BytesRef value(int docId) {
for (int[] ordinal : ordinals) {
int loc = ordinal[docId];
if (loc != 0) {
return values[loc];
}
}
return null;
}
@Override
public BytesRef[] values(int docId) {
int length = 0;
for (int[] ordinal : ordinals) {
if (ordinal[docId] == 0) {
break;
}
length++;
}
if (length == 0) {
return EMPTY_ARRAY;
}
BytesRef[] refs;
if (length < VALUE_CACHE_SIZE) {
refs = valuesCache.get().get()[length];
} else {
refs = new BytesRef[length];
}
for (int i = 0; i < length; i++) {
refs[i] = values[ordinals[i][docId]];
}
return refs;
}
}

View File

@ -1,98 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.common.util.concurrent.ThreadLocals;
/**
*
*/
public class SingleValueStringFieldData extends StringFieldData {
private static final BytesRef[] EMPTY_ARRAY = new BytesRef[0];
private static ThreadLocal<ThreadLocals.CleanableValue<BytesRef[]>> valuesCache = new ThreadLocal<ThreadLocals.CleanableValue<BytesRef[]>>() {
@Override
protected ThreadLocals.CleanableValue<BytesRef[]> initialValue() {
return new ThreadLocals.CleanableValue<BytesRef[]>(new BytesRef[1]);
}
};
// order with value 0 indicates no value
private final int[] ordinals;
public SingleValueStringFieldData(String fieldName, int[] ordinals, BytesRef[] values) {
super(fieldName, values);
this.ordinals = ordinals;
}
@Override
protected long computeSizeInBytes() {
return super.computeSizeInBytes() +
RamUsage.NUM_BYTES_INT * ordinals.length + RamUsage.NUM_BYTES_ARRAY_HEADER;
}
int[] ordinals() {
return ordinals;
}
@Override
public void forEachOrdinalInDoc(int docId, OrdinalInDocProc proc) {
proc.onOrdinal(docId, ordinals[docId]);
}
@Override
public boolean multiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return ordinals[docId] != 0;
}
@Override
public void forEachValueInDoc(int docId, StringValueInDocProc proc) {
int loc = ordinals[docId];
if (loc == 0) {
proc.onMissing(docId);
return;
}
proc.onValue(docId, values[loc]);
}
@Override
public BytesRef value(int docId) {
return values[ordinals[docId]];
}
@Override
public BytesRef[] values(int docId) {
int loc = ordinals[docId];
if (loc == 0) {
return EMPTY_ARRAY;
}
BytesRef[] ret = valuesCache.get().get();
ret[0] = values[loc];
return ret;
}
}

View File

@ -1,61 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.field.data.DocFieldData;
/**
*
*/
public class StringDocFieldData extends DocFieldData<StringFieldData> {
public StringDocFieldData(StringFieldData fieldData) {
super(fieldData);
}
public String getValue() {
BytesRef value = fieldData.value(docId);
if (value == null) {
return null;
}
return value.utf8ToString();
}
public String[] getValues() {
BytesRef[] values = fieldData.values(docId);
if (values == null) {
return null;
}
String[] stringValues = new String[values.length];
for (int i = 0; i < values.length; i++) {
stringValues[i] = values[i].utf8ToString();
}
return stringValues;
}
public BytesRef getBytesValue() {
return fieldData.value(docId);
}
public BytesRef[] getBytesValues() {
return fieldData.values(docId);
}
}

View File

@ -1,120 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.RamUsage;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.support.FieldDataLoader;
import java.io.IOException;
import java.util.ArrayList;
/**
*
*/
public abstract class StringFieldData extends FieldData<StringDocFieldData> {
protected final BytesRef[] values;
protected StringFieldData(String fieldName, BytesRef[] values) {
super(fieldName);
this.values = values;
}
@Override
protected long computeSizeInBytes() {
long size = RamUsage.NUM_BYTES_ARRAY_HEADER;
for (BytesRef value : values) {
if (value != null) {
size += RamUsage.NUM_BYTES_OBJECT_REF + RamUsage.NUM_BYTES_OBJECT_HEADER +
RamUsage.NUM_BYTES_ARRAY_HEADER + (value.length + (2 * RamUsage.NUM_BYTES_INT));
}
}
return size;
}
public BytesRef[] values() {
return this.values;
}
abstract public BytesRef value(int docId);
abstract public BytesRef[] values(int docId);
@Override
public StringDocFieldData docFieldData(int docId) {
return super.docFieldData(docId);
}
@Override
public BytesRef stringValue(int docId) {
return value(docId);
}
@Override
protected StringDocFieldData createFieldData() {
return new StringDocFieldData(this);
}
@Override
public FieldDataType type() {
return FieldDataType.DefaultTypes.STRING;
}
@Override
public void forEachValue(StringValueProc proc) {
for (int i = 1; i < values.length; i++) {
proc.onValue(values[i]);
}
}
public static StringFieldData load(AtomicReader reader, String field) throws IOException {
return FieldDataLoader.load(reader, field, new StringTypeLoader());
}
static class StringTypeLoader extends FieldDataLoader.FreqsTypeLoader<StringFieldData> {
private final ArrayList<BytesRef> terms = new ArrayList<BytesRef>();
StringTypeLoader() {
super();
// the first one indicates null value
terms.add(null);
}
@Override
public void collectTerm(BytesRef term) {
terms.add(term);
}
@Override
public StringFieldData buildSingleValue(String field, int[] ordinals) {
return new SingleValueStringFieldData(field, ordinals, terms.toArray(new BytesRef[terms.size()]));
}
@Override
public StringFieldData buildMultiValue(String field, int[][] ordinals) {
return new MultiValueStringFieldData(field, ordinals, terms.toArray(new BytesRef[terms.size()]));
}
}
}

View File

@ -1,58 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
public class StringFieldDataType implements FieldDataType<StringFieldData> {
@Override
public ExtendedFieldComparatorSource newFieldComparatorSource(final FieldDataCache cache, final String missing) {
if (missing != null) {
throw new ElasticSearchIllegalArgumentException("Sorting on string type field does not support missing parameter");
}
return new ExtendedFieldComparatorSource() {
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new StringOrdValFieldDataComparator(numHits, fieldname, sortPos, reversed, cache);
}
@Override
public SortField.Type reducedType() {
return SortField.Type.STRING;
}
};
}
@Override
public StringFieldData load(AtomicReader reader, String fieldName) throws IOException {
return StringFieldData.load(reader, fieldName);
}
}

View File

@ -1,224 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.strings;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import java.io.IOException;
/**
*
*/
// LUCENE MONITOR: Monitor against FieldComparator#String
public class StringOrdValFieldDataComparator extends FieldComparator<BytesRef> {
private final FieldDataCache fieldDataCache;
private final int[] ords;
private final BytesRef[] values;
private final int[] readerGen;
private int currentReaderGen = -1;
private BytesRef[] lookup;
private int[] order;
private final String field;
private int bottomSlot = -1;
private int bottomOrd;
private boolean bottomSameReader;
private BytesRef bottomValue;
public StringOrdValFieldDataComparator(int numHits, String field, int sortPos, boolean reversed, FieldDataCache fieldDataCache) {
this.fieldDataCache = fieldDataCache;
ords = new int[numHits];
values = new BytesRef[numHits];
readerGen = new int[numHits];
this.field = field;
}
@Override
public int compare(int slot1, int slot2) {
if (readerGen[slot1] == readerGen[slot2]) {
return ords[slot1] - ords[slot2];
}
final BytesRef val1 = values[slot1];
final BytesRef val2 = values[slot2];
if (val1 == null) {
if (val2 == null) {
return 0;
}
return -1;
} else if (val2 == null) {
return 1;
}
return val1.compareTo(val2);
}
@Override
public int compareBottom(int doc) {
assert bottomSlot != -1;
if (bottomSameReader) {
// ord is precisely comparable, even in the equal case
return bottomOrd - this.order[doc];
} else {
// ord is only approx comparable: if they are not
// equal, we can use that; if they are equal, we
// must fallback to compare by value
final int order = this.order[doc];
final int cmp = bottomOrd - order;
if (cmp != 0) {
return cmp;
}
final BytesRef val2 = lookup[order];
if (bottomValue == null) {
if (val2 == null) {
return 0;
}
// bottom wins
return -1;
} else if (val2 == null) {
// doc wins
return 1;
}
return bottomValue.compareTo(val2);
}
}
@Override
public void copy(int slot, int doc) {
final int ord = order[doc];
ords[slot] = ord;
assert ord >= 0;
values[slot] = lookup[ord];
readerGen[slot] = currentReaderGen;
}
@Override
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
FieldData cleanFieldData = fieldDataCache.cache(FieldDataType.DefaultTypes.STRING, context.reader(), field);
if (cleanFieldData instanceof MultiValueStringFieldData) {
throw new IOException("Can't sort on string types with more than one value per doc, or more than one token per field");
}
SingleValueStringFieldData fieldData = (SingleValueStringFieldData) cleanFieldData;
currentReaderGen++;
order = fieldData.ordinals();
lookup = fieldData.values();
assert lookup.length > 0;
if (bottomSlot != -1) {
setBottom(bottomSlot);
}
return this;
}
@Override
public int compareDocToValue(int doc, BytesRef otherVal) throws IOException {
BytesRef val = values[ords[doc]];
if (otherVal == null) {
if (val == null) {
return 0;
}
return -1;
} else if (val == null) {
return 1;
}
return val.compareTo(otherVal);
}
@Override
public void setBottom(final int bottom) {
bottomSlot = bottom;
bottomValue = values[bottomSlot];
if (currentReaderGen == readerGen[bottomSlot]) {
bottomOrd = ords[bottomSlot];
bottomSameReader = true;
} else {
if (bottomValue == null) {
ords[bottomSlot] = 0;
bottomOrd = 0;
bottomSameReader = true;
readerGen[bottomSlot] = currentReaderGen;
} else {
final int index = binarySearch(bottomValue, lookup);
if (index < 0) {
bottomOrd = -index - 2;
bottomSameReader = false;
} else {
bottomOrd = index;
// exact value match
bottomSameReader = true;
readerGen[bottomSlot] = currentReaderGen;
ords[bottomSlot] = bottomOrd;
}
}
}
}
private static int binarySearch(BytesRef value, BytesRef[] values) {
return binarySearch(value, values, 1, values.length-1);
}
private static int binarySearch(BytesRef value, BytesRef[] values, int low, int high) {
while (low <= high) {
int mid = (low + high) >>> 1;
BytesRef midVal = values[mid];
int cmp;
if (midVal != null) {
cmp = midVal.compareTo(value);
} else {
cmp = -1;
}
if (cmp < 0)
low = mid + 1;
else if (cmp > 0)
high = mid - 1;
else
return mid;
}
return -(low + 1);
}
@Override
public BytesRef value(int slot) {
return values[slot];
}
public BytesRef[] getValues() {
return values;
}
public int getBottomSlot() {
return bottomSlot;
}
public String getField() {
return field;
}
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.support;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.field.data.FieldData;
import java.io.IOException;
import java.util.ArrayList;
/**
*
*/
public class FieldDataLoader {
@SuppressWarnings({"StringEquality"})
public static <T extends FieldData> T load(AtomicReader reader, String field, TypeLoader<T> loader) throws IOException {
loader.init();
ArrayList<int[]> ordinals = new ArrayList<int[]>();
int[] idx = new int[reader.maxDoc()];
ordinals.add(new int[reader.maxDoc()]);
int t = 1; // current term number
Terms terms = reader.terms(field);
if (terms == null) {
return loader.buildSingleValue(field, ordinals.get(0)); // Return empty field data if field doesn't exists.
}
TermsEnum termsEnum = terms.iterator(null);
try {
DocsEnum docsEnum = null;
for (BytesRef term = termsEnum.next(); term != null; term = termsEnum.next()) {
loader.collectTerm(BytesRef.deepCopyOf(term));
docsEnum = termsEnum.docs(reader.getLiveDocs(), docsEnum, 0);
for (int docId = docsEnum.nextDoc(); docId != DocsEnum.NO_MORE_DOCS; docId = docsEnum.nextDoc()) {
int[] ordinal;
if (idx[docId] >= ordinals.size()) {
ordinal = new int[reader.maxDoc()];
ordinals.add(ordinal);
} else {
ordinal = ordinals.get(idx[docId]);
}
ordinal[docId] = t;
idx[docId]++;
}
t++;
}
} catch (RuntimeException e) {
if (e.getClass().getName().endsWith("StopFillCacheException")) {
// all is well, in case numeric parsers are used.
} else {
throw e;
}
}
if (ordinals.size() == 1) {
return loader.buildSingleValue(field, ordinals.get(0));
} else {
int[][] nativeOrdinals = new int[ordinals.size()][];
for (int i = 0; i < nativeOrdinals.length; i++) {
nativeOrdinals[i] = ordinals.get(i);
}
return loader.buildMultiValue(field, nativeOrdinals);
}
}
public static interface TypeLoader<T extends FieldData> {
void init();
void collectTerm(BytesRef term);
T buildSingleValue(String fieldName, int[] ordinals);
T buildMultiValue(String fieldName, int[][] ordinals);
}
public static abstract class FreqsTypeLoader<T extends FieldData> implements TypeLoader<T> {
protected FreqsTypeLoader() {
}
@Override
public void init() {
}
}
}

View File

@ -1,54 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.field.data.support;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldComparator;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.field.data.NumericFieldData;
import java.io.IOException;
/**
*
*/
public abstract class NumericFieldDataComparator<N extends Number> extends FieldComparator<N> {
private final String fieldName;
protected final FieldDataCache fieldDataCache;
protected NumericFieldData currentFieldData;
public NumericFieldDataComparator(String fieldName, FieldDataCache fieldDataCache) {
this.fieldName = fieldName;
this.fieldDataCache = fieldDataCache;
}
public abstract FieldDataType fieldDataType();
@Override
public NumericFieldDataComparator<N> setNextReader(AtomicReaderContext context) throws IOException {
currentFieldData = (NumericFieldData) fieldDataCache.cache(fieldDataType(), context.reader(), fieldName);
return this;
}
}

View File

@ -0,0 +1,39 @@
package org.elasticsearch.index.fielddata;
import org.apache.lucene.index.IndexReader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.settings.IndexSettings;
/**
*/
public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends AbstractIndexComponent implements IndexFieldData<FD> {
private final FieldMapper.Names fieldNames;
protected final FieldDataType fieldDataType;
protected final IndexFieldDataCache cache;
public AbstractIndexFieldData(Index index, @IndexSettings Settings indexSettings, FieldMapper.Names fieldNames, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(index, indexSettings);
this.fieldNames = fieldNames;
this.fieldDataType = fieldDataType;
this.cache = cache;
}
@Override
public FieldMapper.Names getFieldNames() {
return this.fieldNames;
}
@Override
public void clear() {
cache.clear(index, fieldNames.indexName());
}
@Override
public void clear(IndexReader reader) {
cache.clear(index, reader);
}
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
/**
* The thread safe {@link org.apache.lucene.index.AtomicReader} level cache of the data.
*/
public interface AtomicFieldData<Script extends ScriptDocValues> {
/**
* Does *one* of the docs contain multi values?
*/
boolean isMultiValued();
/**
* Are the values ordered? (in ascending manner).
*/
boolean isValuesOrdered();
/**
* The number of docs in this field data.
*/
int getNumDocs();
/**
* Size (in bytes) of memory used by this field data.
*/
long getMemorySizeInBytes();
/**
* Use a non thread safe (lightweight) view of the values as bytes.
*/
BytesValues getBytesValues();
/**
* Use a non thread safe (lightweight) view of the values as bytes.
*/
HashedBytesValues getHashedBytesValues();
/**
* Use a non thread safe (lightweight) view of the values as strings.
*/
StringValues getStringValues();
/**
* Returns a "scripting" based values.
*/
Script getScriptValues();
}

View File

@ -0,0 +1,28 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
/**
*/
public interface AtomicGeoPointFieldData<Script extends ScriptDocValues> extends AtomicFieldData<Script> {
GeoPointValues getGeoPointValues();
}

View File

@ -17,24 +17,22 @@
* under the License.
*/
package org.elasticsearch.index.field.data.ints;
import org.elasticsearch.index.field.data.NumericDocFieldData;
package org.elasticsearch.index.fielddata;
/**
*
*/
public class IntDocFieldData extends NumericDocFieldData<IntFieldData> {
public interface AtomicNumericFieldData<Script extends ScriptDocValues> extends AtomicFieldData<Script> {
public IntDocFieldData(IntFieldData fieldData) {
super(fieldData);
}
ByteValues getByteValues();
public int getValue() {
return fieldData.value(docId);
}
ShortValues getShortValues();
public int[] getValues() {
return fieldData.values(docId);
}
}
IntValues getIntValues();
LongValues getLongValues();
FloatValues getFloatValues();
DoubleValues getDoubleValues();
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
/**
*/
public interface AtomicOrdinalFieldData<Script extends ScriptDocValues> extends AtomicFieldData<Script> {
/**
* Use a non thread safe (lightweight) view of the values as bytes.
*/
OrdinalsBytesValues getBytesValues();
/**
* Use a non thread safe (lightweight) view of the values as bytes.
*/
OrdinalsHashedBytesValues getHashedBytesValues();
/**
* Use a non thread safe (lightweight) view of the values as strings.
*/
OrdinalsStringValues getStringValues();
}

View File

@ -0,0 +1,335 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.ByteArrayRef;
import org.elasticsearch.index.fielddata.util.IntArrayRef;
import org.elasticsearch.index.fielddata.util.LongArrayRef;
/**
*/
public interface ByteValues {
static ByteValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
byte getValue(int docId);
byte getValueMissing(int docId, byte missingValue);
ByteArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, byte value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
byte next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public byte next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public byte value;
public boolean done;
public Single reset(byte value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public byte next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements ByteValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public byte getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty ByteValues");
}
@Override
public byte getValueMissing(int docId, byte missingValue) {
return missingValue;
}
@Override
public ByteArrayRef getValues(int docId) {
return ByteArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class IntBased implements ByteValues {
private final IntValues values;
private final ByteArrayRef arrayScratch = new ByteArrayRef(new byte[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public IntBased(IntValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public byte getValue(int docId) {
return (byte) values.getValue(docId);
}
@Override
public byte getValueMissing(int docId, byte missingValue) {
return (byte) values.getValueMissing(docId, missingValue);
}
@Override
public ByteArrayRef getValues(int docId) {
IntArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return ByteArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (byte) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private IntValues.Iter iter;
public ValueIter reset(IntValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public byte next() {
return (byte) iter.next();
}
}
static class Proc implements IntValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, int value) {
proc.onValue(docId, (byte) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class LongBased implements ByteValues {
private final LongValues values;
private final ByteArrayRef arrayScratch = new ByteArrayRef(new byte[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public LongBased(LongValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public byte getValue(int docId) {
return (byte) values.getValue(docId);
}
@Override
public byte getValueMissing(int docId, byte missingValue) {
return (byte) values.getValueMissing(docId, missingValue);
}
@Override
public ByteArrayRef getValues(int docId) {
LongArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return ByteArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (byte) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private LongValues.Iter iter;
public ValueIter reset(LongValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public byte next() {
return (byte) iter.next();
}
}
static class Proc implements LongValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, long value) {
proc.onValue(docId, (byte) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,289 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.BytesRefArrayRef;
import org.elasticsearch.index.fielddata.util.StringArrayRef;
/**
*/
public interface BytesValues {
static final BytesValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
/**
* Converts the provided bytes to "safe" ones from a "non" safe call made (if needed).
*/
BytesRef makeSafe(BytesRef bytes);
/**
* Returns a bytes value for a docId. Note, the content of it might be shared across invocation.
*/
BytesRef getValue(int docId);
/**
* Returns the bytes value for the docId, with the provided "ret" which will be filled with the
* result which will also be returned. If there is no value for this docId, the length will be 0.
* Note, the bytes are not "safe".
*/
BytesRef getValueScratch(int docId, BytesRef ret);
/**
* Returns an array wrapping all the bytes values for a doc. The content is guaranteed not to be shared.
*/
BytesRefArrayRef getValues(int docId);
/**
* Returns a bytes value iterator for a docId. Note, the content of it might be shared across invocation.
*/
Iter getIter(int docId);
/**
* Go over all the possible values in their BytesRef format for a specific doc.
*/
void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, BytesRef value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
BytesRef next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public BytesRef next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public BytesRef value;
public boolean done;
public Single reset(BytesRef value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public BytesRef next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements BytesValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public BytesRef getValue(int docId) {
return null;
}
@Override
public BytesRefArrayRef getValues(int docId) {
return BytesRefArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
@Override
public BytesRef makeSafe(BytesRef bytes) {
//todo we can also throw an excepiton here as the only value this method accepts is a scratch value...
//todo ...extracted from this ByteValues, in our case, there are not values, so this should never be called!?!?
return BytesRef.deepCopyOf(bytes);
}
@Override
public BytesRef getValueScratch(int docId, BytesRef ret) {
ret.length = 0;
return ret;
}
}
public static class StringBased implements BytesValues {
private final StringValues values;
protected final BytesRef scratch = new BytesRef();
private final BytesRefArrayRef arrayScratch = new BytesRefArrayRef(new BytesRef[1], 1);
private final ValueIter valueIter = new ValueIter();
private final Proc proc = new Proc();
public StringBased(StringValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public BytesRef makeSafe(BytesRef bytes) {
// we need to make a copy, since we use scratch to provide it
return BytesRef.deepCopyOf(bytes);
}
@Override
public BytesRef getValue(int docId) {
String value = values.getValue(docId);
if (value == null) return null;
scratch.copyChars(value);
return scratch;
}
@Override
public BytesRef getValueScratch(int docId, BytesRef ret) {
String value = values.getValue(docId);
if (value == null) {
ret.length = 0;
return ret;
}
ret.copyChars(value);
return ret;
}
@Override
public BytesRefArrayRef getValues(int docId) {
StringArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return BytesRefArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
String value = arrayRef.values[i];
arrayScratch.values[arrayScratch.end++] = value == null ? null : new BytesRef(value);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valueIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private final BytesRef scratch = new BytesRef();
private StringValues.Iter iter;
public ValueIter reset(StringValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public BytesRef next() {
scratch.copyChars(iter.next());
return scratch;
}
}
static class Proc implements StringValues.ValueInDocProc {
private final BytesRef scratch = new BytesRef();
private BytesValues.ValueInDocProc proc;
public Proc reset(BytesValues.ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, String value) {
scratch.copyChars(value);
proc.onValue(docId, scratch);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,241 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.DoubleArrayRef;
import org.elasticsearch.index.fielddata.util.IntArrayRef;
import org.elasticsearch.index.fielddata.util.LongArrayRef;
/**
*/
public interface DoubleValues {
static final DoubleValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
double getValue(int docId);
double getValueMissing(int docId, double missingValue);
DoubleArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, double value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
double next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public double next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public double value;
public boolean done;
public Single reset(double value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public double next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements DoubleValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public double getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty DoubleValues");
}
@Override
public double getValueMissing(int docId, double missingValue) {
return missingValue;
}
@Override
public DoubleArrayRef getValues(int docId) {
return DoubleArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class LongBased implements DoubleValues {
private final LongValues values;
private final DoubleArrayRef arrayScratch = new DoubleArrayRef(new double[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public LongBased(LongValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public double getValue(int docId) {
return (double) values.getValue(docId);
}
@Override
public double getValueMissing(int docId, double missingValue) {
if (!values.hasValue(docId)) {
return missingValue;
}
return getValue(docId);
}
@Override
public DoubleArrayRef getValues(int docId) {
LongArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return DoubleArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (double) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return this.iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private LongValues.Iter iter;
private ValueIter reset(LongValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public double next() {
return (double) iter.next();
}
}
static class Proc implements LongValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, long value) {
this.proc.onValue(docId, (double) value);
}
@Override
public void onMissing(int docId) {
this.proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
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.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString;
import java.io.IOException;
/**
*/
public class FieldDataStats implements Streamable, ToXContent {
long memorySize;
public FieldDataStats() {
}
public FieldDataStats(long memorySize) {
this.memorySize = memorySize;
}
public void add(FieldDataStats stats) {
this.memorySize += stats.memorySize;
}
public long getMemorySizeInBytes() {
return this.memorySize;
}
public ByteSizeValue getMemorySize() {
return new ByteSizeValue(memorySize);
}
@Override
public void readFrom(StreamInput in) throws IOException {
memorySize = in.readVLong();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVLong(memorySize);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(Fields.FIELD_DATA);
builder.field(Fields.MEMORY_SIZE, memorySize);
builder.field(Fields.MEMORY_SIZE_IN_BYTES, getMemorySize().toString());
builder.endObject();
return builder;
}
static final class Fields {
static final XContentBuilderString FIELD_DATA = new XContentBuilderString("field_data");
static final XContentBuilderString MEMORY_SIZE = new XContentBuilderString("memory_size");
static final XContentBuilderString MEMORY_SIZE_IN_BYTES = new XContentBuilderString("memory_size_in_bytes");
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import com.google.common.collect.ImmutableMap;
import org.elasticsearch.common.Nullable;
/**
*/
public class FieldDataType {
private final String type;
@Nullable
private final String format;
private final ImmutableMap<String, String> options;
public FieldDataType(String type) {
this(type, null, ImmutableMap.<String, String>of());
}
public FieldDataType(String type, String format) {
this(type, format, ImmutableMap.<String, String>of());
}
public FieldDataType(String type, @Nullable String format, ImmutableMap<String, String> options) {
this.type = type;
this.format = format;
this.options = options;
}
public String getType() {
return this.type;
}
@Nullable
public String getFormat() {
return this.format;
}
public ImmutableMap<String, String> getOptions() {
return this.options;
}
}

View File

@ -0,0 +1,237 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.DoubleArrayRef;
import org.elasticsearch.index.fielddata.util.FloatArrayRef;
/**
*/
public interface FloatValues {
static final FloatValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
float getValue(int docId);
float getValueMissing(int docId, float missingValue);
FloatArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, float value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
float next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public float next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public float value;
public boolean done;
public Single reset(float value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public float next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements FloatValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public float getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty FloatValues");
}
@Override
public float getValueMissing(int docId, float missingValue) {
return missingValue;
}
@Override
public FloatArrayRef getValues(int docId) {
return FloatArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class DoubleBased implements FloatValues {
private final DoubleValues values;
private final FloatArrayRef arrayScratch = new FloatArrayRef(new float[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public DoubleBased(DoubleValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public float getValue(int docId) {
return (float) values.getValue(docId);
}
@Override
public float getValueMissing(int docId, float missingValue) {
return (float) values.getValueMissing(docId, missingValue);
}
@Override
public FloatArrayRef getValues(int docId) {
DoubleArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return FloatArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (float) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private DoubleValues.Iter iter;
public ValueIter reset(DoubleValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public float next() {
return (float) iter.next();
}
}
static class Proc implements DoubleValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, double value) {
proc.onValue(docId, (float) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,176 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.GeoPointArrayRef;
import org.elasticsearch.index.mapper.geo.GeoPoint;
/**
*/
public interface GeoPointValues {
static final GeoPointValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
GeoPoint getValue(int docId);
GeoPoint getValueSafe(int docId);
GeoPointArrayRef getValues(int docId);
Iter getIter(int docId);
Iter getIterSafe(int docId);
/**
* Go over all the possible values in their geo point format for a specific doc.
*/
void forEachValueInDoc(int docId, ValueInDocProc proc);
/**
* Go over all the possible values in their geo point format for a specific doc.
*/
void forEachSafeValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, GeoPoint value);
void onMissing(int docId);
}
/**
* Go over all the possible values in their geo point format for a specific doc.
*/
void forEachLatLonValueInDoc(int docId, LatLonValueInDocProc proc);
public static interface LatLonValueInDocProc {
void onValue(int docId, double lat, double lon);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
GeoPoint next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public GeoPoint next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public GeoPoint value;
public boolean done;
public Single reset(GeoPoint value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public GeoPoint next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements GeoPointValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public GeoPoint getValueSafe(int docId) {
return getValue(docId);
}
@Override
public Iter getIterSafe(int docId) {
return getIter(docId);
}
@Override
public void forEachSafeValueInDoc(int docId, ValueInDocProc proc) {
}
@Override
public void forEachLatLonValueInDoc(int docId, LatLonValueInDocProc proc) {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
public GeoPoint getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty GeoPointValues");
}
@Override
public GeoPointArrayRef getValues(int docId) {
return GeoPointArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
}

View File

@ -0,0 +1,333 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.common.lucene.HashedBytesRef;
/**
*/
public interface HashedBytesValues {
static final HashedBytesValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
/**
* Converts the provided bytes to "safe" ones from a "non" safe call made (if needed).
*/
HashedBytesRef makeSafe(HashedBytesRef bytes);
/**
* Returns a bytes value for a docId. Note, the content of it might be shared across invocation,
* call {@link #makeSafe(org.elasticsearch.common.lucene.HashedBytesRef)} to converts it to a "safe"
* option (if needed).
*/
HashedBytesRef getValue(int docId);
/**
* Returns a bytes value iterator for a docId. Note, the content of it might be shared across invocation.
*/
Iter getIter(int docId);
/**
* Go over all the possible values in their BytesRef format for a specific doc.
*/
void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, HashedBytesRef value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
HashedBytesRef next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public HashedBytesRef next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public HashedBytesRef value;
public boolean done;
public Single reset(HashedBytesRef value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public HashedBytesRef next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements HashedBytesValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public HashedBytesRef getValue(int docId) {
return null;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
@Override
public HashedBytesRef makeSafe(HashedBytesRef bytes) {
//todo maybe better to throw an excepiton here as the only value this method accepts is a scratch value...
//todo ...extracted from this ByteValues, in our case, there are not values, so this should never be called!?!?
return HashedBytesRef.deepCopyOf(bytes);
}
}
/**
* A {@link BytesValues} based implementation.
*/
static class BytesBased implements HashedBytesValues {
private final BytesValues values;
protected final HashedBytesRef scratch = new HashedBytesRef(new BytesRef());
private final ValueIter valueIter = new ValueIter();
private final Proc proc = new Proc();
public BytesBased(BytesValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public HashedBytesRef makeSafe(HashedBytesRef bytes) {
return new HashedBytesRef(values.makeSafe(bytes.bytes), bytes.hash);
}
@Override
public HashedBytesRef getValue(int docId) {
BytesRef value = values.getValue(docId);
if (value == null) return null;
scratch.bytes = value;
return scratch.resetHashCode();
}
@Override
public Iter getIter(int docId) {
return valueIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, final ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private final HashedBytesRef scratch = new HashedBytesRef(new BytesRef());
private BytesValues.Iter iter;
public ValueIter reset(BytesValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public HashedBytesRef next() {
scratch.bytes = iter.next();
return scratch.resetHashCode();
}
}
static class Proc implements BytesValues.ValueInDocProc {
private final HashedBytesRef scratch = new HashedBytesRef();
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, BytesRef value) {
scratch.bytes = value;
proc.onValue(docId, scratch.resetHashCode());
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
static class StringBased implements HashedBytesValues {
private final StringValues values;
protected final HashedBytesRef scratch = new HashedBytesRef(new BytesRef());
private final ValueIter valueIter = new ValueIter();
private final Proc proc = new Proc();
public StringBased(StringValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public HashedBytesRef makeSafe(HashedBytesRef bytes) {
// we use scratch to provide it, so just need to copy it over to a new instance
return new HashedBytesRef(bytes.bytes, bytes.hash);
}
@Override
public HashedBytesRef getValue(int docId) {
String value = values.getValue(docId);
if (value == null) return null;
scratch.bytes.copyChars(value);
return scratch.resetHashCode();
}
@Override
public Iter getIter(int docId) {
return valueIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, final ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private final HashedBytesRef scratch = new HashedBytesRef(new BytesRef());
private StringValues.Iter iter;
public ValueIter reset(StringValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public HashedBytesRef next() {
scratch.bytes.copyChars(iter.next());
return scratch.resetHashCode();
}
}
static class Proc implements StringValues.ValueInDocProc {
private final HashedBytesRef scratch = new HashedBytesRef(new BytesRef());
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, String value) {
scratch.bytes.copyChars(value);
proc.onValue(docId, scratch);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldComparatorSource;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexComponent;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.settings.IndexSettings;
/**
*/
public interface IndexFieldData<FD extends AtomicFieldData> extends IndexComponent {
/**
* The field name.
*/
FieldMapper.Names getFieldNames();
/**
* Are the values ordered? (in ascending manner).
*/
boolean valuesOrdered();
/**
* Loads the atomic field data for the reader, possibly cached.
*/
FD load(AtomicReaderContext context);
/**
* Loads directly the atomic field data for the reader, ignoring any caching involved.
*/
FD loadDirect(AtomicReaderContext context) throws Exception;
/**
* Comparator used for sorting.
*/
XFieldComparatorSource comparatorSource(@Nullable Object missingValue);
/**
* Clears any resources associated with this field data.
*/
void clear();
void clear(IndexReader reader);
// we need this extended source we we have custom comparators to reuse our field data
// in this case, we need to reduce type that will be used when search results are reduced
// on another node (we don't have the custom source them...)
public abstract class XFieldComparatorSource extends FieldComparatorSource {
public abstract SortField.Type reducedType();
}
interface Builder {
IndexFieldData build(Index index, @IndexSettings Settings indexSettings, FieldMapper.Names fieldNames, FieldDataType type, IndexFieldDataCache cache);
}
}

View File

@ -0,0 +1,102 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.SegmentReader;
import org.elasticsearch.index.Index;
import java.util.concurrent.Callable;
/**
* A simple field data cache abstraction.
*/
public interface IndexFieldDataCache {
<FD extends AtomicFieldData, IFD extends IndexFieldData<FD>> FD load(AtomicReaderContext context, IFD indexFieldData) throws Exception;
void clear(Index index);
void clear(Index index, String fieldName);
void clear(Index index, IndexReader reader);
/**
* The resident field data cache is a *per field* cache that keeps all the values in memory.
*/
static abstract class FieldBased implements IndexFieldDataCache, SegmentReader.CoreClosedListener {
private final Cache<Object, AtomicFieldData> cache;
protected FieldBased(Cache<Object, AtomicFieldData> cache) {
this.cache = cache;
}
@Override
public void onClose(SegmentReader owner) {
cache.invalidate(owner.getCoreCacheKey());
}
@Override
public <FD extends AtomicFieldData, IFD extends IndexFieldData<FD>> FD load(final AtomicReaderContext context, final IFD indexFieldData) throws Exception {
//noinspection unchecked
return (FD) cache.get(context.reader().getCoreCacheKey(), new Callable<AtomicFieldData>() {
@Override
public AtomicFieldData call() throws Exception {
if (context.reader() instanceof SegmentReader) {
((SegmentReader) context.reader()).addCoreClosedListener(FieldBased.this);
}
return indexFieldData.loadDirect(context);
}
});
}
@Override
public void clear(Index index) {
cache.invalidateAll();
}
@Override
public void clear(Index index, String fieldName) {
cache.invalidateAll();
}
@Override
public void clear(Index index, IndexReader reader) {
cache.invalidate(reader.getCoreCacheKey());
}
}
static class Resident extends FieldBased {
public Resident() {
super(CacheBuilder.newBuilder().<Object, AtomicFieldData>build());
}
}
static class Soft extends FieldBased {
public Soft() {
super(CacheBuilder.newBuilder().softValues().<Object, AtomicFieldData>build());
}
}
}

View File

@ -17,24 +17,23 @@
* under the License.
*/
package org.elasticsearch.index.field.data.bytes;
package org.elasticsearch.index.fielddata;
import org.elasticsearch.index.field.data.NumericDocFieldData;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
/**
*
*/
public class ByteDocFieldData extends NumericDocFieldData<ByteFieldData> {
public class IndexFieldDataModule extends AbstractModule {
public ByteDocFieldData(ByteFieldData fieldData) {
super(fieldData);
private final Settings settings;
public IndexFieldDataModule(Settings settings) {
this.settings = settings;
}
public byte getValue() {
return fieldData.value(docId);
}
public byte[] getValues() {
return fieldData.values(docId);
@Override
protected void configure() {
bind(IndexFieldDataService.class).asEagerSingleton();
}
}

View File

@ -0,0 +1,163 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import com.google.common.collect.ImmutableMap;
import org.apache.lucene.index.IndexReader;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.plain.*;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.settings.IndexSettings;
import java.util.concurrent.ConcurrentMap;
/**
*/
public class IndexFieldDataService extends AbstractIndexComponent {
private final static ImmutableMap<String, IndexFieldData.Builder> buildersByType;
private final static ImmutableMap<Tuple<String, String>, IndexFieldData.Builder> buildersByTypeAndFormat;
static {
buildersByType = MapBuilder.<String, IndexFieldData.Builder>newMapBuilder()
.put("string", new ConcreteBytesRefIndexFieldData.Builder())
.put("float", new FloatArrayIndexFieldData.Builder())
.put("double", new DoubleArrayIndexFieldData.Builder())
.put("byte", new ByteArrayIndexFieldData.Builder())
.put("short", new ShortArrayIndexFieldData.Builder())
.put("int", new IntArrayIndexFieldData.Builder())
.put("long", new LongArrayIndexFieldData.Builder())
.put("geo_point", new GeoPointDoubleArrayIndexFieldData.Builder())
.immutableMap();
buildersByTypeAndFormat = MapBuilder.<Tuple<String, String>, IndexFieldData.Builder>newMapBuilder()
.put(Tuple.tuple("string", "concrete_bytes"), new ConcreteBytesRefIndexFieldData.Builder())
.put(Tuple.tuple("string", "paged_bytes"), new PagedBytesIndexFieldData.Builder())
.put(Tuple.tuple("float", "array"), new FloatArrayIndexFieldData.Builder())
.put(Tuple.tuple("double", "array"), new DoubleArrayIndexFieldData.Builder())
.put(Tuple.tuple("byte", "array"), new ByteArrayIndexFieldData.Builder())
.put(Tuple.tuple("short", "array"), new ShortArrayIndexFieldData.Builder())
.put(Tuple.tuple("int", "array"), new IntArrayIndexFieldData.Builder())
.put(Tuple.tuple("long", "array"), new LongArrayIndexFieldData.Builder())
.put(Tuple.tuple("geo_point", "array"), new GeoPointDoubleArrayIndexFieldData.Builder())
.immutableMap();
}
private final ConcurrentMap<String, IndexFieldData> loadedFieldData = ConcurrentCollections.newConcurrentMap();
public IndexFieldDataService(Index index) {
this(index, ImmutableSettings.Builder.EMPTY_SETTINGS);
}
@Inject
public IndexFieldDataService(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
}
public void clear() {
synchronized (loadedFieldData) {
for (IndexFieldData fieldData : loadedFieldData.values()) {
fieldData.clear();
}
loadedFieldData.clear();
}
}
public void clearField(String fieldName) {
synchronized (loadedFieldData) {
IndexFieldData fieldData = loadedFieldData.remove(fieldName);
if (fieldData != null) {
fieldData.clear();
}
}
}
public void clear(IndexReader reader) {
for (IndexFieldData indexFieldData : loadedFieldData.values()) {
indexFieldData.clear(reader);
}
}
public FieldDataStats stats() {
// TODO: compute the memory size here...
return new FieldDataStats();
}
public <IFD extends IndexFieldData> IFD getForField(FieldMapper mapper) {
return getForField(mapper.names(), mapper.fieldDataType());
}
public <IFD extends IndexFieldData> IFD getForField(FieldMapper.Names fieldNames, FieldDataType type) {
IndexFieldData fieldData = loadedFieldData.get(type.getType());
if (fieldData == null) {
synchronized (loadedFieldData) {
fieldData = loadedFieldData.get(type.getType());
if (fieldData == null) {
IndexFieldData.Builder builder = null;
if (type.getFormat() != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), type.getFormat()));
if (builder == null) {
logger.warn("failed to find format [" + type.getFormat() + "] for field [" + fieldNames.fullName() + "], will use default");
}
}
String format = indexSettings.get("index.fielddata.type." + type.getType() + ".format", null);
if (format != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), type.getFormat()));
if (builder == null) {
logger.warn("failed to find index level type format [" + format + "] for field [" + fieldNames.fullName() + "], will use default");
}
}
if (builder == null) {
builder = buildersByType.get(type.getType());
}
if (builder == null) {
throw new ElasticSearchIllegalArgumentException("failed to find field data builder for field " + fieldNames.fullName() + ", and type " + type);
}
IndexFieldDataCache cache;
if (type.getOptions().containsKey("cache")) {
String cacheType = type.getOptions().get("cache");
if ("resident".equals(cacheType)) {
cache = new IndexFieldDataCache.Resident();
} else if ("soft".equals(cacheType)) {
cache = new IndexFieldDataCache.Soft();
} else {
throw new ElasticSearchIllegalArgumentException("cache type not supported [" + cacheType + "] for field [" + fieldNames.fullName() + "]");
}
} else {
cache = new IndexFieldDataCache.Resident();
}
fieldData = builder.build(index, indexSettings, fieldNames, type, cache);
loadedFieldData.put(fieldNames.indexName(), fieldData);
}
}
}
return (IFD) fieldData;
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.index.AtomicReaderContext;
/**
*/
public interface IndexGeoPointFieldData<FD extends AtomicGeoPointFieldData> extends IndexFieldData<FD> {
/**
* Loads the atomic field data for the reader, possibly cached.
*/
FD load(AtomicReaderContext context);
/**
* Loads directly the atomic field data for the reader, ignoring any caching involved.
*/
FD loadDirect(AtomicReaderContext context) throws Exception;
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.index.AtomicReaderContext;
/**
*/
public interface IndexNumericFieldData<FD extends AtomicNumericFieldData> extends IndexFieldData<FD> {
static enum NumericType {
BYTE {
@Override
public boolean isFloatingPoint() {
return false;
}
},
SHORT {
@Override
public boolean isFloatingPoint() {
return false;
}
},
INT {
@Override
public boolean isFloatingPoint() {
return false;
}
},
LONG {
@Override
public boolean isFloatingPoint() {
return false;
}
},
FLOAT {
@Override
public boolean isFloatingPoint() {
return true;
}
},
DOUBLE {
@Override
public boolean isFloatingPoint() {
return true;
}
};
public abstract boolean isFloatingPoint();
}
NumericType getNumericType();
/**
* Loads the atomic field data for the reader, possibly cached.
*/
FD load(AtomicReaderContext context);
/**
* Loads directly the atomic field data for the reader, ignoring any caching involved.
*/
FD loadDirect(AtomicReaderContext context) throws Exception;
}

View File

@ -17,24 +17,21 @@
* under the License.
*/
package org.elasticsearch.index.field.data.doubles;
package org.elasticsearch.index.fielddata;
import org.elasticsearch.index.field.data.NumericDocFieldData;
import org.apache.lucene.index.AtomicReaderContext;
/**
*
*/
public class DoubleDocFieldData extends NumericDocFieldData<DoubleFieldData> {
public interface IndexOrdinalFieldData<FD extends AtomicOrdinalFieldData> extends IndexFieldData<FD> {
public DoubleDocFieldData(DoubleFieldData fieldData) {
super(fieldData);
}
/**
* Loads the atomic field data for the reader, possibly cached.
*/
FD load(AtomicReaderContext context);
public double getValue() {
return fieldData.value(docId);
}
public double[] getValues() {
return fieldData.values(docId);
}
}
/**
* Loads directly the atomic field data for the reader, ignoring any caching involved.
*/
FD loadDirect(AtomicReaderContext context) throws Exception;
}

View File

@ -0,0 +1,237 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.IntArrayRef;
import org.elasticsearch.index.fielddata.util.LongArrayRef;
/**
*/
public interface IntValues {
static final IntValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
int getValue(int docId);
int getValueMissing(int docId, int missingValue);
IntArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, int value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
int next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public int next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public int value;
public boolean done;
public Single reset(int value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public int next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements IntValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public int getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty IntValues");
}
@Override
public int getValueMissing(int docId, int missingValue) {
return missingValue;
}
@Override
public IntArrayRef getValues(int docId) {
return IntArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class LongBased implements IntValues {
private final LongValues values;
private final IntArrayRef arrayScratch = new IntArrayRef(new int[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public LongBased(LongValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public int getValue(int docId) {
return (int) values.getValue(docId);
}
@Override
public int getValueMissing(int docId, int missingValue) {
return (int) values.getValueMissing(docId, missingValue);
}
@Override
public IntArrayRef getValues(int docId) {
LongArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return IntArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (int) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private LongValues.Iter iter;
public ValueIter reset(LongValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public int next() {
return (int) iter.next();
}
}
static class Proc implements LongValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, long value) {
proc.onValue(docId, (int) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,142 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.IntArrayRef;
import org.elasticsearch.index.fielddata.util.LongArrayRef;
/**
*/
public interface LongValues {
static final LongValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
long getValue(int docId);
long getValueMissing(int docId, long missingValue);
LongArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, long value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
long next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public long next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public long value;
public boolean done;
public Single reset(long value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public long next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements LongValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public long getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty LongValues");
}
@Override
public long getValueMissing(int docId, long missingValue) {
return missingValue;
}
@Override
public LongArrayRef getValues(int docId) {
return LongArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
}

View File

@ -0,0 +1,104 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
/**
*/
public interface OrdinalsBytesValues extends BytesValues {
Ordinals.Docs ordinals();
BytesRef getValueByOrd(int ord);
/**
* Returns the bytes value for the docId, with the provided "ret" which will be filled with the
* result which will also be returned. If there is no value for this docId, the length will be 0.
* Note, the bytes are not "safe".
*/
BytesRef getValueScratchByOrd(int ord, BytesRef ret);
BytesRef getSafeValueByOrd(int ord);
public static class Empty extends BytesValues.Empty implements OrdinalsBytesValues {
private final Ordinals ordinals;
public Empty(EmptyOrdinals ordinals) {
this.ordinals = ordinals;
}
@Override
public Ordinals.Docs ordinals() {
return ordinals.ordinals();
}
@Override
public BytesRef getValueByOrd(int ord) {
return null;
}
@Override
public BytesRef getValueScratchByOrd(int ord, BytesRef ret) {
ret.length = 0;
return ret;
}
@Override
public BytesRef getSafeValueByOrd(int ord) {
return null;
}
}
public static class StringBased extends BytesValues.StringBased implements OrdinalsBytesValues {
private final OrdinalsStringValues values;
public StringBased(OrdinalsStringValues values) {
super(values);
this.values = values;
}
@Override
public Ordinals.Docs ordinals() {
return values.ordinals();
}
@Override
public BytesRef getValueByOrd(int ord) {
scratch.copyChars(values.getValueByOrd(ord));
return scratch;
}
@Override
public BytesRef getValueScratchByOrd(int ord, BytesRef ret) {
ret.copyChars(values.getValueByOrd(ord));
return ret;
}
@Override
public BytesRef getSafeValueByOrd(int ord) {
return new BytesRef(values.getValueByOrd(ord));
}
}
}

View File

@ -0,0 +1,112 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
/**
*/
public interface OrdinalsHashedBytesValues extends HashedBytesValues {
Ordinals.Docs ordinals();
HashedBytesRef getValueByOrd(int ord);
HashedBytesRef getSafeValueByOrd(int ord);
public static class Empty extends HashedBytesValues.Empty implements OrdinalsHashedBytesValues {
private final Ordinals ordinals;
public Empty(EmptyOrdinals ordinals) {
this.ordinals = ordinals;
}
@Override
public Ordinals.Docs ordinals() {
return ordinals.ordinals();
}
@Override
public HashedBytesRef getValueByOrd(int ord) {
return null;
}
@Override
public HashedBytesRef getSafeValueByOrd(int ord) {
return null;
}
}
static class BytesBased extends HashedBytesValues.BytesBased implements OrdinalsHashedBytesValues {
private final OrdinalsBytesValues values;
public BytesBased(OrdinalsBytesValues values) {
super(values);
this.values = values;
}
@Override
public Ordinals.Docs ordinals() {
return values.ordinals();
}
@Override
public HashedBytesRef getValueByOrd(int ord) {
scratch.bytes = values.getValueByOrd(ord);
return scratch.resetHashCode();
}
@Override
public HashedBytesRef getSafeValueByOrd(int ord) {
return new HashedBytesRef(values.getSafeValueByOrd(ord));
}
}
static class StringBased extends HashedBytesValues.StringBased implements OrdinalsHashedBytesValues {
private final OrdinalsStringValues values;
public StringBased(OrdinalsStringValues values) {
super(values);
this.values = values;
}
@Override
public Ordinals.Docs ordinals() {
return values.ordinals();
}
@Override
public HashedBytesRef getValueByOrd(int ord) {
scratch.bytes.copyChars(values.getValueByOrd(ord));
return scratch.resetHashCode();
}
@Override
public HashedBytesRef getSafeValueByOrd(int ord) {
return new HashedBytesRef(new BytesRef(values.getValueByOrd(ord)));
}
}
}

View File

@ -17,33 +17,35 @@
* under the License.
*/
package org.elasticsearch.index.field.data.longs;
package org.elasticsearch.index.fielddata;
import org.elasticsearch.index.field.data.NumericDocFieldData;
import org.joda.time.MutableDateTime;
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
/**
*
*/
public class LongDocFieldData extends NumericDocFieldData<LongFieldData> {
public interface OrdinalsStringValues extends StringValues {
public LongDocFieldData(LongFieldData fieldData) {
super(fieldData);
}
Ordinals.Docs ordinals();
public long getValue() {
return fieldData.value(docId);
}
String getValueByOrd(int ord);
public long[] getValues() {
return fieldData.values(docId);
}
public static class Empty extends StringValues.Empty implements OrdinalsStringValues {
public MutableDateTime getDate() {
return fieldData.date(docId);
}
private final Ordinals ordinals;
public MutableDateTime[] getDates() {
return fieldData.dates(docId);
public Empty(EmptyOrdinals ordinals) {
this.ordinals = ordinals;
}
@Override
public Ordinals.Docs ordinals() {
return ordinals.ordinals();
}
@Override
public String getValueByOrd(int ord) {
return null;
}
}
}
}

View File

@ -0,0 +1,385 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.util.*;
import org.elasticsearch.index.mapper.geo.GeoPoint;
import org.elasticsearch.index.search.geo.GeoDistance;
import org.joda.time.MutableDateTime;
/**
* Script level doc values, the assumption is that any implementation will implement a <code>getValue</code>
* and a <code>getValues</code> that return the relevant type that then can be used in scripts.
*/
public interface ScriptDocValues {
static final ScriptDocValues EMPTY = new Empty();
static final Strings EMPTY_STRINGS = new Strings(StringValues.EMPTY);
void setNextDocId(int docId);
boolean isEmpty();
static class Empty implements ScriptDocValues {
@Override
public void setNextDocId(int docId) {
}
@Override
public boolean isEmpty() {
return true;
}
}
static class Strings implements ScriptDocValues {
private final StringValues values;
private int docId;
public Strings(StringValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public String getValue() {
return values.getValue(docId);
}
public StringArrayRef getValues() {
return values.getValues(docId);
}
}
static class Bytes implements ScriptDocValues {
private final BytesValues values;
private int docId;
public Bytes(BytesValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public BytesRef getValue() {
return values.getValue(docId);
}
public BytesRefArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericByte implements ScriptDocValues {
private final ByteValues values;
private int docId;
public NumericByte(ByteValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public byte getValue() {
return values.getValue(docId);
}
public ByteArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericShort implements ScriptDocValues {
private final ShortValues values;
private int docId;
public NumericShort(ShortValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public short getValue() {
return values.getValue(docId);
}
public ShortArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericInteger implements ScriptDocValues {
private final IntValues values;
private int docId;
public NumericInteger(IntValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public int getValue() {
return values.getValue(docId);
}
public IntArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericLong implements ScriptDocValues {
private final LongValues values;
private final MutableDateTime date = new MutableDateTime(0);
private int docId;
public NumericLong(LongValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public long getValue() {
return values.getValue(docId);
}
public MutableDateTime getDate() {
date.setMillis(getValue());
return date;
}
public LongArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericFloat implements ScriptDocValues {
private final FloatValues values;
private int docId;
public NumericFloat(FloatValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public float getValue() {
return values.getValue(docId);
}
public FloatArrayRef getValues() {
return values.getValues(docId);
}
}
static class NumericDouble implements ScriptDocValues {
private final DoubleValues values;
private int docId;
public NumericDouble(DoubleValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public double getValue() {
return values.getValue(docId);
}
public DoubleArrayRef getValues() {
return values.getValues(docId);
}
}
static class GeoPoints implements ScriptDocValues {
private final GeoPointValues values;
private int docId;
public GeoPoints(GeoPointValues values) {
this.values = values;
}
@Override
public void setNextDocId(int docId) {
this.docId = docId;
}
@Override
public boolean isEmpty() {
return !values.hasValue(docId);
}
public GeoPoint getValue() {
return values.getValue(docId);
}
public GeoPointArrayRef getValues() {
return values.getValues(docId);
}
public double factorDistance(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.FACTOR.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double factorDistanceWithDefault(double lat, double lon, double defaultValue) {
if (isEmpty()) {
return defaultValue;
}
GeoPoint point = getValue();
return GeoDistance.FACTOR.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double factorDistance02(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.FACTOR.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES) + 1;
}
public double factorDistance13(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.FACTOR.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES) + 2;
}
public double arcDistance(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.ARC.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double arcDistanceWithDefault(double lat, double lon, double defaultValue) {
if (isEmpty()) {
return defaultValue;
}
GeoPoint point = getValue();
return GeoDistance.ARC.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double arcDistanceInKm(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.ARC.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.KILOMETERS);
}
public double arcDistanceInKmWithDefault(double lat, double lon, double defaultValue) {
if (isEmpty()) {
return defaultValue;
}
GeoPoint point = getValue();
return GeoDistance.ARC.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.KILOMETERS);
}
public double distance(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.PLANE.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double distanceWithDefault(double lat, double lon, double defaultValue) {
if (isEmpty()) {
return defaultValue;
}
GeoPoint point = getValue();
return GeoDistance.PLANE.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.MILES);
}
public double distanceInKm(double lat, double lon) {
GeoPoint point = getValue();
return GeoDistance.PLANE.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.KILOMETERS);
}
public double distanceInKmWithDefault(double lat, double lon, double defaultValue) {
if (isEmpty()) {
return defaultValue;
}
GeoPoint point = getValue();
return GeoDistance.PLANE.calculate(point.lat(), point.lon(), lat, lon, DistanceUnit.KILOMETERS);
}
}
}

View File

@ -0,0 +1,335 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.IntArrayRef;
import org.elasticsearch.index.fielddata.util.LongArrayRef;
import org.elasticsearch.index.fielddata.util.ShortArrayRef;
/**
*/
public interface ShortValues {
static final ShortValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
short getValue(int docId);
short getValueMissing(int docId, short missingValue);
ShortArrayRef getValues(int docId);
Iter getIter(int docId);
void forEachValueInDoc(int docId, ValueInDocProc proc);
static interface ValueInDocProc {
void onValue(int docId, short value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
short next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public short next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public short value;
public boolean done;
public Single reset(short value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public short next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements ShortValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public short getValue(int docId) {
throw new ElasticSearchIllegalStateException("Can't retrieve a value from an empty ShortValues");
}
@Override
public short getValueMissing(int docId, short missingValue) {
return missingValue;
}
@Override
public ShortArrayRef getValues(int docId) {
return ShortArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class IntBased implements ShortValues {
private final IntValues values;
private final ShortArrayRef arrayScratch = new ShortArrayRef(new short[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public IntBased(IntValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public short getValue(int docId) {
return (short) values.getValue(docId);
}
@Override
public short getValueMissing(int docId, short missingValue) {
return (short) values.getValueMissing(docId, missingValue);
}
@Override
public ShortArrayRef getValues(int docId) {
IntArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return ShortArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (short) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private IntValues.Iter iter;
public ValueIter reset(IntValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public short next() {
return (short) iter.next();
}
}
static class Proc implements IntValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, int value) {
proc.onValue(docId, (short) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class LongBased implements ShortValues {
private final LongValues values;
private final ShortArrayRef arrayScratch = new ShortArrayRef(new short[1], 1);
private final ValueIter iter = new ValueIter();
private final Proc proc = new Proc();
public LongBased(LongValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public short getValue(int docId) {
return (short) values.getValue(docId);
}
@Override
public short getValueMissing(int docId, short missingValue) {
return (short) values.getValueMissing(docId, missingValue);
}
@Override
public ShortArrayRef getValues(int docId) {
LongArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) {
return ShortArrayRef.EMPTY;
}
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = (short) arrayRef.values[i];
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return iter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValueIter implements Iter {
private LongValues.Iter iter;
public ValueIter reset(LongValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public short next() {
return (short) iter.next();
}
}
static class Proc implements LongValues.ValueInDocProc {
private ValueInDocProc proc;
public Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, long value) {
proc.onValue(docId, (short) value);
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,607 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.index.fielddata.util.*;
/**
*/
public interface StringValues {
static final StringValues EMPTY = new Empty();
/**
* Is one of the documents in this field data values is multi valued?
*/
boolean isMultiValued();
/**
* Is there a value for this doc?
*/
boolean hasValue(int docId);
String getValue(int docId);
StringArrayRef getValues(int docId);
Iter getIter(int docId);
/**
* Go over all the possible values.
*/
void forEachValueInDoc(int docId, ValueInDocProc proc);
public static interface ValueInDocProc {
void onValue(int docId, String value);
void onMissing(int docId);
}
static interface Iter {
boolean hasNext();
String next();
static class Empty implements Iter {
public static final Empty INSTANCE = new Empty();
@Override
public boolean hasNext() {
return false;
}
@Override
public String next() {
throw new ElasticSearchIllegalStateException();
}
}
static class Single implements Iter {
public String value;
public boolean done;
public Single reset(String value) {
this.value = value;
this.done = false;
return this;
}
@Override
public boolean hasNext() {
return !done;
}
@Override
public String next() {
assert !done;
done = true;
return value;
}
}
}
static class Empty implements StringValues {
@Override
public boolean isMultiValued() {
return false;
}
@Override
public boolean hasValue(int docId) {
return false;
}
@Override
public String getValue(int docId) {
return null;
}
@Override
public StringArrayRef getValues(int docId) {
return StringArrayRef.EMPTY;
}
@Override
public Iter getIter(int docId) {
return Iter.Empty.INSTANCE;
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
proc.onMissing(docId);
}
}
public static class ByteBased implements StringValues {
private final ByteValues values;
private final StringArrayRef arrayScratch = new StringArrayRef(new String[1], 1);
private final ValuesIter valuesIter = new ValuesIter();
private final Proc proc = new Proc();
public ByteBased(ByteValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public String getValue(int docId) {
if (!values.hasValue(docId)) {
return null;
}
return Byte.toString(values.getValue(docId));
}
@Override
public StringArrayRef getValues(int docId) {
ByteArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) return StringArrayRef.EMPTY;
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = Byte.toString(arrayRef.values[i]);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valuesIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValuesIter implements Iter {
private ByteValues.Iter iter;
private ValuesIter reset(ByteValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public String next() {
return Byte.toString(iter.next());
}
}
static class Proc implements ByteValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, byte value) {
proc.onValue(docId, Byte.toString(value));
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class ShortBased implements StringValues {
private final ShortValues values;
private final StringArrayRef arrayScratch = new StringArrayRef(new String[1], 1);
private final ValuesIter valuesIter = new ValuesIter();
private final Proc proc = new Proc();
public ShortBased(ShortValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public String getValue(int docId) {
if (!values.hasValue(docId)) {
return null;
}
return Short.toString(values.getValue(docId));
}
@Override
public StringArrayRef getValues(int docId) {
ShortArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) return StringArrayRef.EMPTY;
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = Short.toString(arrayRef.values[i]);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valuesIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValuesIter implements Iter {
private ShortValues.Iter iter;
private ValuesIter reset(ShortValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public String next() {
return Short.toString(iter.next());
}
}
static class Proc implements ShortValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, short value) {
proc.onValue(docId, Short.toString(value));
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class IntBased implements StringValues {
private final IntValues values;
private final StringArrayRef arrayScratch = new StringArrayRef(new String[1], 1);
private final ValuesIter valuesIter = new ValuesIter();
private final Proc proc = new Proc();
public IntBased(IntValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public String getValue(int docId) {
if (!values.hasValue(docId)) {
return null;
}
return Integer.toString(values.getValue(docId));
}
@Override
public StringArrayRef getValues(int docId) {
IntArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) return StringArrayRef.EMPTY;
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = Integer.toString(arrayRef.values[i]);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valuesIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValuesIter implements Iter {
private IntValues.Iter iter;
private ValuesIter reset(IntValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public String next() {
return Integer.toString(iter.next());
}
}
static class Proc implements IntValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, int value) {
proc.onValue(docId, Integer.toString(value));
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class LongBased implements StringValues {
private final LongValues values;
private final StringArrayRef arrayScratch = new StringArrayRef(new String[1], 1);
private final ValuesIter valuesIter = new ValuesIter();
private final Proc proc = new Proc();
public LongBased(LongValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public String getValue(int docId) {
if (!values.hasValue(docId)) {
return null;
}
return Long.toString(values.getValue(docId));
}
@Override
public StringArrayRef getValues(int docId) {
LongArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) return StringArrayRef.EMPTY;
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = Long.toString(arrayRef.values[i]);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valuesIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValuesIter implements Iter {
private LongValues.Iter iter;
private ValuesIter reset(LongValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public String next() {
return Long.toString(iter.next());
}
}
static class Proc implements LongValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, long value) {
proc.onValue(docId, Long.toString(value));
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
public static class FloatBased implements StringValues {
private final FloatValues values;
private final StringArrayRef arrayScratch = new StringArrayRef(new String[1], 1);
private final ValuesIter valuesIter = new ValuesIter();
private final Proc proc = new Proc();
public FloatBased(FloatValues values) {
this.values = values;
}
@Override
public boolean isMultiValued() {
return values.isMultiValued();
}
@Override
public boolean hasValue(int docId) {
return values.hasValue(docId);
}
@Override
public String getValue(int docId) {
if (!values.hasValue(docId)) {
return null;
}
return Float.toString(values.getValue(docId));
}
@Override
public StringArrayRef getValues(int docId) {
FloatArrayRef arrayRef = values.getValues(docId);
int size = arrayRef.size();
if (size == 0) return StringArrayRef.EMPTY;
arrayScratch.reset(size);
for (int i = arrayRef.start; i < arrayRef.end; i++) {
arrayScratch.values[arrayScratch.end++] = Float.toString(arrayRef.values[i]);
}
return arrayScratch;
}
@Override
public Iter getIter(int docId) {
return valuesIter.reset(values.getIter(docId));
}
@Override
public void forEachValueInDoc(int docId, ValueInDocProc proc) {
values.forEachValueInDoc(docId, this.proc.reset(proc));
}
static class ValuesIter implements Iter {
private FloatValues.Iter iter;
private ValuesIter reset(FloatValues.Iter iter) {
this.iter = iter;
return this;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public String next() {
return Float.toString(iter.next());
}
}
static class Proc implements FloatValues.ValueInDocProc {
private ValueInDocProc proc;
private Proc reset(ValueInDocProc proc) {
this.proc = proc;
return this;
}
@Override
public void onValue(int docId, float value) {
proc.onValue(docId, Float.toString(value));
}
@Override
public void onMissing(int docId) {
proc.onMissing(docId);
}
}
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.elasticsearch.index.fielddata.ByteValues;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class ByteValuesComparator extends FieldComparator<Byte> {
private final IndexNumericFieldData indexFieldData;
private final byte missingValue;
protected final byte[] values;
private byte bottom;
private ByteValues readerValues;
public ByteValuesComparator(IndexNumericFieldData indexFieldData, byte missingValue, int numHits) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
this.values = new byte[numHits];
}
@Override
public int compare(int slot1, int slot2) {
final byte v1 = values[slot1];
final byte v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void setBottom(int slot) {
this.bottom = values[slot];
}
@Override
public int compareBottom(int doc) throws IOException {
byte v2 = readerValues.getValueMissing(doc, missingValue);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void copy(int slot, int doc) throws IOException {
values[slot] = readerValues.getValueMissing(doc, missingValue);
}
@Override
public FieldComparator<Byte> setNextReader(AtomicReaderContext context) throws IOException {
this.readerValues = indexFieldData.load(context).getByteValues();
return this;
}
@Override
public Byte value(int slot) {
return Byte.valueOf(values[slot]);
}
@Override
public int compareDocToValue(int doc, Byte valueObj) throws IOException {
final byte value = valueObj.byteValue();
byte docValue = readerValues.getValueMissing(doc, missingValue);
if (docValue < value) {
return -1;
} else if (docValue > value) {
return 1;
} else {
return 0;
}
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class ByteValuesComparatorSource extends IndexFieldData.XFieldComparatorSource {
private final IndexNumericFieldData indexFieldData;
private final Object missingValue;
public ByteValuesComparatorSource(IndexNumericFieldData indexFieldData, @Nullable Object missingValue) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
}
@Override
public SortField.Type reducedType() {
return SortField.Type.INT;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert fieldname.equals(indexFieldData.getFieldNames().indexName());
byte dMissingValue;
if (missingValue == null || "_last".equals(missingValue)) {
dMissingValue = reversed ? Byte.MIN_VALUE : Byte.MAX_VALUE;
} else if ("_first".equals(missingValue)) {
dMissingValue = reversed ? Byte.MAX_VALUE : Byte.MIN_VALUE;
} else {
dMissingValue = missingValue instanceof Number ? ((Number) missingValue).byteValue() : Byte.parseByte(missingValue.toString());
}
return new ByteValuesComparator(indexFieldData, dMissingValue, numHits);
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexOrdinalFieldData;
import java.io.IOException;
/**
*/
public class BytesRefFieldComparatorSource extends IndexFieldData.XFieldComparatorSource {
private final IndexFieldData indexFieldData;
public BytesRefFieldComparatorSource(IndexFieldData indexFieldData) {
this.indexFieldData = indexFieldData;
}
@Override
public SortField.Type reducedType() {
return SortField.Type.STRING;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert fieldname.equals(indexFieldData.getFieldNames().indexName());
if (indexFieldData.valuesOrdered() && indexFieldData instanceof IndexOrdinalFieldData) {
return new BytesRefOrdValComparator((IndexOrdinalFieldData) indexFieldData, numHits);
}
return new BytesRefValComparator(indexFieldData, numHits);
}
}

View File

@ -0,0 +1,475 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexOrdinalFieldData;
import org.elasticsearch.index.fielddata.OrdinalsBytesValues;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import java.io.IOException;
/**
* Sorts by field's natural Term sort order, using
* ordinals. This is functionally equivalent to {@link
* org.apache.lucene.search.FieldComparator.TermValComparator}, but it first resolves the string
* to their relative ordinal positions (using the index
* returned by {@link org.apache.lucene.search.FieldCache#getTermsIndex}), and
* does most comparisons using the ordinals. For medium
* to large results, this comparator will be much faster
* than {@link org.apache.lucene.search.FieldComparator.TermValComparator}. For very small
* result sets it may be slower.
*/
public final class BytesRefOrdValComparator extends FieldComparator<BytesRef> {
final IndexOrdinalFieldData indexFieldData;
/* Ords for each slot.
@lucene.internal */
final int[] ords;
/* Values for each slot.
@lucene.internal */
final BytesRef[] values;
/* Which reader last copied a value into the slot. When
we compare two slots, we just compare-by-ord if the
readerGen is the same; else we must compare the
values (slower).
@lucene.internal */
final int[] readerGen;
/* Gen of current reader we are on.
@lucene.internal */
int currentReaderGen = -1;
/* Current reader's doc ord/values.
@lucene.internal */
OrdinalsBytesValues termsIndex;
/* Bottom slot, or -1 if queue isn't full yet
@lucene.internal */
int bottomSlot = -1;
/* Bottom ord (same as ords[bottomSlot] once bottomSlot
is set). Cached for faster compares.
@lucene.internal */
int bottomOrd;
/* True if current bottom slot matches the current
reader.
@lucene.internal */
boolean bottomSameReader;
/* Bottom value (same as values[bottomSlot] once
bottomSlot is set). Cached for faster compares.
@lucene.internal */
BytesRef bottomValue;
final BytesRef tempBR = new BytesRef();
public BytesRefOrdValComparator(IndexOrdinalFieldData indexFieldData, int numHits) {
this.indexFieldData = indexFieldData;
ords = new int[numHits];
values = new BytesRef[numHits];
readerGen = new int[numHits];
}
@Override
public int compare(int slot1, int slot2) {
if (readerGen[slot1] == readerGen[slot2]) {
return ords[slot1] - ords[slot2];
}
final BytesRef val1 = values[slot1];
final BytesRef val2 = values[slot2];
if (val1 == null) {
if (val2 == null) {
return 0;
}
return -1;
} else if (val2 == null) {
return 1;
}
return val1.compareTo(val2);
}
@Override
public int compareBottom(int doc) {
throw new UnsupportedOperationException();
}
@Override
public void copy(int slot, int doc) {
throw new UnsupportedOperationException();
}
@Override
public int compareDocToValue(int doc, BytesRef value) {
BytesRef docValue = termsIndex.getValue(doc);
if (docValue == null) {
if (value == null) {
return 0;
}
return -1;
} else if (value == null) {
return 1;
}
return docValue.compareTo(value);
}
/**
* Base class for specialized (per bit width of the
* ords) per-segment comparator. NOTE: this is messy;
* we do this only because hotspot can't reliably inline
* the underlying array access when looking up doc->ord
*
* @lucene.internal
*/
abstract class PerSegmentComparator extends FieldComparator<BytesRef> {
@Override
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
return BytesRefOrdValComparator.this.setNextReader(context);
}
@Override
public int compare(int slot1, int slot2) {
return BytesRefOrdValComparator.this.compare(slot1, slot2);
}
@Override
public void setBottom(final int bottom) {
BytesRefOrdValComparator.this.setBottom(bottom);
}
@Override
public BytesRef value(int slot) {
return BytesRefOrdValComparator.this.value(slot);
}
@Override
public int compareValues(BytesRef val1, BytesRef val2) {
if (val1 == null) {
if (val2 == null) {
return 0;
}
return -1;
} else if (val2 == null) {
return 1;
}
return val1.compareTo(val2);
}
@Override
public int compareDocToValue(int doc, BytesRef value) {
return BytesRefOrdValComparator.this.compareDocToValue(doc, value);
}
}
// Used per-segment when bit width of doc->ord is 8:
private final class ByteOrdComparator extends PerSegmentComparator {
private final byte[] readerOrds;
private final OrdinalsBytesValues termsIndex;
private final int docBase;
public ByteOrdComparator(byte[] readerOrds, OrdinalsBytesValues termsIndex, int docBase) {
this.readerOrds = readerOrds;
this.termsIndex = termsIndex;
this.docBase = docBase;
}
@Override
public int compareBottom(int doc) {
assert bottomSlot != -1;
final int docOrd = (readerOrds[doc] & 0xFF);
if (bottomSameReader) {
// ord is precisely comparable, even in the equal case
return bottomOrd - docOrd;
} else if (bottomOrd >= docOrd) {
// the equals case always means bottom is > doc
// (because we set bottomOrd to the lower bound in
// setBottom):
return 1;
} else {
return -1;
}
}
@Override
public void copy(int slot, int doc) {
final int ord = readerOrds[doc] & 0xFF;
ords[slot] = ord;
if (ord == 0) {
values[slot] = null;
} else {
assert ord > 0;
if (values[slot] == null) {
values[slot] = new BytesRef();
}
termsIndex.getValueScratchByOrd(ord, values[slot]);
}
readerGen[slot] = currentReaderGen;
}
}
// Used per-segment when bit width of doc->ord is 16:
private final class ShortOrdComparator extends PerSegmentComparator {
private final short[] readerOrds;
private final OrdinalsBytesValues termsIndex;
private final int docBase;
public ShortOrdComparator(short[] readerOrds, OrdinalsBytesValues termsIndex, int docBase) {
this.readerOrds = readerOrds;
this.termsIndex = termsIndex;
this.docBase = docBase;
}
@Override
public int compareBottom(int doc) {
assert bottomSlot != -1;
final int docOrd = (readerOrds[doc] & 0xFFFF);
if (bottomSameReader) {
// ord is precisely comparable, even in the equal case
return bottomOrd - docOrd;
} else if (bottomOrd >= docOrd) {
// the equals case always means bottom is > doc
// (because we set bottomOrd to the lower bound in
// setBottom):
return 1;
} else {
return -1;
}
}
@Override
public void copy(int slot, int doc) {
final int ord = readerOrds[doc] & 0xFFFF;
ords[slot] = ord;
if (ord == 0) {
values[slot] = null;
} else {
assert ord > 0;
if (values[slot] == null) {
values[slot] = new BytesRef();
}
termsIndex.getValueScratchByOrd(ord, values[slot]);
}
readerGen[slot] = currentReaderGen;
}
}
// Used per-segment when bit width of doc->ord is 32:
private final class IntOrdComparator extends PerSegmentComparator {
private final int[] readerOrds;
private final OrdinalsBytesValues termsIndex;
private final int docBase;
public IntOrdComparator(int[] readerOrds, OrdinalsBytesValues termsIndex, int docBase) {
this.readerOrds = readerOrds;
this.termsIndex = termsIndex;
this.docBase = docBase;
}
@Override
public int compareBottom(int doc) {
assert bottomSlot != -1;
final int docOrd = readerOrds[doc];
if (bottomSameReader) {
// ord is precisely comparable, even in the equal case
return bottomOrd - docOrd;
} else if (bottomOrd >= docOrd) {
// the equals case always means bottom is > doc
// (because we set bottomOrd to the lower bound in
// setBottom):
return 1;
} else {
return -1;
}
}
@Override
public void copy(int slot, int doc) {
final int ord = readerOrds[doc];
ords[slot] = ord;
if (ord == 0) {
values[slot] = null;
} else {
assert ord > 0;
if (values[slot] == null) {
values[slot] = new BytesRef();
}
termsIndex.getValueScratchByOrd(ord, values[slot]);
}
readerGen[slot] = currentReaderGen;
}
}
// Used per-segment when bit width is not a native array
// size (8, 16, 32):
final class AnyOrdComparator extends PerSegmentComparator {
private final IndexFieldData fieldData;
private final Ordinals.Docs readerOrds;
private final OrdinalsBytesValues termsIndex;
private final int docBase;
public AnyOrdComparator(IndexFieldData fieldData, OrdinalsBytesValues termsIndex, int docBase) {
this.fieldData = fieldData;
this.readerOrds = termsIndex.ordinals();
this.termsIndex = termsIndex;
this.docBase = docBase;
}
@Override
public int compareBottom(int doc) {
assert bottomSlot != -1;
final int docOrd = readerOrds.getOrd(doc);
if (bottomSameReader) {
// ord is precisely comparable, even in the equal case
return bottomOrd - docOrd;
} else if (bottomOrd >= docOrd) {
// the equals case always means bottom is > doc
// (because we set bottomOrd to the lower bound in
// setBottom):
return 1;
} else {
return -1;
}
}
@Override
public void copy(int slot, int doc) {
final int ord = readerOrds.getOrd(doc);
ords[slot] = ord;
if (ord == 0) {
values[slot] = null;
} else {
assert ord > 0;
if (values[slot] == null) {
values[slot] = new BytesRef();
}
termsIndex.getValueScratchByOrd(ord, values[slot]);
}
readerGen[slot] = currentReaderGen;
}
}
@Override
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
final int docBase = context.docBase;
termsIndex = indexFieldData.load(context).getBytesValues();
// TODO, we should support sorting on multi valued field, take the best ascending value out of all the values
if (termsIndex.isMultiValued()) {
throw new ElasticSearchIllegalArgumentException("can't sort on a multi valued field");
}
final Ordinals.Docs docToOrd = termsIndex.ordinals();
Object ordsStorage = docToOrd.ordinals().getBackingStorage();
FieldComparator<BytesRef> perSegComp = null;
if (ordsStorage instanceof byte[]) {
perSegComp = new ByteOrdComparator((byte[]) ordsStorage, termsIndex, docBase);
} else if (ordsStorage instanceof short[]) {
perSegComp = new ShortOrdComparator((short[]) ordsStorage, termsIndex, docBase);
} else if (ordsStorage instanceof int[]) {
perSegComp = new IntOrdComparator((int[]) ordsStorage, termsIndex, docBase);
}
// Don't specialize the long[] case since it's not
// possible, ie, worse case is MAX_INT-1 docs with
// every one having a unique value.
// TODO: ES - should we optimize for the PackedInts.Reader case as well?
if (perSegComp == null) {
perSegComp = new AnyOrdComparator(indexFieldData, termsIndex, docBase);
}
currentReaderGen++;
if (bottomSlot != -1) {
perSegComp.setBottom(bottomSlot);
}
return perSegComp;
}
@Override
public void setBottom(final int bottom) {
bottomSlot = bottom;
bottomValue = values[bottomSlot];
if (currentReaderGen == readerGen[bottomSlot]) {
bottomOrd = ords[bottomSlot];
bottomSameReader = true;
} else {
if (bottomValue == null) {
// 0 ord is null for all segments
assert ords[bottomSlot] == 0;
bottomOrd = 0;
bottomSameReader = true;
readerGen[bottomSlot] = currentReaderGen;
} else {
final int index = binarySearch(termsIndex, bottomValue);
if (index < 0) {
bottomOrd = -index - 2;
bottomSameReader = false;
} else {
bottomOrd = index;
// exact value match
bottomSameReader = true;
readerGen[bottomSlot] = currentReaderGen;
ords[bottomSlot] = bottomOrd;
}
}
}
}
@Override
public BytesRef value(int slot) {
return values[slot];
}
final protected static int binarySearch(OrdinalsBytesValues a, BytesRef key) {
return binarySearch(a, key, 1, a.ordinals().getNumDocs() - 1);
}
final protected static int binarySearch(OrdinalsBytesValues a, BytesRef key, int low, int high) {
while (low <= high) {
int mid = (low + high) >>> 1;
BytesRef midVal = a.getValueByOrd(mid);
int cmp;
if (midVal != null) {
cmp = midVal.compareTo(key);
} else {
cmp = -1;
}
if (cmp < 0)
low = mid + 1;
else if (cmp > 0)
high = mid - 1;
else
return mid;
}
return -(low + 1);
}
}

View File

@ -17,38 +17,32 @@
* under the License.
*/
package org.elasticsearch.index.field.data.strings;
package org.elasticsearch.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.cache.field.data.FieldDataCache;
import org.elasticsearch.index.field.data.FieldData;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.fielddata.BytesValues;
import org.elasticsearch.index.fielddata.IndexFieldData;
import java.io.IOException;
/**
*
* Sorts by field's natural Term sort order. All
* comparisons are done using BytesRef.compareTo, which is
* slow for medium to large result sets but possibly
* very fast for very small results sets.
*/
// LUCENE MONITOR: Monitor against FieldComparator#String
public class StringValFieldDataComparator extends FieldComparator<BytesRef> {
private final String fieldName;
protected final FieldDataCache fieldDataCache;
protected FieldData currentFieldData;
public final class BytesRefValComparator extends FieldComparator<BytesRef> {
private final IndexFieldData indexFieldData;
private BytesRef[] values;
private BytesValues docTerms;
private BytesRef bottom;
public StringValFieldDataComparator(int numHits, String fieldName, FieldDataCache fieldDataCache) {
this.fieldName = fieldName;
this.fieldDataCache = fieldDataCache;
BytesRefValComparator(IndexFieldData indexFieldData, int numHits) {
values = new BytesRef[numHits];
this.indexFieldData = indexFieldData;
}
@Override
@ -69,7 +63,7 @@ public class StringValFieldDataComparator extends FieldComparator<BytesRef> {
@Override
public int compareBottom(int doc) {
final BytesRef val2 = currentFieldData.stringValue(doc);
BytesRef val2 = docTerms.getValue(doc);
if (bottom == null) {
if (val2 == null) {
return 0;
@ -83,29 +77,18 @@ public class StringValFieldDataComparator extends FieldComparator<BytesRef> {
@Override
public void copy(int slot, int doc) {
values[slot] = currentFieldData.stringValue(doc);
if (values[slot] == null) {
values[slot] = new BytesRef();
}
docTerms.getValueScratch(doc, values[slot]);
}
@Override
public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
currentFieldData = fieldDataCache.cache(FieldDataType.DefaultTypes.STRING, context.reader(), fieldName);
docTerms = indexFieldData.load(context).getBytesValues();
return this;
}
@Override
public int compareDocToValue(int doc, BytesRef val2) throws IOException {
BytesRef val1 = currentFieldData.stringValue(doc);
if (val1 == null) {
if (val2 == null) {
return 0;
}
return -1;
} else if (val2 == null) {
return 1;
}
return currentFieldData.stringValue(doc).compareTo(val2);
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
@ -115,4 +98,22 @@ public class StringValFieldDataComparator extends FieldComparator<BytesRef> {
public BytesRef value(int slot) {
return values[slot];
}
@Override
public int compareValues(BytesRef val1, BytesRef val2) {
if (val1 == null) {
if (val2 == null) {
return 0;
}
return -1;
} else if (val2 == null) {
return 1;
}
return val1.compareTo(val2);
}
@Override
public int compareDocToValue(int doc, BytesRef value) {
return docTerms.getValue(doc).compareTo(value);
}
}

View File

@ -17,13 +17,13 @@
* under the License.
*/
package org.elasticsearch.index.field.function.sort;
package org.elasticsearch.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SortField;
import org.elasticsearch.index.field.data.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.script.SearchScript;
import java.io.IOException;
@ -32,13 +32,13 @@ import java.io.IOException;
*
*/
// LUCENE MONITOR: Monitor against FieldComparator.Double
public class DoubleFieldsFunctionDataComparator extends FieldComparator<Double> {
public class DoubleScriptDataComparator extends FieldComparator<Double> {
public static FieldDataType.ExtendedFieldComparatorSource comparatorSource(SearchScript script) {
public static IndexFieldData.XFieldComparatorSource comparatorSource(SearchScript script) {
return new InnerSource(script);
}
private static class InnerSource extends FieldDataType.ExtendedFieldComparatorSource {
private static class InnerSource extends IndexFieldData.XFieldComparatorSource {
private final SearchScript script;
@ -48,7 +48,7 @@ public class DoubleFieldsFunctionDataComparator extends FieldComparator<Double>
@Override
public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
return new DoubleFieldsFunctionDataComparator(numHits, script);
return new DoubleScriptDataComparator(numHits, script);
}
@Override
@ -62,7 +62,7 @@ public class DoubleFieldsFunctionDataComparator extends FieldComparator<Double>
private final double[] values;
private double bottom;
public DoubleFieldsFunctionDataComparator(int numHits, SearchScript script) {
public DoubleScriptDataComparator(int numHits, SearchScript script) {
this.script = script;
values = new double[numHits];
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.elasticsearch.index.fielddata.DoubleValues;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class DoubleValuesComparator extends FieldComparator<Double> {
private final IndexNumericFieldData indexFieldData;
private final double missingValue;
protected final double[] values;
private double bottom;
private DoubleValues readerValues;
public DoubleValuesComparator(IndexNumericFieldData indexFieldData, double missingValue, int numHits) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
this.values = new double[numHits];
}
@Override
public int compare(int slot1, int slot2) {
final double v1 = values[slot1];
final double v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void setBottom(int slot) {
this.bottom = values[slot];
}
@Override
public int compareBottom(int doc) throws IOException {
double v2 = readerValues.getValueMissing(doc, missingValue);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void copy(int slot, int doc) throws IOException {
values[slot] = readerValues.getValueMissing(doc, missingValue);
}
@Override
public FieldComparator<Double> setNextReader(AtomicReaderContext context) throws IOException {
this.readerValues = indexFieldData.load(context).getDoubleValues();
return this;
}
@Override
public Double value(int slot) {
return Double.valueOf(values[slot]);
}
@Override
public int compareDocToValue(int doc, Double valueObj) throws IOException {
final double value = valueObj.doubleValue();
double docValue = readerValues.getValueMissing(doc, missingValue);
if (docValue < value) {
return -1;
} else if (docValue > value) {
return 1;
} else {
return 0;
}
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class DoubleValuesComparatorSource extends IndexFieldData.XFieldComparatorSource {
private final IndexNumericFieldData indexFieldData;
private final Object missingValue;
public DoubleValuesComparatorSource(IndexNumericFieldData indexFieldData, @Nullable Object missingValue) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert fieldname.equals(indexFieldData.getFieldNames().indexName());
double dMissingValue;
if (missingValue == null || "_last".equals(missingValue)) {
dMissingValue = reversed ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY;
} else if ("_first".equals(missingValue)) {
dMissingValue = reversed ? Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
} else {
dMissingValue = missingValue instanceof Number ? ((Number) missingValue).doubleValue() : Double.parseDouble(missingValue.toString());
}
return new DoubleValuesComparator(indexFieldData, dMissingValue, numHits);
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.elasticsearch.index.fielddata.FloatValues;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class FloatValuesComparator extends FieldComparator<Float> {
private final IndexNumericFieldData indexFieldData;
private final float missingValue;
protected final float[] values;
private float bottom;
private FloatValues readerValues;
public FloatValuesComparator(IndexNumericFieldData indexFieldData, float missingValue, int numHits) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
this.values = new float[numHits];
}
@Override
public int compare(int slot1, int slot2) {
final float v1 = values[slot1];
final float v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void setBottom(int slot) {
this.bottom = values[slot];
}
@Override
public int compareBottom(int doc) throws IOException {
float v2 = readerValues.getValueMissing(doc, missingValue);
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public void copy(int slot, int doc) throws IOException {
values[slot] = readerValues.getValueMissing(doc, missingValue);
}
@Override
public FieldComparator<Float> setNextReader(AtomicReaderContext context) throws IOException {
this.readerValues = indexFieldData.load(context).getFloatValues();
return this;
}
@Override
public Float value(int slot) {
return Float.valueOf(values[slot]);
}
@Override
public int compareDocToValue(int doc, Float valueObj) throws IOException {
final float value = valueObj.floatValue();
float docValue = readerValues.getValueMissing(doc, missingValue);
if (docValue < value) {
return -1;
} else if (docValue > value) {
return 1;
} else {
return 0;
}
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import java.io.IOException;
/**
*/
public class FloatValuesComparatorSource extends IndexFieldData.XFieldComparatorSource {
private final IndexNumericFieldData indexFieldData;
private final Object missingValue;
public FloatValuesComparatorSource(IndexNumericFieldData indexFieldData, @Nullable Object missingValue) {
this.indexFieldData = indexFieldData;
this.missingValue = missingValue;
}
@Override
public SortField.Type reducedType() {
return SortField.Type.FLOAT;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert fieldname.equals(indexFieldData.getFieldNames().indexName());
float dMissingValue;
if (missingValue == null || "_last".equals(missingValue)) {
dMissingValue = reversed ? Float.NEGATIVE_INFINITY : Float.POSITIVE_INFINITY;
} else if ("_first".equals(missingValue)) {
dMissingValue = reversed ? Float.POSITIVE_INFINITY : Float.NEGATIVE_INFINITY;
} else {
dMissingValue = missingValue instanceof Number ? ((Number) missingValue).floatValue() : Float.parseFloat(missingValue.toString());
}
return new FloatValuesComparator(indexFieldData, dMissingValue, numHits);
}
}

View File

@ -0,0 +1,133 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.search.FieldComparator;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.GeoPointValues;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.geo.GeoPoint;
import org.elasticsearch.index.search.geo.GeoDistance;
import java.io.IOException;
/**
*/
public class GeoDistanceComparator extends FieldComparator<Double> {
protected final IndexGeoPointFieldData indexFieldData;
protected final double lat;
protected final double lon;
protected final DistanceUnit unit;
protected final GeoDistance geoDistance;
protected final GeoDistance.FixedSourceDistance fixedSourceDistance;
private final double[] values;
private double bottom;
private GeoPointValues readerValues;
public GeoDistanceComparator(int numHits, IndexGeoPointFieldData indexFieldData, double lat, double lon, DistanceUnit unit, GeoDistance geoDistance) {
this.values = new double[numHits];
this.indexFieldData = indexFieldData;
this.lat = lat;
this.lon = lon;
this.unit = unit;
this.geoDistance = geoDistance;
this.fixedSourceDistance = geoDistance.fixedSourceDistance(lat, lon, unit);
}
@Override
public FieldComparator<Double> setNextReader(AtomicReaderContext context) throws IOException {
this.readerValues = indexFieldData.load(context).getGeoPointValues();
return this;
}
@Override
public int compare(int slot1, int slot2) {
final double v1 = values[slot1];
final double v2 = values[slot2];
if (v1 > v2) {
return 1;
} else if (v1 < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareBottom(int doc) {
double distance;
GeoPoint geoPoint = readerValues.getValue(doc);
if (geoPoint == null) {
// is this true? push this to the "end"
distance = Double.MAX_VALUE;
} else {
distance = fixedSourceDistance.calculate(geoPoint.lat(), geoPoint.lon());
}
final double v2 = distance;
if (bottom > v2) {
return 1;
} else if (bottom < v2) {
return -1;
} else {
return 0;
}
}
@Override
public int compareDocToValue(int doc, Double distance2) throws IOException {
double distance1;
GeoPoint geoPoint = readerValues.getValue(doc);
if (geoPoint == null) {
// is this true? push this to the "end"
distance1 = Double.MAX_VALUE;
} else {
distance1 = fixedSourceDistance.calculate(geoPoint.lat(), geoPoint.lon());
}
return (int) (distance1 - distance2);
}
@Override
public void copy(int slot, int doc) {
double distance;
GeoPoint geoPoint = readerValues.getValue(doc);
if (geoPoint == null) {
// is this true? push this to the "end"
distance = Double.MAX_VALUE;
} else {
distance = fixedSourceDistance.calculate(geoPoint.lat(), geoPoint.lon());
}
values[slot] = distance;
}
@Override
public void setBottom(final int bottom) {
this.bottom = values[bottom];
}
@Override
public Double value(int slot) {
return values[slot];
}
}

View File

@ -0,0 +1,60 @@
/*
* Licensed to ElasticSearch and Shay Banon 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.index.fielddata.fieldcomparator;
import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.search.geo.GeoDistance;
import java.io.IOException;
/**
*/
public class GeoDistanceComparatorSource extends IndexFieldData.XFieldComparatorSource {
private final IndexGeoPointFieldData indexFieldData;
private final double lat;
private final double lon;
private final DistanceUnit unit;
private final GeoDistance geoDistance;
public GeoDistanceComparatorSource(IndexGeoPointFieldData indexFieldData, double lat, double lon, DistanceUnit unit, GeoDistance geoDistance) {
this.indexFieldData = indexFieldData;
this.lat = lat;
this.lon = lon;
this.unit = unit;
this.geoDistance = geoDistance;
}
@Override
public SortField.Type reducedType() {
return SortField.Type.DOUBLE;
}
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert indexFieldData.getFieldNames().indexName().equals(fieldname);
// TODO support multi value?
return new GeoDistanceComparator(numHits, indexFieldData, lat, lon, unit, geoDistance);
}
}

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