Jelajahi Sumber

Add doc value for binary field.

Close #5669
Kevin Wang 11 tahun lalu
induk
melakukan
866c520abb

+ 1 - 0
docs/reference/mapping/types/core-types.asciidoc

@@ -446,6 +446,7 @@ Defaults to the property/field name.
 |`store` |Set to `true` to store actual field in the index, `false` to not
 store it. Defaults to `false` (note, the JSON document itself is stored,
 and it can be retrieved from it).
+|`doc_values` |Set to `true` to store field values in a column-stride fashion.
 |=======================================================================
 
 [float]

+ 57 - 0
src/main/java/org/elasticsearch/common/util/CollectionUtils.java

@@ -22,10 +22,12 @@ package org.elasticsearch.common.util;
 import com.carrotsearch.hppc.DoubleArrayList;
 import com.carrotsearch.hppc.FloatArrayList;
 import com.carrotsearch.hppc.LongArrayList;
+import com.carrotsearch.hppc.ObjectArrayList;
 import org.apache.lucene.util.IntroSorter;
 import org.elasticsearch.common.Preconditions;
 
 import java.util.AbstractList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.RandomAccess;
 
@@ -223,6 +225,61 @@ public enum CollectionUtils {
         return new RotatedList<>(list, d);
     }
 
