Add doc values support to boolean fields.

This pull request makes boolean handled like dates and ipv4 addresses: things
are stored as as numerics under the hood and aggregations add some special
formatting logic in order to return true/false in addition to 1/0.

For example, here is an output of a terms aggregation on a boolean field:
```
   "aggregations": {
      "top_f": {
         "doc_count_error_upper_bound": 0,
         "buckets": [
            {
               "key": 0,
               "key_as_string": "false",
               "doc_count": 2
            },
            {
               "key": 1,
               "key_as_string": "true",
               "doc_count": 1
            }
         ]
      }
   }
```

Sorted numeric doc values are used under the hood.

Close #4678
Close #7851
This commit is contained in:
Adrien Grand 2014-10-02 16:16:54 +02:00
parent e390ef5bc1
commit 08f93cf33f
99 changed files with 496 additions and 19 deletions

View File

@ -61,7 +61,8 @@ def index_documents(es, index_name, type, num_docs):
for id in range(0, num_docs):
es.index(index=index_name, doc_type=type, id=id, body={'string': str(random.randint(0, 100)),
'long_sort': random.randint(0, 100),
'double_sort' : float(random.randint(0, 100))})
'double_sort' : float(random.randint(0, 100)),
'bool' : random.choice([True, False])})
if rarely():
es.indices.refresh(index=index_name)
if rarely():

View File

@ -273,6 +273,38 @@ to provide special features. They now have limited configuration options.
* `_field_names` configuration is limited to disabling the field.
* `_size` configuration is limited to enabling the field.
=== Boolean fields
Boolean fields used to have a string fielddata with `F` meaning `false` and `T`
meaning `true`. They have been refactored to use numeric fielddata, with `0`
for `false` and `1` for `true`. As a consequence, the format of the responses of
the following APIs changed when applied to boolean fields: `0`/`1` is returned
instead of `F`/`T`:
- <<search-request-fielddata-fields,fielddata fields>>
- <<search-request-sort,sort values>>
- <<search-aggregations-bucket-terms-aggregation,terms aggregations>>
In addition, terms aggregations use a custom formatter for boolean (like for
dates and ip addresses, which are also backed by numbers) in order to return
the user-friendly representation of boolean fields: `false`/`true`:
[source,json]
---------------
"buckets": [
{
"key": 0,
"key_as_string": "false",
"doc_count": 42
},
{
"key": 1,
"key_as_string": "true",
"doc_count": 12
}
]
---------------
=== Codecs
It is no longer possible to specify per-field postings and doc values formats

View File

