1 package org.apache.lucene.util;
4 * Licensed to the Apache Software Foundation (ASF) under one or more
5 * contributor license agreements. See the NOTICE file distributed with
6 * this work for additional information regarding copyright ownership.
7 * The ASF licenses this file to You under the Apache License, Version 2.0
8 * (the "License"); you may not use this file except in compliance with
9 * the License. You may obtain a copy of the License at
11 * http://www.apache.org/licenses/LICENSE-2.0
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
20 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
21 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT;
22 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
24 import java.util.Arrays;
25 import java.util.Comparator;
26 import java.util.concurrent.atomic.AtomicLong;
28 import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
31 * {@link BytesRefHash} is a special purpose hash-map like data-structure
32 * optimized for {@link BytesRef} instances. BytesRefHash maintains mappings of
33 * byte arrays to ordinal (Map<BytesRef,int>) storing the hashed bytes
34 * efficiently in continuous storage. The mapping to the ordinal is
35 * encapsulated inside {@link BytesRefHash} and is guaranteed to be increased
36 * for each added {@link BytesRef}.
39 * Note: The maximum capacity {@link BytesRef} instance passed to
40 * {@link #add(BytesRef)} must not be longer than {@link ByteBlockPool#BYTE_BLOCK_SIZE}-2.
41 * The internal storage is limited to 2GB total byte storage.
46 public final class BytesRefHash {
48 public static final int DEFAULT_CAPACITY = 16;
50 // the following fields are needed by comparator,
51 // so package private to prevent access$-methods:
52 final ByteBlockPool pool;
55 private final BytesRef scratch1 = new BytesRef();
57 private int hashHalfSize;
60 private int lastCount = -1;
62 private final BytesStartArray bytesStartArray;
63 private AtomicLong bytesUsed;
66 * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a
67 * {@link DirectAllocator}.
69 public BytesRefHash() {
70 this(new ByteBlockPool(new DirectAllocator()));
74 * Creates a new {@link BytesRefHash}
76 public BytesRefHash(ByteBlockPool pool) {
77 this(pool, DEFAULT_CAPACITY, new DirectBytesStartArray(DEFAULT_CAPACITY));
81 * Creates a new {@link BytesRefHash}
83 public BytesRefHash(ByteBlockPool pool, int capacity,
84 BytesStartArray bytesStartArray) {
86 hashHalfSize = hashSize >> 1;
87 hashMask = hashSize - 1;
89 ords = new int[hashSize];
90 Arrays.fill(ords, -1);
91 this.bytesStartArray = bytesStartArray;
92 bytesStart = bytesStartArray.init();
93 bytesUsed = bytesStartArray.bytesUsed() == null? new AtomicLong(0) : bytesStartArray.bytesUsed();;
94 bytesUsed.addAndGet(hashSize * RamUsageEstimator.NUM_BYTES_INT);
98 * Returns the number of {@link BytesRef} values in this {@link BytesRefHash}.
100 * @return the number of {@link BytesRef} values in this {@link BytesRefHash}.
107 * Populates and returns a {@link BytesRef} with the bytes for the given ord.
109 * Note: the given ord must be a positive integer less that the current size (
114 * @param ref the {@link BytesRef} to populate
116 * @return the given BytesRef instance populated with the bytes for the given ord
118 public BytesRef get(int ord, BytesRef ref) {
119 assert bytesStart != null : "bytesStart is null - not initialized";
120 assert ord < bytesStart.length: "ord exceeds byteStart len: " + bytesStart.length;
121 return pool.setBytesRef(ref, bytesStart[ord]);
125 * Returns the ords array in arbitrary order. Valid ords start at offset of 0
126 * and end at a limit of {@link #size()} - 1
128 * Note: This is a destructive operation. {@link #clear()} must be called in
129 * order to reuse this {@link BytesRefHash} instance.
132 public int[] compact() {
133 assert bytesStart != null : "Bytesstart is null - not initialized";
135 for (int i = 0; i < hashSize; i++) {
138 ords[upto] = ords[i];
145 assert upto == count;
151 * Returns the values array sorted by the referenced byte values.
153 * Note: This is a destructive operation. {@link #clear()} must be called in
154 * order to reuse this {@link BytesRefHash} instance.
158 * the {@link Comparator} used for sorting
160 public int[] sort(final Comparator<BytesRef> comp) {
161 final int[] compact = compact();
162 new SorterTemplate() {
164 protected void swap(int i, int j) {
165 final int o = compact[i];
166 compact[i] = compact[j];
171 protected int compare(int i, int j) {
172 final int ord1 = compact[i], ord2 = compact[j];
173 assert bytesStart.length > ord1 && bytesStart.length > ord2;
174 return comp.compare(pool.setBytesRef(scratch1, bytesStart[ord1]),
175 pool.setBytesRef(scratch2, bytesStart[ord2]));
179 protected void setPivot(int i) {
180 final int ord = compact[i];
181 assert bytesStart.length > ord;
182 pool.setBytesRef(pivot, bytesStart[ord]);
186 protected int comparePivot(int j) {
187 final int ord = compact[j];
188 assert bytesStart.length > ord;
189 return comp.compare(pivot,
190 pool.setBytesRef(scratch2, bytesStart[ord]));
193 private final BytesRef pivot = new BytesRef(),
194 scratch1 = new BytesRef(), scratch2 = new BytesRef();
195 }.quickSort(0, count - 1);
199 private boolean equals(int ord, BytesRef b) {
200 return pool.setBytesRef(scratch1, bytesStart[ord]).bytesEquals(b);
203 private boolean shrink(int targetSize) {
204 // Cannot use ArrayUtil.shrink because we require power
206 int newSize = hashSize;
207 while (newSize >= 8 && newSize / 4 > targetSize) {
210 if (newSize != hashSize) {
211 bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
212 * -(hashSize - newSize));
214 ords = new int[hashSize];
215 Arrays.fill(ords, -1);
216 hashHalfSize = newSize / 2;
217 hashMask = newSize - 1;
225 * Clears the {@link BytesRef} which maps to the given {@link BytesRef}
227 public void clear(boolean resetPool) {
231 pool.dropBuffersAndReset();
233 bytesStart = bytesStartArray.clear();
234 if (lastCount != -1 && shrink(lastCount)) {
235 // shrink clears the hash entries
238 Arrays.fill(ords, -1);
241 public void clear() {
246 * Closes the BytesRefHash and releases all internally used memory
248 public void close() {
251 bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT
256 * Adds a new {@link BytesRef}
260 * @return the ord the given bytes are hashed if there was no mapping for the
261 * given bytes, otherwise <code>(-(ord)-1)</code>. This guarantees
262 * that the return value will always be >= 0 if the given bytes
263 * haven't been hashed before.
265 * @throws MaxBytesLengthExceededException
266 * if the given bytes are > 2 +
267 * {@link ByteBlockPool#BYTE_BLOCK_SIZE}
269 public int add(BytesRef bytes) {
270 return add(bytes, bytes.hashCode());
274 * Adds a new {@link BytesRef} with a pre-calculated hash code.
279 * the bytes hash code
282 * Hashcode is defined as:
286 * for (int i = offset; i < offset + length; i++) {
287 * hash = 31 * hash + bytes[i];
291 * @return the ord the given bytes are hashed if there was no mapping for the
292 * given bytes, otherwise <code>(-(ord)-1)</code>. This guarantees
293 * that the return value will always be >= 0 if the given bytes
294 * haven't been hashed before.
296 * @throws MaxBytesLengthExceededException
297 * if the given bytes are >
298 * {@link ByteBlockPool#BYTE_BLOCK_SIZE} - 2
300 public int add(BytesRef bytes, int code) {
301 assert bytesStart != null : "Bytesstart is null - not initialized";
302 final int length = bytes.length;
304 int hashPos = code & hashMask;
305 int e = ords[hashPos];
306 if (e != -1 && !equals(e, bytes)) {
307 // Conflict: keep searching different locations in
309 final int inc = ((code >> 8) + code) | 1;
312 hashPos = code & hashMask;
314 } while (e != -1 && !equals(e, bytes));
319 final int len2 = 2 + bytes.length;
320 if (len2 + pool.byteUpto > BYTE_BLOCK_SIZE) {
321 if (len2 > BYTE_BLOCK_SIZE) {
322 throw new MaxBytesLengthExceededException("bytes can be at most "
323 + (BYTE_BLOCK_SIZE - 2) + " in length; got " + bytes.length);
327 final byte[] buffer = pool.buffer;
328 final int bufferUpto = pool.byteUpto;
329 if (count >= bytesStart.length) {
330 bytesStart = bytesStartArray.grow();
331 assert count < bytesStart.length + 1 : "count: " + count + " len: "
336 bytesStart[e] = bufferUpto + pool.byteOffset;
338 // We first encode the length, followed by the
339 // bytes. Length is encoded as vInt, but will consume
340 // 1 or 2 bytes at most (we reject too-long terms,
343 // 1 byte to store length
344 buffer[bufferUpto] = (byte) length;
345 pool.byteUpto += length + 1;
346 assert length >= 0: "Length must be positive: " + length;
347 System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 1,
350 // 2 byte to store length
351 buffer[bufferUpto] = (byte) (0x80 | (length & 0x7f));
352 buffer[bufferUpto + 1] = (byte) ((length >> 7) & 0xff);
353 pool.byteUpto += length + 2;
354 System.arraycopy(bytes.bytes, bytes.offset, buffer, bufferUpto + 2,
357 assert ords[hashPos] == -1;
360 if (count == hashHalfSize) {
361 rehash(2 * hashSize, true);
368 public int addByPoolOffset(int offset) {
369 assert bytesStart != null : "Bytesstart is null - not initialized";
372 int hashPos = offset & hashMask;
373 int e = ords[hashPos];
374 if (e != -1 && bytesStart[e] != offset) {
375 // Conflict: keep searching different locations in
377 final int inc = ((code >> 8) + code) | 1;
380 hashPos = code & hashMask;
382 } while (e != -1 && bytesStart[e] != offset);
386 if (count >= bytesStart.length) {
387 bytesStart = bytesStartArray.grow();
388 assert count < bytesStart.length + 1 : "count: " + count + " len: "
392 bytesStart[e] = offset;
393 assert ords[hashPos] == -1;
396 if (count == hashHalfSize) {
397 rehash(2 * hashSize, false);
405 * Called when hash is too small (> 50% occupied) or too large (< 20%
408 private void rehash(final int newSize, boolean hashOnData) {
409 final int newMask = newSize - 1;
410 bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * (newSize));
411 final int[] newHash = new int[newSize];
412 Arrays.fill(newHash, -1);
413 for (int i = 0; i < hashSize; i++) {
414 final int e0 = ords[i];
418 final int off = bytesStart[e0];
419 final int start = off & BYTE_BLOCK_MASK;
420 final byte[] bytes = pool.buffers[off >> BYTE_BLOCK_SHIFT];
424 if ((bytes[start] & 0x80) == 0) {
429 len = (bytes[start] & 0x7f) + ((bytes[start + 1] & 0xff) << 7);
433 final int endPos = pos + len;
434 while (pos < endPos) {
435 code = BytesRef.HASH_PRIME * code + bytes[pos++];
438 code = bytesStart[e0];
441 int hashPos = code & newMask;
443 if (newHash[hashPos] != -1) {
444 final int inc = ((code >> 8) + code) | 1;
447 hashPos = code & newMask;
448 } while (newHash[hashPos] != -1);
450 newHash[hashPos] = e0;
455 bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * (-ords.length));
458 hashHalfSize = newSize / 2;
462 * reinitializes the {@link BytesRefHash} after a previous {@link #clear()}
463 * call. If {@link #clear()} has not been called previously this method has no
466 public void reinit() {
467 if (bytesStart == null) {
468 bytesStart = bytesStartArray.init();
472 ords = new int[hashSize];
473 bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT * hashSize);
478 * Returns the bytesStart offset into the internally used
479 * {@link ByteBlockPool} for the given ord
483 * @return the bytesStart offset into the internally used
484 * {@link ByteBlockPool} for the given ord
486 public int byteStart(int ord) {
487 assert bytesStart != null : "Bytesstart is null - not initialized";
488 assert ord >= 0 && ord < count : ord;
489 return bytesStart[ord];
493 * Thrown if a {@link BytesRef} exceeds the {@link BytesRefHash} limit of
494 * {@link ByteBlockPool#BYTE_BLOCK_SIZE}-2.
496 @SuppressWarnings("serial")
497 public static class MaxBytesLengthExceededException extends RuntimeException {
498 MaxBytesLengthExceededException(String message) {
503 public abstract static class BytesStartArray {
505 * Initializes the BytesStartArray. This call will allocate memory
507 * @return the initialized bytes start array
509 public abstract int[] init();
512 * Grows the {@link BytesStartArray}
514 * @return the grown array
516 public abstract int[] grow();
519 * clears the {@link BytesStartArray} and returns the cleared instance.
521 * @return the cleared instance, this might be <code>null</code>
523 public abstract int[] clear();
526 * A {@link AtomicLong} reference holding the number of bytes used by this
527 * {@link BytesStartArray}. The {@link BytesRefHash} uses this reference to
528 * track it memory usage
530 * @return a {@link AtomicLong} reference holding the number of bytes used
531 * by this {@link BytesStartArray}.
533 public abstract AtomicLong bytesUsed();
537 * A direct {@link BytesStartArray} that tracks all memory allocation using an {@link AtomicLong} instance.
539 public static class TrackingDirectBytesStartArray extends BytesStartArray {
540 protected final int initSize;
541 private int[] bytesStart;
542 protected final AtomicLong bytesUsed;
544 public TrackingDirectBytesStartArray(int initSize, AtomicLong bytesUsed) {
545 this.initSize = initSize;
546 this.bytesUsed = bytesUsed;
550 public int[] clear() {
551 if (bytesStart != null) {
552 bytesUsed.addAndGet(-bytesStart.length * RamUsageEstimator.NUM_BYTES_INT);
554 return bytesStart = null;
558 public int[] grow() {
559 assert bytesStart != null;
560 final int oldSize = bytesStart.length;
561 bytesStart = ArrayUtil.grow(bytesStart, bytesStart.length + 1);
562 bytesUsed.addAndGet((bytesStart.length - oldSize) * RamUsageEstimator.NUM_BYTES_INT);
567 public int[] init() {
568 bytesStart = new int[ArrayUtil.oversize(initSize,
569 RamUsageEstimator.NUM_BYTES_INT)];
570 bytesUsed.addAndGet((bytesStart.length) * RamUsageEstimator.NUM_BYTES_INT);
575 public AtomicLong bytesUsed() {
580 public static class DirectBytesStartArray extends BytesStartArray {
581 protected final int initSize;
582 private int[] bytesStart;
583 private final AtomicLong bytesUsed;
585 public DirectBytesStartArray(int initSize) {
586 this.bytesUsed = new AtomicLong(0);
587 this.initSize = initSize;
592 public int[] clear() {
593 return bytesStart = null;
597 public int[] grow() {
598 assert bytesStart != null;
599 return bytesStart = ArrayUtil.grow(bytesStart, bytesStart.length + 1);
603 public int[] init() {
604 return bytesStart = new int[ArrayUtil.oversize(initSize,
605 RamUsageEstimator.NUM_BYTES_INT)];
609 public AtomicLong bytesUsed() {