dsmiley commented on code in PR #13327: URL: https://github.com/apache/lucene/pull/13327#discussion_r1597484741
########## lucene/core/src/java/org/apache/lucene/index/FieldInfos.java: ########## @@ -139,17 +148,38 @@ public FieldInfos(FieldInfo[] infos) { this.softDeletesField = softDeletesField; this.parentField = parentField; - FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, infos.length); - Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, fi2.number)); - int maxFieldNumber = infos.length == 0 ? -1 : sortedFieldInfos[infos.length - 1].number; - // If there are many fields and the max field number is greater than twice the number - // of fields, then a map structure is more compact to store the by-number mapping. - byNumber = - maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32 - ? new MapFieldInfoByNumber(infos) - : new ArrayFieldInfoByNumber(infos, maxFieldNumber); - // The iteration of FieldInfo is ordered by ascending field number. - values = Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos)); + if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) { + // The input FieldInfo[] contains all fields numbered from 0 to infos.length - 1 and they are + // sorted, use it directly. This is an optimization when reading a segment with all fields + // since the FieldInfo[] is sorted. + byNumber = infos; // We could copy the input array, but do we need to? + values = Arrays.asList(byNumber); + } else { + byNumber = new FieldInfo[maxFieldNumber + 1]; + for (FieldInfo fieldInfo : infos) { + FieldInfo previous = byNumber[fieldInfo.number]; + if (previous != null) { + throw new IllegalArgumentException( + "duplicate field numbers: " + + previous.name + + " and " + + fieldInfo.name + + " have: " + + fieldInfo.number); + } + byNumber[fieldInfo.number] = fieldInfo; + } + if (maxFieldNumber == infos.length - 1) { + // No fields are missing, use byNumber. + values = Arrays.asList(byNumber); + } else { + // The below code is faster than Arrays.stream(byNumber).filter(Objects::nonNull).toList(), + // mainly when the input FieldInfo[] is sorted, when reading a segment. + FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, infos.length); Review Comment: Don't even need to copy it as we've given ourselves permission to take ownership of the input array; to manipulate it. ########## lucene/core/src/java/org/apache/lucene/index/FieldInfos.java: ########## @@ -156,15 +151,38 @@ public FieldInfos(FieldInfo[] infos) { this.softDeletesField = softDeletesField; this.parentField = parentField; - List<FieldInfo> valuesTemp = new ArrayList<>(infos.length); - byNumber = new FieldInfo[size]; - for (int i = 0; i < size; i++) { - byNumber[i] = byNumberTemp[i]; - if (byNumberTemp[i] != null) { - valuesTemp.add(byNumberTemp[i]); + if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) { + // The input FieldInfo[] contains all fields numbered from 0 to infos.length - 1, and they are + // sorted, use it directly. This is an optimization when reading a segment with all fields + // since the FieldInfo[] is sorted. + byNumber = infos; + values = Arrays.asList(byNumber); + } else { + byNumber = new FieldInfo[maxFieldNumber + 1]; + for (FieldInfo fieldInfo : infos) { + FieldInfo existing = byNumber[fieldInfo.number]; + if (existing != null) { + throw new IllegalArgumentException( + "duplicate field numbers: " + + existing.name + + " and " + + fieldInfo.name + + " have: " + + fieldInfo.number); + } + byNumber[fieldInfo.number] = fieldInfo; + } + if (maxFieldNumber == infos.length - 1) { + // No fields are missing, use byNumber. + values = Arrays.asList(byNumber); + } else { + // The below code is faster than Arrays.stream(byNumber).filter(Objects::nonNull).toList(), + // mainly when the input FieldInfo[] is sorted, when reading a segment. Review Comment: If fieldNumberStrictlyAscending, and we've given ourselves permission to accept the input array, we can merely wrap it in Arrays.asList and we're done. -- 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