@ -646,6 +646,25 @@ public final class XContentBuilder implements BytesStream, Releasable {
return this;
}
public XContentBuilder field(String name, boolean... value) throws IOException {
startArray(name);
for (boolean o : value) {
value(o);
}
endArray();
return this;
}
public XContentBuilder field(XContentBuilderString name, boolean... value) throws IOException {
startArray(name);
for (boolean o : value) {
value(o);
}
endArray();
return this;
}
public XContentBuilder field(String name, String... value) throws IOException {
startArray(name);
for (String o : value) {

View File

@ -33,6 +33,7 @@ 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.mapper.core.BooleanFieldMapper;
import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.IndexService;
@ -78,6 +79,7 @@ public class IndexFieldDataService extends AbstractIndexComponent {
.put(ParentFieldMapper.NAME, new ParentChildIndexFieldData.Builder())
.put(IndexFieldMapper.NAME, new IndexIndexFieldData.Builder())
.put("binary", new DisabledIndexFieldData.Builder())
.put(BooleanFieldMapper.CONTENT_TYPE, new PackedArrayIndexFieldData.Builder().setNumericType(IndexNumericFieldData.NumericType.BOOLEAN))
.immutableMap();
docValuesBuildersByType = MapBuilder.<String, IndexFieldData.Builder>newMapBuilder()
@ -90,6 +92,7 @@ public class IndexFieldDataService extends AbstractIndexComponent {
.put("long", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.LONG))
.put("geo_point", new GeoPointBinaryDVIndexFieldData.Builder())
.put("binary", new BytesBinaryDVIndexFieldData.Builder())
.put(BooleanFieldMapper.CONTENT_TYPE, new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BOOLEAN))
.immutableMap();
buildersByTypeAndFormat = MapBuilder.<Tuple<String, String>, IndexFieldData.Builder>newMapBuilder()
@ -130,6 +133,10 @@ public class IndexFieldDataService extends AbstractIndexComponent {
.put(Tuple.tuple("binary", DOC_VALUES_FORMAT), new BytesBinaryDVIndexFieldData.Builder())
.put(Tuple.tuple("binary", DISABLED_FORMAT), new DisabledIndexFieldData.Builder())
.put(Tuple.tuple(BooleanFieldMapper.CONTENT_TYPE, ARRAY_FORMAT), new PackedArrayIndexFieldData.Builder().setNumericType(IndexNumericFieldData.NumericType.BOOLEAN))
.put(Tuple.tuple(BooleanFieldMapper.CONTENT_TYPE, DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BOOLEAN))
.put(Tuple.tuple(BooleanFieldMapper.CONTENT_TYPE, DISABLED_FORMAT), new DisabledIndexFieldData.Builder())
.immutableMap();
}

View File

@ -24,13 +24,38 @@ import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
/**
*/
public interface IndexNumericFieldData extends IndexFieldData<AtomicNumericFieldData> {
public static enum NumericType {
BOOLEAN(1, false, SortField.Type.INT, 0, 1) {
@Override
public long toLong(BytesRef indexForm) {
if (indexForm.equals(BooleanFieldMapper.Values.FALSE)) {
return 0;
} else if (indexForm.equals(BooleanFieldMapper.Values.TRUE)) {
return 1;
} else {
throw new ElasticsearchIllegalArgumentException("Cannot convert " + indexForm + " to a boolean");
}
}
@Override
public void toIndexForm(Number number, BytesRefBuilder bytes) {
bytes.append(number.intValue() != 0 ? BooleanFieldMapper.Values.TRUE : BooleanFieldMapper.Values.FALSE);
}
@Override
public Number toNumber(BytesRef indexForm) {
return toLong(indexForm);
}
},
BYTE(8, false, SortField.Type.INT, Byte.MIN_VALUE, Byte.MAX_VALUE) {
@Override
public long toLong(BytesRef indexForm) {
@ -174,7 +199,9 @@ public interface IndexNumericFieldData extends IndexFieldData<AtomicNumericField
public abstract Number toNumber(BytesRef indexForm);
public final TermsEnum wrapTermsEnum(TermsEnum termsEnum) {
if (requiredBits() > 32) {
if (requiredBits() == 1) { // boolean, no prefix-terms
return termsEnum;
} else if (requiredBits() > 32) {
return OrdinalsBuilder.wrapNumeric64Bit(termsEnum);
} else {
return OrdinalsBuilder.wrapNumeric32Bit(termsEnum);

View File

@ -99,7 +99,7 @@ public class PackedArrayIndexFieldData extends AbstractIndexFieldData<AtomicNume
CircuitBreakerService breakerService) {
super(index, indexSettings, fieldNames, fieldDataType, cache);
Preconditions.checkNotNull(numericType);
Preconditions.checkArgument(EnumSet.of(NumericType.BYTE, NumericType.SHORT, NumericType.INT, NumericType.LONG).contains(numericType), getClass().getSimpleName() + " only supports integer types, not " + numericType);
Preconditions.checkArgument(EnumSet.of(NumericType.BOOLEAN, NumericType.BYTE, NumericType.SHORT, NumericType.INT, NumericType.LONG).contains(numericType), getClass().getSimpleName() + " only supports integer types, not " + numericType);
this.numericType = numericType;
this.breakerService = breakerService;
}
@ -127,16 +127,13 @@ public class PackedArrayIndexFieldData extends AbstractIndexFieldData<AtomicNume
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
TermsEnum termsEnum = estimator.beforeLoad(terms);
assert !getNumericType().isFloatingPoint();
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(-1, reader.maxDoc(), acceptableTransientOverheadRatio)) {
BytesRefIterator iter = builder.buildFromTerms(termsEnum);
BytesRef term;
assert !getNumericType().isFloatingPoint();
final boolean indexedAsLong = getNumericType().requiredBits() > 32;
while ((term = iter.next()) != null) {
final long value = indexedAsLong
? NumericUtils.prefixCodedToLong(term)
: NumericUtils.prefixCodedToInt(term);
final long value = numericType.toLong(term);
valuesBuilder.add(value);
}
final PackedLongValues values = valuesBuilder.build();

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.mapper.core;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.queries.TermFilter;
import org.apache.lucene.search.Filter;
@ -51,9 +52,8 @@ import static org.elasticsearch.index.mapper.MapperBuilders.booleanField;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseField;
/**
*
* A field mapper for boolean fields.
*/
// TODO this can be made better, maybe storing a byte for it?
public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
public static final String CONTENT_TYPE = "boolean";
@ -100,7 +100,7 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
@Override
public BooleanFieldMapper build(BuilderContext context) {
return new BooleanFieldMapper(buildNames(context), boost, fieldType, nullValue,
return new BooleanFieldMapper(buildNames(context), boost, fieldType, docValues, nullValue,
similarity, normsLoading, fieldDataSettings, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
}
@ -128,10 +128,10 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
private Boolean nullValue;
protected BooleanFieldMapper(Names names, float boost, FieldType fieldType, Boolean nullValue,
protected BooleanFieldMapper(Names names, float boost, FieldType fieldType, Boolean docValues, Boolean nullValue,
SimilarityProvider similarity, Loading normsLoading,
@Nullable Settings fieldDataSettings, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(names, boost, fieldType, false, Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, similarity, normsLoading, fieldDataSettings, indexSettings, multiFields, copyTo);
super(names, boost, fieldType, docValues, Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, similarity, normsLoading, fieldDataSettings, indexSettings, multiFields, copyTo);
this.nullValue = nullValue;
}
@ -143,7 +143,7 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
@Override
public FieldDataType defaultFieldDataType() {
// TODO have a special boolean type?
return new FieldDataType("string");
return new FieldDataType(CONTENT_TYPE);
}
@Override
@ -210,7 +210,7 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
@Override
protected void parseCreateField(ParseContext context, List<Field> fields) throws IOException {
if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored()) {
if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored() && !hasDocValues()) {
return;
}
@ -230,6 +230,9 @@ public class BooleanFieldMapper extends AbstractFieldMapper<Boolean> {
return;
}
fields.add(new Field(names.indexName(), value ? "T" : "F", fieldType));
if (hasDocValues()) {
fields.add(new SortedNumericDocValuesField(names.indexName(), value ? 1 : 0));
}
}
@Override

View File

@ -25,6 +25,7 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.mapper.ip.IpFieldMapper;
@ -209,6 +210,9 @@ public class ValuesSourceParser<VS extends ValuesSource> {
if (mapper instanceof IpFieldMapper) {
return ValueFormat.IPv4;
}
if (mapper instanceof BooleanFieldMapper) {
return ValueFormat.BOOLEAN;
}
if (mapper instanceof NumberFieldMapper) {
return format != null ? ValueFormat.Number.format(format) : ValueFormat.RAW;
}

View File

@ -28,6 +28,7 @@ public class ValueFormat {
public static final ValueFormat RAW = new ValueFormat(ValueFormatter.RAW, ValueParser.RAW);
public static final ValueFormat IPv4 = new ValueFormat(ValueFormatter.IPv4, ValueParser.IPv4);
public static final ValueFormat BOOLEAN = new ValueFormat(ValueFormatter.BOOLEAN, ValueParser.BOOLEAN);
private final ValueFormatter formatter;
private final ValueParser parser;

View File

@ -19,8 +19,6 @@
package org.elasticsearch.search.aggregations.support.format;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -45,6 +43,7 @@ public interface ValueFormatter extends Streamable {
public final static ValueFormatter RAW = new Raw();
public final static ValueFormatter IPv4 = new IPv4Formatter();
public final static ValueFormatter GEOHASH = new GeoHash();
public final static ValueFormatter BOOLEAN = new BooleanFormatter();
/**
* Uniquely identifies this formatter (used for efficient serialization)
@ -266,4 +265,31 @@ public interface ValueFormatter extends Streamable {
}
}
static class BooleanFormatter implements ValueFormatter {
static final byte ID = 10;
@Override
public byte id() {
return ID;
}
@Override
public String format(long value) {
return Boolean.valueOf(value != 0).toString();
}
@Override
public String format(double value) {
return Boolean.valueOf(value != 0).toString();
}
@Override
public void readFrom(StreamInput in) throws IOException {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
}
}
}

View File

@ -38,6 +38,7 @@ public class ValueFormatterStreams {
case ValueFormatter.DateTime.ID: formatter = new ValueFormatter.DateTime(); break;
case ValueFormatter.Number.Pattern.ID: formatter = new ValueFormatter.Number.Pattern(); break;
case ValueFormatter.GeoHash.ID: formatter = ValueFormatter.GEOHASH; break;
case ValueFormatter.BooleanFormatter.ID: formatter = ValueFormatter.BOOLEAN; break;
default: throw new ElasticsearchIllegalArgumentException("Unknown value formatter with id [" + id + "]");
}
formatter.readFrom(in);

View File

@ -41,6 +41,7 @@ public interface ValueParser {
static final ValueParser IPv4 = new IPv4();
static final ValueParser RAW = new Raw();
static final ValueParser BOOLEAN = new Boolean();
long parseLong(String value, SearchContext searchContext);
@ -184,4 +185,20 @@ public interface ValueParser {
}
}
static class Boolean implements ValueParser {
private Boolean() {
}
@Override
public long parseLong(String value, SearchContext searchContext) {
return java.lang.Boolean.parseBoolean(value) ? 1 : 0;
}
@Override
public double parseDouble(String value, SearchContext searchContext) {
return parseLong(value, searchContext);
}
}
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.bwcompat;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
import org.apache.lucene.util.TimeUnits;
import org.elasticsearch.Version;
import org.elasticsearch.action.get.GetResponse;
@ -34,6 +35,9 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.node.Node;
import org.elasticsearch.rest.action.admin.indices.upgrade.UpgradeTest;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
import org.elasticsearch.test.index.merge.NoMergePolicyProvider;
@ -47,7 +51,12 @@ import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.*;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.CoreMatchers.equalTo;
@ -114,6 +123,7 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
loadIndex(index, settings);
logMemoryStats();
assertBasicSearchWorks();
assertBasicAggregationWorks();
assertRealtimeGetWorks();
assertNewReplicasWork();
Version version = extractVersion(index);
@ -147,6 +157,28 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
assertThat(numDocs, equalTo(searchRsp.getHits().getTotalHits()));
}
void assertBasicAggregationWorks() {
// histogram on a long
SearchResponse searchRsp = client().prepareSearch("test").addAggregation(AggregationBuilders.histogram("histo").field("long_sort").interval(10)).get();
ElasticsearchAssertions.assertSearchResponse(searchRsp);
Histogram histo = searchRsp.getAggregations().get("histo");
assertNotNull(histo);
long totalCount = 0;
for (Histogram.Bucket bucket : histo.getBuckets()) {
totalCount += bucket.getDocCount();
}
assertEquals(totalCount, searchRsp.getHits().getTotalHits());
// terms on a boolean
searchRsp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("bool_terms").field("bool")).get();
Terms terms = searchRsp.getAggregations().get("bool_terms");
totalCount = 0;
for (Terms.Bucket bucket : terms.getBuckets()) {
totalCount += bucket.getDocCount();
}
assertEquals(totalCount, searchRsp.getHits().getTotalHits());
}
void assertRealtimeGetWorks() {
assertAcked(client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder()
.put("refresh_interval", -1)

View File

@ -0,0 +1,129 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.core;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.FieldMapperListener;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import org.junit.Before;
import java.io.IOException;
public class BooleanFieldMapperTests extends ElasticsearchSingleNodeTest {
IndexService indexService;
DocumentMapperParser parser;
@Before
public void before() {
indexService = createIndex("test");
parser = indexService.mapperService().documentMapperParser();
}
public void testDefaults() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "boolean").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = parser.parse(mapping);
ParsedDocument doc = defaultMapper.parse("type", "1", XContentFactory.jsonBuilder()
.startObject()
.field("field", true)
.endObject()
.bytes());
try (Directory dir = new RAMDirectory();
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(getRandom())))) {
w.addDocuments(doc.docs());
try (DirectoryReader reader = DirectoryReader.open(w, true)) {
final LeafReader leaf = reader.leaves().get(0).reader();
// boolean fields are indexed and have doc values by default
assertEquals(new BytesRef("T"), leaf.terms("field").iterator(null).next());
SortedNumericDocValues values = leaf.getSortedNumericDocValues("field");
assertNotNull(values);
values.setDocument(0);
assertEquals(1, values.count());
assertEquals(1, values.valueAt(0));
}
}
}
public void testSerialization() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field").field("type", "boolean").endObject().endObject()
.endObject().endObject().string();
DocumentMapper defaultMapper = parser.parse(mapping);
final FieldMapper<?>[] mapper = new BooleanFieldMapper[1];
defaultMapper.root().traverse(new FieldMapperListener() {
@Override
public void fieldMapper(FieldMapper<?> fieldMapper) {
if (fieldMapper.name().equals("field")) {
mapper[0] = fieldMapper;
}
}
});
XContentBuilder builder = XContentFactory.jsonBuilder().startObject();
mapper[0].toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
assertEquals("{\"field\":{\"type\":\"boolean\"}}", builder.string());
// now change some parameters
mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "boolean")
.field("doc_values", "false")
.field("null_value", true)
.endObject().endObject()
.endObject().endObject().string();
defaultMapper = parser.parse(mapping);
defaultMapper.root().traverse(new FieldMapperListener() {
@Override
public void fieldMapper(FieldMapper<?> fieldMapper) {
if (fieldMapper.name().equals("field")) {
mapper[0] = fieldMapper;
}
}
});
builder = XContentFactory.jsonBuilder().startObject();
mapper[0].toXContent(builder, ToXContent.EMPTY_PARAMS);
builder.endObject();
assertEquals("{\"field\":{\"type\":\"boolean\",\"doc_values\":false,\"null_value\":true}}", builder.string());
}
}

View File

@ -0,0 +1,170 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.core.IsNull.notNullValue;
/**
*
*/
@ElasticsearchIntegrationTest.SuiteScopeTest
public class BooleanTermsTests extends ElasticsearchIntegrationTest {
private static final String SINGLE_VALUED_FIELD_NAME = "b_value";
private static final String MULTI_VALUED_FIELD_NAME = "b_values";
static int numSingleTrues, numSingleFalses, numMultiTrues, numMultiFalses;
@Override
public void setupSuiteScopeCluster() throws Exception {
createIndex("idx");
createIndex("idx_unmapped");
ensureSearchable();
final int numDocs = randomInt(5);
IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs];
for (int i = 0; i < builders.length; i++) {
final boolean singleValue = randomBoolean();
if (singleValue) {
numSingleTrues++;
} else {
numSingleFalses++;
}
final boolean[] multiValue;
switch (randomInt(3)) {
case 0:
multiValue = new boolean[0];
break;
case 1:
numMultiFalses++;
multiValue = new boolean[] {false};
break;
case 2:
numMultiTrues++;
multiValue = new boolean[] {true};
break;
case 3:
numMultiFalses++; numMultiTrues++;
multiValue = new boolean[] {false, true};
break;
default:
throw new AssertionError();
}
builders[i] = client().prepareIndex("idx", "type").setSource(jsonBuilder()
.startObject()
.field(SINGLE_VALUED_FIELD_NAME, singleValue)
.field(MULTI_VALUED_FIELD_NAME, multiValue)
.endObject());
}
indexRandom(true, builders);
}
@Test
public void singleValueField() throws Exception {
SearchResponse response = client().prepareSearch("idx").setTypes("type")
.addAggregation(terms("terms")
.field(SINGLE_VALUED_FIELD_NAME)
.collectMode(randomFrom(SubAggCollectionMode.values())))
.execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
final int bucketCount = numSingleFalses > 0 && numSingleTrues > 0 ? 2 : numSingleFalses + numSingleTrues > 0 ? 1 : 0;
assertThat(terms.getBuckets().size(), equalTo(bucketCount));
Terms.Bucket bucket = terms.getBucketByKey("0");
if (numSingleFalses == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numSingleFalses, bucket.getDocCount());
}
bucket = terms.getBucketByKey("1");
if (numSingleTrues == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numSingleTrues, bucket.getDocCount());
}
}
@Test
public void multiValueField() throws Exception {
SearchResponse response = client().prepareSearch("idx").setTypes("type")
.addAggregation(terms("terms")
.field(MULTI_VALUED_FIELD_NAME)
.collectMode(randomFrom(SubAggCollectionMode.values())))
.execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
final int bucketCount = numMultiFalses > 0 && numMultiTrues > 0 ? 2 : numMultiFalses + numMultiTrues > 0 ? 1 : 0;
assertThat(terms.getBuckets().size(), equalTo(bucketCount));
Terms.Bucket bucket = terms.getBucketByKey("0");
if (numMultiFalses == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numMultiFalses, bucket.getDocCount());
}
bucket = terms.getBucketByKey("1");
if (numMultiTrues == 0) {
assertNull(bucket);
} else {
assertNotNull(bucket);
assertEquals(numMultiTrues, bucket.getDocCount());
}
}
@Test
public void unmapped() throws Exception {
SearchResponse response = client().prepareSearch("idx_unmapped").setTypes("type")
.addAggregation(terms("terms")
.field(SINGLE_VALUED_FIELD_NAME)
.size(randomInt(5))
.collectMode(randomFrom(SubAggCollectionMode.values())))
.execute().actionGet();
assertSearchResponse(response);
Terms terms = response.getAggregations().get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
assertThat(terms.getBuckets().size(), equalTo(0));
}
}

View File

@ -569,7 +569,7 @@ public class SearchFieldsTests extends ElasticsearchIntegrationTest {
assertThat(searchResponse.getHits().getAt(0).fields().get("float_field").value(), equalTo((Object) 5.0));
assertThat(searchResponse.getHits().getAt(0).fields().get("double_field").value(), equalTo((Object) 6.0d));
assertThat(searchResponse.getHits().getAt(0).fields().get("date_field").value(), equalTo((Object) 1332374400000L));
assertThat(searchResponse.getHits().getAt(0).fields().get("boolean_field").value().toString(), equalTo("T"));
assertThat(searchResponse.getHits().getAt(0).fields().get("boolean_field").value(), equalTo((Object) 1L));
}

View File

@ -380,6 +380,17 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
.startObject("mapping")
.field("doc_values", randomBoolean())
.startObject("fielddata")
.field(Loading.KEY, randomFrom(Loading.LAZY, Loading.EAGER))
.endObject()
.endObject()
.endObject()
.endObject()
.startObject()
.startObject("template-booleans")
.field("match_mapping_type", "boolean")
.startObject("mapping")
.startObject("fielddata")
.field(FieldDataType.FORMAT_KEY, randomFrom("array", "doc_values"))
.field(Loading.KEY, randomFrom(Loading.LAZY, Loading.EAGER))
.endObject()
.endObject()