Re: [PR] Reduce ArrayUtil#grow in decompress [lucene]
dungba88 commented on code in PR #12996: URL: https://github.com/apache/lucene/pull/12996#discussion_r1443985033 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/LZ4WithPresetDictCompressionMode.java: ## @@ -128,10 +128,12 @@ public void decompress(DataInput in, int originalLength, int offset, int length, } // Read blocks that intersect with the interval we need + if (offsetInBlock < offset + length) { Review Comment: Wondering if growExact would be better here. I think grow will try to oversize it? -- 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
Re: [PR] Reduce ArrayUtil#grow in decompress [lucene]
easyice commented on code in PR #12996: URL: https://github.com/apache/lucene/pull/12996#discussion_r1443987082 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/LZ4WithPresetDictCompressionMode.java: ## @@ -128,10 +128,12 @@ public void decompress(DataInput in, int originalLength, int offset, int length, } // Read blocks that intersect with the interval we need + if (offsetInBlock < offset + length) { Review Comment: You are right, thank you! -- 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
Re: [PR] Reduce ArrayUtil#grow in decompress [lucene]
easyice commented on code in PR #12996: URL: https://github.com/apache/lucene/pull/12996#discussion_r1443993181 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/LZ4WithPresetDictCompressionMode.java: ## @@ -128,10 +128,12 @@ public void decompress(DataInput in, int originalLength, int offset, int length, } // Read blocks that intersect with the interval we need + if (offsetInBlock < offset + length) { Review Comment: I kept the logic that if the new length is less than the current array then we don't do anything. it appears in some cases. -- 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
[PR] Avoid reset BlockDocsEnum#freqBuffer when indexHasFreq is false [lucene]
easyice opened a new pull request, #12997: URL: https://github.com/apache/lucene/pull/12997 As discussion in https://github.com/apache/lucene/pull/12954, this is the second implementation can be considered. this change yielded a 13% speedup for `PKLookup`. -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444066152 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -38,27 +38,49 @@ * @lucene.experimental */ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable { + private static final int CHUNK_SIZE = 8192; - private final int[] parents; + private final ChunkedArray parents; // the following two arrays are lazily initialized. note that we only keep a // single boolean member as volatile, instead of declaring the arrays // volatile. the code guarantees that only after the boolean is set to true, // the arrays are returned. private volatile boolean initializedChildren = false; - private int[] children, siblings; + private ChunkedArray children, siblings; + + private static class ChunkedArray extends ParallelTaxonomyArrays.IntArray { +private final int[][] values; + +private ChunkedArray(int[][] values) { + this.values = values; +} + +@Override +public int get(int i) { + return values[i / CHUNK_SIZE][i % CHUNK_SIZE]; Review Comment: That's a great idea! -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444066462 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray(copyFrom.parents.values, parentArray); +initParents(parentArray, reader, copyFrom.parents.length()); +parents = new ChunkedArray(parentArray); if (copyFrom.initializedChildren) { initChildrenSiblings(copyFrom); } } + private static int[][] allocateChunkedArray(int size) { +int chunkCount = size / CHUNK_SIZE + 1; Review Comment: In practice, we always have `size >= 1` (because the `TaxonomyWriter` writes the root ordinal on startup) -- but relying on that behavior may bite us. I'll add a check for the `size == 0` case. -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444069001 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray(copyFrom.parents.values, parentArray); +initParents(parentArray, reader, copyFrom.parents.length()); +parents = new ChunkedArray(parentArray); if (copyFrom.initializedChildren) { initChildrenSiblings(copyFrom); } } + private static int[][] allocateChunkedArray(int size) { +int chunkCount = size / CHUNK_SIZE + 1; +int lastChunkSize = size % CHUNK_SIZE; +int[][] array = new int[chunkCount][]; +if (array.length > 0) { + for (int i = 0; i < chunkCount - 1; i++) { +array[i] = new int[CHUNK_SIZE]; + } + array[chunkCount - 1] = new int[lastChunkSize]; +} +return array; + } + + private static void copyChunkedArray(int[][] oldArray, int[][] newArray) { +// Copy all but the last (maybe partial) chunk from the old array +if (oldArray.length > 1) { + System.arraycopy(oldArray, 0, newArray, 0, oldArray.length - 1); +} +int[] lastCopyChunk = oldArray[oldArray.length - 1]; +System.arraycopy(lastCopyChunk, 0, newArray[oldArray.length - 1], 0, lastCopyChunk.length); Review Comment: The last chunk is the only one that changes -- the other chunks can be reused. Suppose the old array had size 20,000. It would look like: ``` 0 -> 8192 element chunk (c0) 1 -> 8192 element chunk (c1) 2 -> 3616 element chunk (c2) ``` Suppose I want to add 10 more elements. In the old way, that would mean allocating a whole new 20,010 element array and copying over the 20,000 elements from the original array. In the new, chunked world, we allocate a new 3-element `int[][]`, and populate it like: ``` 0 -> c0 (reusing the old array) 1 -> c1 (reusing the old array) 2 -> c2' (new 3626-element array). ``` Now I copy over the 3616 elements from `c2` to `c2'`. The total "temporary" extra space is 3 references for the new top-level array, plus 3616 integers (since we're holding onto `c2` until we've copied it). The old model needed to allocate a temporary 20,000. elements. In general, where the old model required `n` temporary extra `int`s (so `4n` bytes plus a few bytes of array overhead), the new model needs `ceil(n/8192)` temporary pointers (the new `int[][]` array) plus up to 8192 extra `int`s for the last chunk. In terms of minimizing waste, assuming references are 32-bit, I suppose the ideal chunk size is the square root of `n` (if references are 64-bit, we would be better off with bigger chunks). If we want to more efficiently handle more than 67 million ordinals, I guess we could set chunk size to something bigger than 8192. I just copied the value from `IntBlockPool`. -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444069632 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); Review Comment: Oof -- you're right. That was a terrible attempt at code reuse on my part, and indeed it would make the whole thing useless. That's embarrassing! I originally had some of the code from `allocateChunkedArray` in `copyChunkedArray` and I tried to make it reusable in the "fresh reader, no copying" use-case. Thank you for catching that! -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444070455 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -153,12 +203,13 @@ private void initParents(IndexReader reader, int first) throws IOException { * NOTE: you should call this method from a thread-safe code. */ TaxonomyIndexArrays add(int ordinal, int parentOrdinal) { -if (ordinal >= parents.length) { - int[] newarray = ArrayUtil.grow(parents, ordinal + 1); - newarray[ordinal] = parentOrdinal; - return new TaxonomyIndexArrays(newarray); +if (ordinal >= parents.length()) { + int[][] newParents = allocateChunkedArray(ArrayUtil.oversize(ordinal + 1, Integer.BYTES)); Review Comment: This is actually consistent with the existing behavior: -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444070455 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -153,12 +203,13 @@ private void initParents(IndexReader reader, int first) throws IOException { * NOTE: you should call this method from a thread-safe code. */ TaxonomyIndexArrays add(int ordinal, int parentOrdinal) { -if (ordinal >= parents.length) { - int[] newarray = ArrayUtil.grow(parents, ordinal + 1); - newarray[ordinal] = parentOrdinal; - return new TaxonomyIndexArrays(newarray); +if (ordinal >= parents.length()) { + int[][] newParents = allocateChunkedArray(ArrayUtil.oversize(ordinal + 1, Integer.BYTES)); Review Comment: This is actually consistent with the old behavior: ``` int[] newarray = ArrayUtil.grow(parents, ordinal + 1); newarray[ordinal] = parentOrdinal; return new TaxonomyIndexArrays(newarray); ``` That call to `ArrayUtil.grow` would oversize the array, leaving some spare extra elements at the end. It confused me when I saw it. Then I realized that this kind of instance of the array is only used on the write path and (as far as I can tell) is never passed to the `TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom)` constructor. So, yes, it "wastes" (really "pre-allocates") some extra space at the end, so the length doesn't reflect the number of assigned ordinals, but that's okay because the writer's source of truth is the `nextId` value. -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
msfroh commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444073439 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -38,27 +38,49 @@ * @lucene.experimental */ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable { + private static final int CHUNK_SIZE = 8192; - private final int[] parents; + private final ChunkedArray parents; // the following two arrays are lazily initialized. note that we only keep a // single boolean member as volatile, instead of declaring the arrays // volatile. the code guarantees that only after the boolean is set to true, // the arrays are returned. private volatile boolean initializedChildren = false; - private int[] children, siblings; + private ChunkedArray children, siblings; + + private static class ChunkedArray extends ParallelTaxonomyArrays.IntArray { +private final int[][] values; + +private ChunkedArray(int[][] values) { + this.values = values; +} + +@Override +public int get(int i) { + return values[i / CHUNK_SIZE][i % CHUNK_SIZE]; +} + +public void set(int i, int val) { + values[i / CHUNK_SIZE][i % CHUNK_SIZE] = val; +} + +@Override +public int length() { + return (values.length - 1) * CHUNK_SIZE + values[values.length - 1].length; +} + } /** Used by {@link #add(int, int)} after the array grew. */ - private TaxonomyIndexArrays(int[] parents) { -this.parents = parents; + private TaxonomyIndexArrays(int[][] parents) { +this.parents = new ChunkedArray(parents); } public TaxonomyIndexArrays(IndexReader reader) throws IOException { -parents = new int[reader.maxDoc()]; -if (parents.length > 0) { - initParents(reader, 0); - parents[0] = TaxonomyReader.INVALID_ORDINAL; -} +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); Review Comment: I was a little torn on this. I figured I would try to avoid method calls where possible (assuming method calls have more overhead than array access), and could happily muck about in the `ChunkedArray` internals within `TaxonomyIndexArrays`(since it's a private implementation detail). We could move `allocateChunkedArray` and `copyChunkedArray` logic into `ChunkedArray` itself (and call them from `ChunkedArray` constructors). It's still in the same file, so I don't have strong opinions either way. -- 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
Re: [I] Taxonomy facets: can we change massive `int[]` for parent/child/sibling tree to paged/block `int[]` to reduce RAM pressure? [lucene]
msfroh commented on issue #12989: URL: https://github.com/apache/lucene/issues/12989#issuecomment-188097 > It's very possible I'm missing something, but right now it looks to me like we haven't improved on that front. Doing shallow copies of the old array without allocating new memory would solve it though. What you've missed is that I'm a big dum-dum 😁 Thanks for catching that! I refactored some code into a shared method (between the "reuse old arrays" case and the "start fresh with a TaxonomyReader" case) and foolishly applied the "start fresh" logic every time. I've fixed it in a subsequent commit (allocating chunks only starting from the index of the last chunk of the old array). I also incorporated several of the other changes that you suggested. Thanks a lot! -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
stefanvodita commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444211059 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray(copyFrom.parents.values, parentArray); +initParents(parentArray, reader, copyFrom.parents.length()); +parents = new ChunkedArray(parentArray); if (copyFrom.initializedChildren) { initChildrenSiblings(copyFrom); } } + private static int[][] allocateChunkedArray(int size) { +int chunkCount = size / CHUNK_SIZE + 1; +int lastChunkSize = size % CHUNK_SIZE; +int[][] array = new int[chunkCount][]; +if (array.length > 0) { + for (int i = 0; i < chunkCount - 1; i++) { +array[i] = new int[CHUNK_SIZE]; + } + array[chunkCount - 1] = new int[lastChunkSize]; +} +return array; + } + + private static void copyChunkedArray(int[][] oldArray, int[][] newArray) { +// Copy all but the last (maybe partial) chunk from the old array +if (oldArray.length > 1) { + System.arraycopy(oldArray, 0, newArray, 0, oldArray.length - 1); +} +int[] lastCopyChunk = oldArray[oldArray.length - 1]; +System.arraycopy(lastCopyChunk, 0, newArray[oldArray.length - 1], 0, lastCopyChunk.length); Review Comment: Thank you for the thorough explanation! Follow-up question: Why can't we shallowly copy even the last array? If we expected the old reader to keep functioning as before, they I agree we would need to deep copy the last array, but I think that's already not the case with the taxo reader because of the way the caches are reused after a refresh. Wouldn't it be safe to keep working with `oldArray`? ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -38,27 +38,49 @@ * @lucene.experimental */ class TaxonomyIndexArrays extends ParallelTaxonomyArrays implements Accountable { + private static final int CHUNK_SIZE = 8192; - private final int[] parents; + private final ChunkedArray parents; // the following two arrays are lazily initialized. note that we only keep a // single boolean member as volatile, instead of declaring the arrays // volatile. the code guarantees that only after the boolean is set to true, // the arrays are returned. private volatile boolean initializedChildren = false; - private int[] children, siblings; + private ChunkedArray children, siblings; + + private static class ChunkedArray extends ParallelTaxonomyArrays.IntArray { +private final int[][] values; + +private ChunkedArray(int[][] values) { + this.values = values; +} + +@Override +public int get(int i) { + return values[i / CHUNK_SIZE][i % CHUNK_SIZE]; +} + +public void set(int i, int val) { + values[i / CHUNK_SIZE][i % CHUNK_SIZE] = val; +} + +@Override +public int length() { + return (values.length - 1) * CHUNK_SIZE + values[values.length - 1].length; +} + } /** Used by {@link #add(int, int)} after the array grew. */ - private TaxonomyIndexArrays(int[] parents) { -this.parents = parents; + private TaxonomyIndexArrays(int[][] parents) { +this.parents = new ChunkedArray(parents); } public TaxonomyIndexArrays(IndexReader reader) throws IOException { -parents = new int[reader.maxDoc()]; -if (parents.length > 0) { - initParents(reader, 0); - parents[0] = TaxonomyReader.INVALID_ORDINAL; -} +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); Review Comment: I'm not sure one way is better than the other either, so let's keep it as is. ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray
Re: [PR] Split taxonomy arrays across chunks [lucene]
stefanvodita commented on PR #12995: URL: https://github.com/apache/lucene/pull/12995#issuecomment-1880461956 As far as testing, can we add some unit tests that allocate more than one chunk and exercise the new functionality? Should we also run some benchmarks to understand if there's any sort of performance regression? -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
stefanvodita commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444224140 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray(copyFrom.parents.values, parentArray); +initParents(parentArray, reader, copyFrom.parents.length()); +parents = new ChunkedArray(parentArray); if (copyFrom.initializedChildren) { initChildrenSiblings(copyFrom); } } + private static int[][] allocateChunkedArray(int size) { +int chunkCount = size / CHUNK_SIZE + 1; +int lastChunkSize = size % CHUNK_SIZE; +int[][] array = new int[chunkCount][]; +if (array.length > 0) { + for (int i = 0; i < chunkCount - 1; i++) { +array[i] = new int[CHUNK_SIZE]; + } + array[chunkCount - 1] = new int[lastChunkSize]; +} +return array; + } + + private static void copyChunkedArray(int[][] oldArray, int[][] newArray) { +// Copy all but the last (maybe partial) chunk from the old array +if (oldArray.length > 1) { + System.arraycopy(oldArray, 0, newArray, 0, oldArray.length - 1); +} +int[] lastCopyChunk = oldArray[oldArray.length - 1]; +System.arraycopy(lastCopyChunk, 0, newArray[oldArray.length - 1], 0, lastCopyChunk.length); Review Comment: Actually, I guess the real concern here is that the last array is not properly sized, so always deep-copying it makes sense, since it's capacity requirements will have always changed. -- 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
Re: [PR] Split taxonomy arrays across chunks [lucene]
stefanvodita commented on code in PR #12995: URL: https://github.com/apache/lucene/pull/12995#discussion_r1444224140 ## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java: ## @@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, TaxonomyIndexArrays copyFrom) thr // it may be caused if e.g. the taxonomy segments were merged, and so an updated // NRT reader was obtained, even though nothing was changed. this is not very likely // to happen. -int[] copyParents = copyFrom.parents(); -this.parents = new int[reader.maxDoc()]; -System.arraycopy(copyParents, 0, parents, 0, copyParents.length); -initParents(reader, copyParents.length); - +int[][] parentArray = allocateChunkedArray(reader.maxDoc()); +copyChunkedArray(copyFrom.parents.values, parentArray); +initParents(parentArray, reader, copyFrom.parents.length()); +parents = new ChunkedArray(parentArray); if (copyFrom.initializedChildren) { initChildrenSiblings(copyFrom); } } + private static int[][] allocateChunkedArray(int size) { +int chunkCount = size / CHUNK_SIZE + 1; +int lastChunkSize = size % CHUNK_SIZE; +int[][] array = new int[chunkCount][]; +if (array.length > 0) { + for (int i = 0; i < chunkCount - 1; i++) { +array[i] = new int[CHUNK_SIZE]; + } + array[chunkCount - 1] = new int[lastChunkSize]; +} +return array; + } + + private static void copyChunkedArray(int[][] oldArray, int[][] newArray) { +// Copy all but the last (maybe partial) chunk from the old array +if (oldArray.length > 1) { + System.arraycopy(oldArray, 0, newArray, 0, oldArray.length - 1); +} +int[] lastCopyChunk = oldArray[oldArray.length - 1]; +System.arraycopy(lastCopyChunk, 0, newArray[oldArray.length - 1], 0, lastCopyChunk.length); Review Comment: Actually, I guess the real concern here is that the last array is not properly sized, so always deep-copying it makes sense, since it's capacity requirements will have always changed when we get to this point. -- 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