+    public static void sortAndDedup(final ObjectArrayList<byte[]> array) {
+        int len = array.size();
+        if (len > 1) {
+            sort(array);
+            int uniqueCount = 1;
+            for (int i = 1; i < len; ++i) {
+                if (!Arrays.equals(array.get(i), array.get(i - 1))) {
+                    array.set(uniqueCount++, array.get(i));
+                }
+            }
+            array.elementsCount = uniqueCount;
+        }
+    }
+
+    public static void sort(final ObjectArrayList<byte[]> array) {
+        new IntroSorter() {
+
+            byte[] pivot;
+
+            @Override
+            protected void swap(int i, int j) {
+                final byte[] tmp = array.get(i);
+                array.set(i, array.get(j));
+                array.set(j, tmp);
+            }
+
+            @Override
+            protected int compare(int i, int j) {
+                return compare(array.get(i), array.get(j));
+            }
+
+            @Override
+            protected void setPivot(int i) {
+                pivot = array.get(i);
+            }
+
+            @Override
+            protected int comparePivot(int j) {
+                return compare(pivot, array.get(j));
+            }
+
+            private int compare(byte[] left, byte[] right) {
+                for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) {
+                    int a = left[i] & 0xFF;
+                    int b = right[j] & 0xFF;
+                    if (a != b) {
+                        return a - b;
+                    }
+                }
+                return left.length - right.length;
+            }
+
+        }.sort(0, array.size());
+    }
+
     private static class RotatedList<T> extends AbstractList<T> implements RandomAccess {
 
         private final List<T> in;

+ 5 - 0
src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataService.java

@@ -73,6 +73,7 @@ public class IndexFieldDataService extends AbstractIndexComponent {
                 .put("long", new PackedArrayIndexFieldData.Builder().setNumericType(IndexNumericFieldData.NumericType.LONG))
                 .put("geo_point", new GeoPointDoubleArrayIndexFieldData.Builder())
                 .put(ParentFieldMapper.NAME, new ParentChildIndexFieldData.Builder())
+                .put("binary", new DisabledIndexFieldData.Builder())
                 .immutableMap();
 
         docValuesBuildersByType = MapBuilder.<String, IndexFieldData.Builder>newMapBuilder()
@@ -84,6 +85,7 @@ public class IndexFieldDataService extends AbstractIndexComponent {
                 .put("int", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.INT))
                 .put("long", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.LONG))
                 .put("geo_point", new GeoPointBinaryDVIndexFieldData.Builder())
+                .put("binary", new BytesBinaryDVIndexFieldData.Builder())
                 .immutableMap();
 
         buildersByTypeAndFormat = MapBuilder.<Tuple<String, String>, IndexFieldData.Builder>newMapBuilder()
@@ -121,6 +123,9 @@ public class IndexFieldDataService extends AbstractIndexComponent {
                 .put(Tuple.tuple("geo_point", DISABLED_FORMAT), new DisabledIndexFieldData.Builder())
                 .put(Tuple.tuple("geo_point", COMPRESSED_FORMAT), new GeoPointCompressedIndexFieldData.Builder())
 
+                .put(Tuple.tuple("binary", DOC_VALUES_FORMAT), new BytesBinaryDVIndexFieldData.Builder())
+                .put(Tuple.tuple("binary", DISABLED_FORMAT), new DisabledIndexFieldData.Builder())
+
                 .immutableMap();
     }
 

+ 106 - 0
src/main/java/org/elasticsearch/index/fielddata/plain/BytesBinaryDVAtomicFieldData.java

@@ -0,0 +1,106 @@
+/*
+ * 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.fielddata.plain;
+
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.index.fielddata.*;
+
+final class BytesBinaryDVAtomicFieldData implements AtomicFieldData<ScriptDocValues> {
+
+    private final AtomicReader reader;
+    private final BinaryDocValues values;
+
+    BytesBinaryDVAtomicFieldData(AtomicReader reader, BinaryDocValues values) {
+        super();
+        this.reader = reader;
+        this.values = values == null ? BinaryDocValues.EMPTY : values;
+    }
+
+    @Override
+    public boolean isMultiValued() {
+        return true;
+    }
+
+    @Override
+    public boolean isValuesOrdered() {
+        return false;
+    }
+
+    @Override
+    public int getNumDocs() {
+        return reader.maxDoc();
+    }
+
+    @Override
+    public long getNumberUniqueValues() {
+        return Long.MAX_VALUE;
+    }
+
+    @Override
+    public long getMemorySizeInBytes() {
+        return -1; // not exposed by Lucene
+    }
+
+    @Override
+    public BytesValues getBytesValues(boolean needsHashes) {
+        return new BytesValues(true) {
+
+            final BytesRef bytes = new BytesRef();
+            final ByteArrayDataInput in = new ByteArrayDataInput();
+
+            @Override
+            public int setDocument(int docId) {
+                values.get(docId, bytes);
+                in.reset(bytes.bytes, bytes.offset, bytes.length);
+                if (bytes.length == 0) {
+                    return 0;
+                } else {
+                    return in.readVInt();
+                }
+            }
+
+            @Override
+            public BytesRef nextValue() {
+                final int length = in.readVInt();
+                scratch.grow(length);
+                in.readBytes(scratch.bytes, 0, length);
+                scratch.length = length;
+                scratch.offset = 0;
+                return scratch;
+            }
+
+        };
+    }
+
+    @Override
+    public ScriptDocValues getScriptValues() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void close() {
+        // no-op
+    }
+
+
+}

+ 81 - 0
src/main/java/org/elasticsearch/index/fielddata/plain/BytesBinaryDVIndexFieldData.java

@@ -0,0 +1,81 @@
+/*
+ * 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.fielddata.plain;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.elasticsearch.ElasticsearchIllegalArgumentException;
+import org.elasticsearch.ElasticsearchIllegalStateException;
+import org.elasticsearch.common.Nullable;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.index.fielddata.FieldDataType;
+import org.elasticsearch.index.fielddata.IndexFieldData;
+import org.elasticsearch.index.fielddata.IndexFieldDataCache;
+import org.elasticsearch.index.fielddata.fieldcomparator.SortMode;
+import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
+import org.elasticsearch.index.mapper.FieldMapper;
+import org.elasticsearch.index.mapper.FieldMapper.Names;
+import org.elasticsearch.index.mapper.MapperService;
+import org.elasticsearch.indices.fielddata.breaker.CircuitBreakerService;
+
+import java.io.IOException;
+
+public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData implements IndexFieldData<BytesBinaryDVAtomicFieldData> {
+
+    public BytesBinaryDVIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType) {
+        super(index, fieldNames, fieldDataType);
+    }
+
+    @Override
+    public boolean valuesOrdered() {
+        return false;
+    }
+
+    @Override
+    public final XFieldComparatorSource comparatorSource(@Nullable Object missingValue, SortMode sortMode) {
+        throw new ElasticsearchIllegalArgumentException("can't sort on binary field");
+    }
+
+    @Override
+    public BytesBinaryDVAtomicFieldData load(AtomicReaderContext context) {
+        try {
+            return new BytesBinaryDVAtomicFieldData(context.reader(), context.reader().getBinaryDocValues(fieldNames.indexName()));
+        } catch (IOException e) {
+            throw new ElasticsearchIllegalStateException("Cannot load doc values", e);
+        }
+    }
+
+    @Override
+    public BytesBinaryDVAtomicFieldData loadDirect(AtomicReaderContext context) throws Exception {
+        return load(context);
+    }
+
+    public static class Builder implements IndexFieldData.Builder {
+
+        @Override
+        public IndexFieldData<?> build(Index index, Settings indexSettings, FieldMapper<?> mapper, IndexFieldDataCache cache,
+                                       CircuitBreakerService breakerService, MapperService mapperService, GlobalOrdinalsBuilder globalOrdinalBuilder) {
+            // Ignore breaker
+            final Names fieldNames = mapper.names();
+            return new BytesBinaryDVIndexFieldData(index, fieldNames, mapper.fieldDataType());
+        }
+
+    }
+}

+ 74 - 33
src/main/java/org/elasticsearch/index/mapper/core/BinaryFieldMapper.java

@@ -19,18 +19,25 @@
 
 package org.elasticsearch.index.mapper.core;
 
+import com.carrotsearch.hppc.ObjectArrayList;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.store.ByteArrayDataOutput;
 import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.ElasticsearchException;
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.common.Base64;
+import org.elasticsearch.common.Nullable;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.compress.CompressorFactory;
 import org.elasticsearch.common.io.stream.BytesStreamOutput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.CollectionUtils;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentParser;
 import org.elasticsearch.index.codec.docvaluesformat.DocValuesFormatProvider;
@@ -86,8 +93,8 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
 
         @Override
         public BinaryFieldMapper build(BuilderContext context) {
-            return new BinaryFieldMapper(buildNames(context), fieldType, compress, compressThreshold, postingsProvider,
-                    docValuesProvider, multiFieldsBuilder.build(this, context), copyTo);
+            return new BinaryFieldMapper(buildNames(context), fieldType, docValues, compress, compressThreshold, postingsProvider,
+                    docValuesProvider, fieldDataSettings, multiFieldsBuilder.build(this, context), copyTo);
         }
     }
 
@@ -119,10 +126,10 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
 
     private long compressThreshold;
 
-    protected BinaryFieldMapper(Names names, FieldType fieldType, Boolean compress, long compressThreshold,
-                                PostingsFormatProvider postingsProvider, DocValuesFormatProvider docValuesProvider,
+    protected BinaryFieldMapper(Names names, FieldType fieldType, Boolean docValues, Boolean compress, long compressThreshold,
+                                PostingsFormatProvider postingsProvider, DocValuesFormatProvider docValuesProvider, @Nullable Settings fieldDataSettings,
                                 MultiFields multiFields, CopyTo copyTo) {
-        super(names, 1.0f, fieldType, null, null, null, postingsProvider, docValuesProvider, null, null, null, null, multiFields, copyTo);
+        super(names, 1.0f, fieldType, docValues, null, null, postingsProvider, docValuesProvider, null, null, fieldDataSettings, null, multiFields, copyTo);
         this.compress = compress;
         this.compressThreshold = compressThreshold;
     }
@@ -134,7 +141,7 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
 
     @Override
     public FieldDataType defaultFieldDataType() {
-        return null;
+        return new FieldDataType("binary");
     }
 
     @Override
@@ -171,7 +178,7 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
 
     @Override
     protected void parseCreateField(ParseContext context, List<Field> fields) throws IOException {
-        if (!fieldType().stored()) {
+        if (!fieldType().stored() && !hasDocValues()) {
             return;
         }
         byte[] value = context.parseExternalValue(byte[].class);
@@ -194,7 +201,20 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
                 value = bStream.bytes().toBytes();
             }
         }
-        fields.add(new Field(names.indexName(), value, fieldType));
+        if (fieldType().stored()) {
+            fields.add(new Field(names.indexName(), value, fieldType));
+        }
+
+        if (hasDocValues()) {
+            CustomBinaryDocValuesField field = (CustomBinaryDocValuesField) context.doc().getByKey(names().indexName());
+            if (field == null) {
+                field = new CustomBinaryDocValuesField(names().indexName(), value);
+                context.doc().addWithKey(names().indexName(), field);
+            } else {
+                field.add(value);
+            }
+        }
+
     }
 
     @Override
@@ -204,10 +224,7 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
 
     @Override
     protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
-        builder.field("type", contentType());
-        if (includeDefaults || !names.name().equals(names.indexNameClean())) {
-            builder.field("index_name", names.indexNameClean());
-        }
+        super.doXContentBody(builder, includeDefaults, params);
         if (compress != null) {
             builder.field("compress", compress);
         } else if (includeDefaults) {
@@ -218,32 +235,16 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
         } else if (includeDefaults) {
             builder.field("compress_threshold", -1);
         }
-        if (includeDefaults || fieldType.stored() != defaultFieldType().stored()) {
-            builder.field("store", fieldType.stored());
-        }
     }
 
     @Override
     public void merge(Mapper mergeWith, MergeContext mergeContext) throws MergeMappingException {
-        if (!(mergeWith instanceof BinaryFieldMapper)) {
-            String mergedType = mergeWith.getClass().getSimpleName();
-            if (mergeWith instanceof AbstractFieldMapper) {
-                mergedType = ((AbstractFieldMapper) mergeWith).contentType();
-            }
-            mergeContext.addConflict("mapper [" + names.fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]");
-            // different types, return
+        super.merge(mergeWith, mergeContext);
+        if (!this.getClass().equals(mergeWith.getClass())) {
             return;
         }
 
         BinaryFieldMapper sourceMergeWith = (BinaryFieldMapper) mergeWith;
-
-        if (this.fieldType().stored() != sourceMergeWith.fieldType().stored()) {
-            mergeContext.addConflict("mapper [" + names.fullName() + "] has different store values");
-        }
-        if (!this.names().equals(sourceMergeWith.names())) {
-            mergeContext.addConflict("mapper [" + names.fullName() + "] has different index_name");
-        }
-
         if (!mergeContext.mergeFlags().simulate()) {
             if (sourceMergeWith.compress != null) {
                 this.compress = sourceMergeWith.compress;
@@ -254,8 +255,48 @@ public class BinaryFieldMapper extends AbstractFieldMapper<BytesReference> {
         }
     }
 
-    @Override
-    public boolean hasDocValues() {
-        return false;
+    public static class CustomBinaryDocValuesField extends NumberFieldMapper.CustomNumericDocValuesField {
+
+        public static final FieldType TYPE = new FieldType();
+        static {
+            TYPE.setDocValueType(FieldInfo.DocValuesType.BINARY);
+            TYPE.freeze();
+        }
+
+        private final ObjectArrayList<byte[]> bytesList;
+
+        private int totalSize = 0;
+
+        public CustomBinaryDocValuesField(String  name, byte[] bytes) {
+            super(name);
+            bytesList = new ObjectArrayList<>();
+            add(bytes);
+        }
+
+        public void add(byte[] bytes) {
+            bytesList.add(bytes);
+            totalSize += bytes.length;
+        }
+
+        @Override
+        public BytesRef binaryValue() {
+            try {
+                CollectionUtils.sortAndDedup(bytesList);
+                int size = bytesList.size();
+                final byte[] bytes = new byte[totalSize + (size + 1) * 5];
+                ByteArrayDataOutput out = new ByteArrayDataOutput(bytes);
+                out.writeVInt(size);  // write total number of values
+                for (int i = 0; i < size; i ++) {
+                    final byte[] value = bytesList.get(i);
+                    int valueLength = value.length;
+                    out.writeVInt(valueLength);
+                    out.writeBytes(value, 0, valueLength);
+                }
+                return new BytesRef(bytes, 0, out.getPosition());
+            } catch (IOException e) {
+                throw new ElasticsearchException("Failed to get binary value", e);
+            }
+
+        }
     }
 }

+ 2 - 0
src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTests.java

@@ -76,6 +76,8 @@ public abstract class AbstractFieldDataTests extends ElasticsearchTestCase {
             mapper = MapperBuilders.geoPointField(fieldName).fieldDataSettings(type.getSettings()).build(context);
         } else if (type.getType().equals("_parent")) {
             mapper = MapperBuilders.parent().type(fieldName).build(context);
+        } else if (type.getType().equals("binary")) {
+            mapper = MapperBuilders.binaryField(fieldName).fieldDataSettings(type.getSettings()).build(context);
         } else {
             throw new UnsupportedOperationException(type.getType());
         }

+ 113 - 0
src/test/java/org/elasticsearch/index/fielddata/BinaryDVFieldDataTests.java

@@ -0,0 +1,113 @@
+/*
+ * 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.fielddata;
+
+import com.carrotsearch.hppc.ObjectArrayList;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.common.util.CollectionUtils;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.index.mapper.DocumentMapper;
+import org.elasticsearch.index.mapper.MapperTestUtils;
+import org.elasticsearch.index.mapper.ParsedDocument;
+import org.junit.Test;
+
+import static org.hamcrest.Matchers.equalTo;
+
+/**
+ *
+ */
+public class BinaryDVFieldDataTests extends AbstractFieldDataTests {
+
+    @Test
+    public void testDocValue() throws Exception {
+        String mapping = XContentFactory.jsonBuilder().startObject().startObject("test")
+                .startObject("properties")
+                .startObject("field")
+                .field("type", "binary")
+                .startObject("fielddata").field("format", "doc_values").endObject()
+                .endObject()
+                .endObject()
+                .endObject().endObject().string();
+
+        final DocumentMapper mapper = MapperTestUtils.newParser().parse(mapping);
+
+
+        ObjectArrayList<byte[]> bytesList1 = new ObjectArrayList<>(2);
+        bytesList1.add(randomBytes());
+        bytesList1.add(randomBytes());
+        XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startArray("field").value(bytesList1.get(0)).value(bytesList1.get(1)).endArray().endObject();
+        ParsedDocument d = mapper.parse("test", "1", doc.bytes());
+        writer.addDocument(d.rootDoc());
+
+        byte[] bytes1 = randomBytes();
+        doc = XContentFactory.jsonBuilder().startObject().field("field", bytes1).endObject();
+        d = mapper.parse("test", "2", doc.bytes());
+        writer.addDocument(d.rootDoc());
+
+        doc = XContentFactory.jsonBuilder().startObject().endObject();
+        d = mapper.parse("test", "3", doc.bytes());
+        writer.addDocument(d.rootDoc());
+
+        // test remove duplicate value
+        ObjectArrayList<byte[]> bytesList2 = new ObjectArrayList<>(2);
+        bytesList2.add(randomBytes());
+        bytesList2.add(randomBytes());
+        doc = XContentFactory.jsonBuilder().startObject().startArray("field").value(bytesList2.get(0)).value(bytesList2.get(1)).value(bytesList2.get(0)).endArray().endObject();
+        d = mapper.parse("test", "4", doc.bytes());
+        writer.addDocument(d.rootDoc());
+
+        AtomicReaderContext reader = refreshReader();
+        IndexFieldData indexFieldData = getForField("field");
+        AtomicFieldData fieldData = indexFieldData.load(reader);
+        assertThat(fieldData.getNumDocs(), equalTo(4));
+
+        BytesValues bytesValues = fieldData.getBytesValues(randomBoolean());
+
+        CollectionUtils.sortAndDedup(bytesList1);
+        assertThat(bytesValues.setDocument(0), equalTo(2));
+        assertThat(bytesValues.nextValue(), equalTo(new BytesRef(bytesList1.get(0))));
+        assertThat(bytesValues.nextValue(), equalTo(new BytesRef(bytesList1.get(1))));
+
+        assertThat(bytesValues.setDocument(1), equalTo(1));
+        assertThat(bytesValues.nextValue(), equalTo(new BytesRef(bytes1)));
+
+        assertThat(bytesValues.setDocument(2), equalTo(0));
+
+        CollectionUtils.sortAndDedup(bytesList2);
+        assertThat(bytesValues.setDocument(3), equalTo(2));
+        assertThat(bytesValues.nextValue(), equalTo(new BytesRef(bytesList2.get(0))));
+        assertThat(bytesValues.nextValue(), equalTo(new BytesRef(bytesList2.get(1))));
+    }
+
+    private byte[] randomBytes() {
+        int size = randomIntBetween(10, 1000);
+        byte[] bytes = new byte[size];
+        getRandom().nextBytes(bytes);
+        return bytes;
+    }
+
+    @Override
+    protected FieldDataType getFieldDataType() {
+        return new FieldDataType("binary", ImmutableSettings.builder().put("format", "doc_values"));
+    }
+}