spike-liu commented on a change in pull request #464: URL: https://github.com/apache/lucene/pull/464#discussion_r781808923
########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; Review comment: sorry for the code smell. fixed. ########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; + +import java.io.IOException; + +import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.*; Review comment: fixed. ########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; + +import java.io.IOException; + +import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.*; +import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +/** + * Writer for {@link DirectDocValuesFormat} + */ + +class DirectDocValuesConsumer extends DocValuesConsumer { + static final int MISSING_BITSET = Integer.MIN_VALUE; + final int maxDoc; + IndexOutput data, meta; + + DirectDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.maxDoc(); + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeVInt(field.number); + meta.writeByte(NUMBER); + addNumericFieldValues(field, new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return new SortedNumericDocValues() { + final NumericDocValues values = valuesProducer.getNumeric(field); + long[] ords = LongsRef.EMPTY_LONGS; + int docIDUpto, i, docValueCount; + + @Override + public long nextValue() { + return ords[i++]; + } + + @Override + public int docValueCount() { + return docValueCount; + } + + @Override + public boolean advanceExact(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public int docID() { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() throws IOException { + if (docIDUpto == maxDoc) { + return NO_MORE_DOCS; + } + int docID = values.nextDoc(); + if (docID == NO_MORE_DOCS) { + docID = MISSING_BITSET; + } + docValueCount = 0; + while (docIDUpto <= values.docID() && docIDUpto < maxDoc) { + ords = ArrayUtil.grow(ords, docValueCount + 1); + if (docIDUpto++ == values.docID()) { + ords[docValueCount++] = values.longValue(); + } else { + ords[docValueCount++] = MISSING_BITSET; + } + } + i = 0; + return docID; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + }; + } + }); + } + + private void addNumericFieldValues(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + meta.writeLong(data.getFilePointer()); + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + boolean missing = false; + + long count = 0; + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + for (int docID = values.nextDoc(); docID != DocIdSetIterator.NO_MORE_DOCS; docID = values.nextDoc()) { + for (int i = 0, docValueCount = values.docValueCount(); i < docValueCount; ++i) { + long v = values.nextValue(); + if (v != MISSING_BITSET) { + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } else { + missing = true; + } + count++; + if (count >= DirectDocValuesFormat.MAX_SORTED_SET_ORDS) { + throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + DirectDocValuesFormat.MAX_SORTED_SET_ORDS + " values/total ords"); + } + } + } + + meta.writeInt((int) count); + + if (missing) { + long start = data.getFilePointer(); + writeMissingBitset(new TermsEnum() { + final SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + int docID = values.nextDoc(); + int i; + int docValueCount = values.docValueCount(); + + @Override + public AttributeSource attributes() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean seekExact(BytesRef text) { + throw new UnsupportedOperationException(); + } + + @Override + public SeekStatus seekCeil(BytesRef text) { + throw new UnsupportedOperationException(); + } + + @Override + public void seekExact(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public void seekExact(BytesRef term, TermState state) { + throw new UnsupportedOperationException(); + } + + @Override + public BytesRef term() { + throw new UnsupportedOperationException(); + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) { + throw new UnsupportedOperationException(); + } + + @Override + public ImpactsEnum impacts(int flags) { + throw new UnsupportedOperationException(); + } + + @Override + public TermState termState() { + throw new UnsupportedOperationException(); + } + + @Override + public BytesRef next() throws IOException { + if (i < docValueCount) { + i++; + } else { + docID = values.nextDoc(); + if (docID == NO_MORE_DOCS) return null; + i = 1; + docValueCount = values.docValueCount(); + } + long value = values.nextValue(); + BytesRef result = new BytesRef(); + if (value == MISSING_BITSET) { + result.bytes = null; + } + return result; + } + }); + + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } else { + meta.writeLong(-1L); + } + + byte byteWidth; + if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { + byteWidth = 1; + } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) { + byteWidth = 2; + } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) { + byteWidth = 4; + } else { + byteWidth = 8; + } + meta.writeByte(byteWidth); + + values = valuesProducer.getSortedNumeric(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, docValueCount = values.docValueCount(); i < docValueCount; ++i) { + long v = values.nextValue(); + if (v == MISSING_BITSET) { + v = 0; + } + + switch (byteWidth) { + case 1: + data.writeByte((byte) v); + break; + case 2: + data.writeShort((short) v); + break; + case 4: + data.writeInt((int) v); + break; + case 8: + data.writeLong(v); + break; + } + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeVInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + data = meta = null; + } + } + + @Override + public void addBinaryField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + meta.writeVInt(field.number); + meta.writeByte(BYTES); + addBinaryFieldValues(field, new EmptyDocValuesProducer() { + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return new SortedSetDocValues() { + final BinaryDocValues values = valuesProducer.getBinary(field); + + @Override + public long nextOrd() { + return 0; + } + + @Override + public BytesRef lookupOrd(long ord) throws IOException { + if (ord > values.docID()) { + values.nextDoc(); + } + BytesRef result; + if (ord == values.docID()) { + result = values.binaryValue(); + } else { + result = new BytesRef(); + result.bytes = null; + } + return result; + } + + @Override + public long getValueCount() { + return maxDoc; + } + + @Override + public boolean advanceExact(int target) { + return false; + } + + @Override + public int docID() { + return 0; + } + + @Override + public int nextDoc() { + return 0; + } + + @Override + public int advance(int target) { + return 0; + } + + @Override + public long cost() { + return 0; + } + }; + } + }); + } + + private void addBinaryFieldValues(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + // write the byte[] data + final long startFP = data.getFilePointer(); + boolean missing = false; + long totalBytes = 0; + int count = 0; + TermsEnum iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if (term.bytes != null) { + data.writeBytes(term.bytes, term.offset, term.length); + totalBytes += term.length; + if (totalBytes > DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH) { + throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, cannot have more than DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH (" + DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH + ") bytes"); + } + } else { + missing = true; + } + count++; + } + + meta.writeLong(startFP); + meta.writeInt((int) totalBytes); + meta.writeInt(count); + if (missing) { + long start = data.getFilePointer(); + iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + writeMissingBitset(iterator); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } else { + meta.writeLong(-1L); + } + + int addr = 0; + iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + data.writeInt(addr); + if (term.bytes != null) { + addr += term.length; + } + } + data.writeInt(addr); + } + + // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on, + // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode) + void writeMissingBitset(TermsEnum termsEnum) throws IOException { + long bits = 0; + int count = 0; + for (BytesRef term = termsEnum.next(); term != null; term = termsEnum.next()) { + if (count == 64) { + data.writeLong(bits); + count = 0; + bits = 0; + } + if (term.bytes != null) { + bits |= 1L << (count & 0x3f); Review comment: verified and they are exactly the same. fixed. ########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; + +import java.io.IOException; + +import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.*; +import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +/** + * Writer for {@link DirectDocValuesFormat} + */ + +class DirectDocValuesConsumer extends DocValuesConsumer { + static final int MISSING_BITSET = Integer.MIN_VALUE; + final int maxDoc; + IndexOutput data, meta; + + DirectDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.maxDoc(); + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeVInt(field.number); + meta.writeByte(NUMBER); + addNumericFieldValues(field, new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return new SortedNumericDocValues() { + final NumericDocValues values = valuesProducer.getNumeric(field); + long[] ords = LongsRef.EMPTY_LONGS; Review comment: more intuitive. fixed with another 7 similar name convention together. ########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; + +import java.io.IOException; + +import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.*; +import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +/** + * Writer for {@link DirectDocValuesFormat} + */ + +class DirectDocValuesConsumer extends DocValuesConsumer { + static final int MISSING_BITSET = Integer.MIN_VALUE; Review comment: You are right. This is a bug and try to fix it with: 1. remove MISSING_BITSET; 2. + private long[] nullValues within SortedNumericDocValues to store the missingValue position; 3. + public boolean isNextValueMissing() to return if nextValue is missing; ########## File path: lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectDocValuesConsumer.java ########## @@ -0,0 +1,980 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF 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.apache.lucene.codecs.memory; + + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.*; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.*; + +import java.io.IOException; + +import static org.apache.lucene.codecs.memory.DirectDocValuesProducer.*; +import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; +import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; + +/** + * Writer for {@link DirectDocValuesFormat} + */ + +class DirectDocValuesConsumer extends DocValuesConsumer { + static final int MISSING_BITSET = Integer.MIN_VALUE; + final int maxDoc; + IndexOutput data, meta; + + DirectDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException { + maxDoc = state.segmentInfo.maxDoc(); + boolean success = false; + try { + String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension); + data = state.directory.createOutput(dataName, state.context); + CodecUtil.writeIndexHeader(data, dataCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension); + meta = state.directory.createOutput(metaName, state.context); + CodecUtil.writeIndexHeader(meta, metaCodec, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(this); + } + } + } + + @Override + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { + meta.writeVInt(field.number); + meta.writeByte(NUMBER); + addNumericFieldValues(field, new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return new SortedNumericDocValues() { + final NumericDocValues values = valuesProducer.getNumeric(field); + long[] ords = LongsRef.EMPTY_LONGS; + int docIDUpto, i, docValueCount; + + @Override + public long nextValue() { + return ords[i++]; + } + + @Override + public int docValueCount() { + return docValueCount; + } + + @Override + public boolean advanceExact(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public int docID() { + throw new UnsupportedOperationException(); + } + + @Override + public int nextDoc() throws IOException { + if (docIDUpto == maxDoc) { + return NO_MORE_DOCS; + } + int docID = values.nextDoc(); + if (docID == NO_MORE_DOCS) { + docID = MISSING_BITSET; + } + docValueCount = 0; + while (docIDUpto <= values.docID() && docIDUpto < maxDoc) { + ords = ArrayUtil.grow(ords, docValueCount + 1); + if (docIDUpto++ == values.docID()) { + ords[docValueCount++] = values.longValue(); + } else { + ords[docValueCount++] = MISSING_BITSET; + } + } + i = 0; + return docID; + } + + @Override + public int advance(int target) { + throw new UnsupportedOperationException(); + } + + @Override + public long cost() { + throw new UnsupportedOperationException(); + } + }; + } + }); + } + + private void addNumericFieldValues(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + meta.writeLong(data.getFilePointer()); + long minValue = Long.MAX_VALUE; + long maxValue = Long.MIN_VALUE; + boolean missing = false; + + long count = 0; + SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + for (int docID = values.nextDoc(); docID != DocIdSetIterator.NO_MORE_DOCS; docID = values.nextDoc()) { + for (int i = 0, docValueCount = values.docValueCount(); i < docValueCount; ++i) { + long v = values.nextValue(); + if (v != MISSING_BITSET) { + minValue = Math.min(minValue, v); + maxValue = Math.max(maxValue, v); + } else { + missing = true; + } + count++; + if (count >= DirectDocValuesFormat.MAX_SORTED_SET_ORDS) { + throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, must be <= " + DirectDocValuesFormat.MAX_SORTED_SET_ORDS + " values/total ords"); + } + } + } + + meta.writeInt((int) count); + + if (missing) { + long start = data.getFilePointer(); + writeMissingBitset(new TermsEnum() { + final SortedNumericDocValues values = valuesProducer.getSortedNumeric(field); + int docID = values.nextDoc(); + int i; + int docValueCount = values.docValueCount(); + + @Override + public AttributeSource attributes() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean seekExact(BytesRef text) { + throw new UnsupportedOperationException(); + } + + @Override + public SeekStatus seekCeil(BytesRef text) { + throw new UnsupportedOperationException(); + } + + @Override + public void seekExact(long ord) { + throw new UnsupportedOperationException(); + } + + @Override + public void seekExact(BytesRef term, TermState state) { + throw new UnsupportedOperationException(); + } + + @Override + public BytesRef term() { + throw new UnsupportedOperationException(); + } + + @Override + public long ord() { + throw new UnsupportedOperationException(); + } + + @Override + public int docFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public long totalTermFreq() { + throw new UnsupportedOperationException(); + } + + @Override + public PostingsEnum postings(PostingsEnum reuse, int flags) { + throw new UnsupportedOperationException(); + } + + @Override + public ImpactsEnum impacts(int flags) { + throw new UnsupportedOperationException(); + } + + @Override + public TermState termState() { + throw new UnsupportedOperationException(); + } + + @Override + public BytesRef next() throws IOException { + if (i < docValueCount) { + i++; + } else { + docID = values.nextDoc(); + if (docID == NO_MORE_DOCS) return null; + i = 1; + docValueCount = values.docValueCount(); + } + long value = values.nextValue(); + BytesRef result = new BytesRef(); + if (value == MISSING_BITSET) { + result.bytes = null; + } + return result; + } + }); + + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } else { + meta.writeLong(-1L); + } + + byte byteWidth; + if (minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) { + byteWidth = 1; + } else if (minValue >= Short.MIN_VALUE && maxValue <= Short.MAX_VALUE) { + byteWidth = 2; + } else if (minValue >= Integer.MIN_VALUE && maxValue <= Integer.MAX_VALUE) { + byteWidth = 4; + } else { + byteWidth = 8; + } + meta.writeByte(byteWidth); + + values = valuesProducer.getSortedNumeric(field); + for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) { + for (int i = 0, docValueCount = values.docValueCount(); i < docValueCount; ++i) { + long v = values.nextValue(); + if (v == MISSING_BITSET) { + v = 0; + } + + switch (byteWidth) { + case 1: + data.writeByte((byte) v); + break; + case 2: + data.writeShort((short) v); + break; + case 4: + data.writeInt((int) v); + break; + case 8: + data.writeLong(v); + break; + } + } + } + } + + @Override + public void close() throws IOException { + boolean success = false; + try { + if (meta != null) { + meta.writeVInt(-1); // write EOF marker + CodecUtil.writeFooter(meta); // write checksum + } + if (data != null) { + CodecUtil.writeFooter(data); + } + success = true; + } finally { + if (success) { + IOUtils.close(data, meta); + } else { + IOUtils.closeWhileHandlingException(data, meta); + } + data = meta = null; + } + } + + @Override + public void addBinaryField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + meta.writeVInt(field.number); + meta.writeByte(BYTES); + addBinaryFieldValues(field, new EmptyDocValuesProducer() { + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return new SortedSetDocValues() { + final BinaryDocValues values = valuesProducer.getBinary(field); + + @Override + public long nextOrd() { + return 0; + } + + @Override + public BytesRef lookupOrd(long ord) throws IOException { + if (ord > values.docID()) { + values.nextDoc(); + } + BytesRef result; + if (ord == values.docID()) { + result = values.binaryValue(); + } else { + result = new BytesRef(); + result.bytes = null; + } + return result; + } + + @Override + public long getValueCount() { + return maxDoc; + } + + @Override + public boolean advanceExact(int target) { + return false; + } + + @Override + public int docID() { + return 0; + } + + @Override + public int nextDoc() { + return 0; + } + + @Override + public int advance(int target) { + return 0; + } + + @Override + public long cost() { + return 0; + } + }; + } + }); + } + + private void addBinaryFieldValues(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException { + // write the byte[] data + final long startFP = data.getFilePointer(); + boolean missing = false; + long totalBytes = 0; + int count = 0; + TermsEnum iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + if (term.bytes != null) { + data.writeBytes(term.bytes, term.offset, term.length); + totalBytes += term.length; + if (totalBytes > DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH) { + throw new IllegalArgumentException("DocValuesField \"" + field.name + "\" is too large, cannot have more than DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH (" + DirectDocValuesFormat.MAX_TOTAL_BYTES_LENGTH + ") bytes"); + } + } else { + missing = true; + } + count++; + } + + meta.writeLong(startFP); + meta.writeInt((int) totalBytes); + meta.writeInt(count); + if (missing) { + long start = data.getFilePointer(); + iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + writeMissingBitset(iterator); + meta.writeLong(start); + meta.writeLong(data.getFilePointer() - start); + } else { + meta.writeLong(-1L); + } + + int addr = 0; + iterator = new SortedSetDocValuesTermsEnum(valuesProducer.getSortedSet(field)); + for (BytesRef term = iterator.next(); term != null; term = iterator.next()) { + data.writeInt(addr); + if (term.bytes != null) { + addr += term.length; + } + } + data.writeInt(addr); + } + + // TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on, + // but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode) + void writeMissingBitset(TermsEnum termsEnum) throws IOException { Review comment: refactor with using DocIdSetIterator instead of TermsEnum. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org For additional commands, e-mail: issues-h...@lucene.apache.org