From 00f08652c000dc4700761fdc06c6d37e4084f514 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Wed, 19 Feb 2020 20:09:07 +0800 Subject: [PATCH] Introduce BitSetRecyclable that leverage with netty recycler --- .../main/resources/pulsar/suppressions.xml | 1 + .../mledger/impl/ManagedCursorImpl.java | 36 +- .../bookkeeper/mledger/impl/PositionImpl.java | 8 +- .../mledger/impl/ManagedCursorTest.java | 2 +- .../pulsar/broker/service/Consumer.java | 4 +- ...sistentAcknowledgmentsGroupingTracker.java | 3 +- .../pulsar/common/protocol/Commands.java | 7 +- .../util/collections/BitSetRecyclable.java | 1169 ++++++++++++++++- ...va => BitSetRecyclableRecyclableTest.java} | 13 +- 9 files changed, 1208 insertions(+), 35 deletions(-) rename pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/{BitSetRecyclableTest.java => BitSetRecyclableRecyclableTest.java} (76%) diff --git a/buildtools/src/main/resources/pulsar/suppressions.xml b/buildtools/src/main/resources/pulsar/suppressions.xml index 7fd94d913f377a..d6d38a2dfa0a6b 100644 --- a/buildtools/src/main/resources/pulsar/suppressions.xml +++ b/buildtools/src/main/resources/pulsar/suppressions.xml @@ -40,4 +40,5 @@ + diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f09620688ff13d..80b83cc2a09325 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -44,7 +44,6 @@ import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.BitSet; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -156,7 +155,7 @@ public class ManagedCursorImpl implements ManagedCursor { // Maintain the deletion status for batch messages // (ledgerId, entryId) -> deletion indexes - private final ConcurrentSkipListMap batchDeletedIndexes; + private final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -413,7 +412,7 @@ private void recoverBatchDeletedIndexes (List propertie } callback.markDeleteFailed( new ManagedLedgerException("Reset cursor in progress - unable to mark delete position " - + ((PositionImpl) position).toString()), + + position.toString()), ctx); } @@ -1518,10 +1518,12 @@ public void asyncMarkDelete(final Position position, Map propertie if (config.isDeletionAtBatchIndexLevelEnabled()) { if (newPosition.ackSet != null) { - batchDeletedIndexes.put(newPosition, newPosition.ackSet); + batchDeletedIndexes.put(newPosition, BitSetRecyclable.create().resetWords(newPosition.ackSet)); newPosition = ledger.getPreviousPosition(newPosition); } - batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition).clear(); + Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition); + subMap.values().forEach(BitSetRecyclable::recycle); + subMap.clear(); } else if (newPosition.ackSet != null) { callback.markDeleteFailed(new ManagedLedgerException("Batch ack set not support"), ctx); return; @@ -1619,7 +1621,9 @@ public void operationComplete() { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true).clear(); + Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); + subMap.values().forEach(BitSetRecyclable::recycle); + subMap.clear(); } } finally { lock.writeLock().unlock(); @@ -1756,7 +1760,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) || position.compareTo(markDeletePosition) <= 0) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } if (log.isDebugEnabled()) { log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); @@ -1765,7 +1769,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (position.ackSet == null) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make // the RangeSet recognize the "continuity" between adjacent Positions @@ -1779,14 +1783,16 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb individualDeletedMessages); } } else if (config.isDeletionAtBatchIndexLevelEnabled()) { - BitSet bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSet.valueOf(position.ackSet.toLongArray())); - bitSet.and(position.ackSet); + BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSetRecyclable.create().resetWords(position.ackSet)); + BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); + bitSet.and(givenBitSet); + givenBitSet.recycle(); if (bitSet.isEmpty()) { PositionImpl previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); ++messagesConsumedCounter; - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } } } @@ -2331,9 +2337,9 @@ private List buildBatchEntryDeletio MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo .newBuilder(); List result = Lists.newArrayList(); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < config.getMaxBatchDeletedIndexToPersist()) { - Map.Entry entry = iterator.next(); + Map.Entry entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); @@ -2736,7 +2742,7 @@ public void trimDeletedEntries(List entries) { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - BitSet bitSet = batchDeletedIndexes.get(position); + BitSetRecyclable bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java index 7995840d5cf9c4..1c35ef1f6325a3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java @@ -26,13 +26,11 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; -import java.util.BitSet; - public class PositionImpl implements Position, Comparable { protected long ledgerId; protected long entryId; - protected BitSet ackSet; + protected long[] ackSet; public static final PositionImpl earliest = new PositionImpl(-1, -1); public static final PositionImpl latest = new PositionImpl(Long.MAX_VALUE, Long.MAX_VALUE); @@ -52,7 +50,7 @@ public PositionImpl(long ledgerId, long entryId) { this.entryId = entryId; } - public PositionImpl(long ledgerId, long entryId, BitSet ackSet) { + public PositionImpl(long ledgerId, long entryId, long[] ackSet) { this.ledgerId = ledgerId; this.entryId = entryId; this.ackSet = ackSet; @@ -67,7 +65,7 @@ public static PositionImpl get(long ledgerId, long entryId) { return new PositionImpl(ledgerId, entryId); } - public static PositionImpl get(long ledgerId, long entryId, BitSet ackSet) { + public static PositionImpl get(long ledgerId, long entryId, long[] ackSet) { return new PositionImpl(ledgerId, entryId, ackSet); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 0b6a9d8682e6dd..60ce5997aa7fd3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3138,7 +3138,7 @@ private void deleteBatchIndex(ManagedCursor cursor, Position position, int batch deleteIndexes.forEach(intRange -> { bitSet.clear(intRange.getStart(), intRange.getEnd() + 1); }); - pos.ackSet = bitSet; + pos.ackSet = bitSet.toLongArray(); cursor.asyncDelete(pos, new DeleteCallback() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java index 8a2e64d6949988..541b4bd2898315 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java @@ -373,7 +373,7 @@ void messageAcked(CommandAck ack) { if (ack.getMessageIdCount() == 1) { MessageIdData msgId = ack.getMessageId(0); if (msgId.getAckSetCount() > 0) { - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), SafeCollectionUtils.longListToArray(msgId.getAckSetList())); } else { position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); } @@ -386,7 +386,7 @@ void messageAcked(CommandAck ack) { MessageIdData msgId = ack.getMessageId(i); PositionImpl position; if (msgId.getAckSetCount() > 0) { - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), SafeCollectionUtils.longListToArray(msgId.getAckSetList())); } else { position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 67bd147b551394..045419593b982c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -40,7 +40,6 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentBitSet; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; /** @@ -190,7 +189,7 @@ private boolean doImmediateBatchIndexAck(BatchMessageIdImpl msgId, int batchInde if (cnx == null) { return false; } - ConcurrentBitSet bitSet = new ConcurrentBitSet(batchSize); + BitSetRecyclable bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); if (ackType == AckType.Cumulative) { bitSet.clear(0, batchIndex + 1); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 19c70b0947b4a8..32390477ad1656 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -30,7 +30,6 @@ import io.netty.buffer.Unpooled; import java.io.IOException; -import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; @@ -119,6 +118,7 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.SafeCollectionUtils; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream; import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream; @@ -901,12 +901,12 @@ public static ByteBuf newMultiMessageAck(long consumerId, return res; } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType, ValidationError validationError, Map properties) { return newAck(consumerId, ledgerId, entryId, ackSet, ackType, validationError, properties, 0, 0); } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType, ValidationError validationError, Map properties, long txnIdLeastBits, long txnIdMostBits) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); @@ -937,6 +937,7 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSe ByteBuf res = serializeWithSize(BaseCommand.newBuilder().setType(Type.ACK).setAck(ack)); ack.recycle(); + ackSet.recycle(); ackBuilder.recycle(); messageIdDataBuilder.recycle(); messageIdData.recycle(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java index 649f5cdb80575a..285416ac606b24 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java @@ -21,14 +21,1169 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; +import java.io.*; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.LongBuffer; +import java.util.Arrays; import java.util.BitSet; /** - * BitSet leverage netty recycler. + * This this copy of {@link BitSet}. + * Provides {@link BitSetRecyclable#resetWords(long[])} method and leverage with netty recycler. */ -public class BitSetRecyclable extends BitSet { +public class BitSetRecyclable implements Cloneable, java.io.Serializable { + /* + * BitSets are packed into arrays of "words." Currently a word is + * a long, which consists of 64 bits, requiring 6 address bits. + * The choice of word size is determined purely by performance concerns. + */ + private final static int ADDRESS_BITS_PER_WORD = 6; + private final static int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD; + private final static int BIT_INDEX_MASK = BITS_PER_WORD - 1; - private final Handle recyclerHandle; + /* Used to shift left or right for a partial word mask */ + private static final long WORD_MASK = 0xffffffffffffffffL; + + /** + * @serialField bits long[] + * + * The bits in this BitSet. The ith bit is stored in bits[i/64] at + * bit position i % 64 (where bit position 0 refers to the least + * significant bit and 63 refers to the most significant bit). + */ + private static final ObjectStreamField[] serialPersistentFields = { + new ObjectStreamField("bits", long[].class), + }; + + /** + * The internal field corresponding to the serialField "bits". + */ + private long[] words; + + /** + * The number of words in the logical size of this BitSet. + */ + private transient int wordsInUse = 0; + + /** + * Whether the size of "words" is user-specified. If so, we assume + * the user knows what he's doing and try harder to preserve it. + */ + private transient boolean sizeIsSticky = false; + + /* use serialVersionUID from JDK 1.0.2 for interoperability */ + private static final long serialVersionUID = 7997698588986878753L; + + /** + * Given a bit index, return word index containing it. + */ + private static int wordIndex(int bitIndex) { + return bitIndex >> ADDRESS_BITS_PER_WORD; + } + + /** + * Every public method must preserve these invariants. + */ + private void checkInvariants() { + assert(wordsInUse == 0 || words[wordsInUse - 1] != 0); + assert(wordsInUse >= 0 && wordsInUse <= words.length); + assert(wordsInUse == words.length || words[wordsInUse] == 0); + } + + /** + * Sets the field wordsInUse to the logical size in words of the bit set. + * WARNING:This method assumes that the number of words actually in use is + * less than or equal to the current value of wordsInUse! + */ + private void recalculateWordsInUse() { + // Traverse the bitset until a used word is found + int i; + for (i = wordsInUse-1; i >= 0; i--) + if (words[i] != 0) + break; + + wordsInUse = i+1; // The new logical size + } + + /** + * Creates a new bit set. All bits are initially {@code false}. + */ + public BitSetRecyclable() { + initWords(BITS_PER_WORD); + sizeIsSticky = false; + } + + /** + * Creates a bit set whose initial size is large enough to explicitly + * represent bits with indices in the range {@code 0} through + * {@code nbits-1}. All bits are initially {@code false}. + * + * @param nbits the initial size of the bit set + * @throws NegativeArraySizeException if the specified initial size + * is negative + */ + public BitSetRecyclable(int nbits) { + // nbits can't be negative; size 0 is OK + if (nbits < 0) + throw new NegativeArraySizeException("nbits < 0: " + nbits); + + initWords(nbits); + sizeIsSticky = true; + } + + private void initWords(int nbits) { + words = new long[wordIndex(nbits-1) + 1]; + } + + /** + * Creates a bit set using words as the internal representation. + * The last word (if there is one) must be non-zero. + */ + private BitSetRecyclable(long[] words) { + this.words = words; + this.wordsInUse = words.length; + checkInvariants(); + } + + /** + * Returns a new bit set containing all the bits in the given long array. + * + *

More precisely, + *
{@code BitSet.valueOf(longs).get(n) == ((longs[n/64] & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * longs.length}. + * + *

This method is equivalent to + * {@code BitSet.valueOf(LongBuffer.wrap(longs))}. + * + * @param longs a long array containing a little-endian representation + * of a sequence of bits to be used as the initial bits of the + * new bit set + * @return a {@code BitSet} containing all the bits in the long array + * @since 1.7 + */ + public static BitSetRecyclable valueOf(long[] longs) { + int n; + for (n = longs.length; n > 0 && longs[n - 1] == 0; n--) + ; + return new BitSetRecyclable(Arrays.copyOf(longs, n)); + } + + /** + * Returns a new bit set containing all the bits in the given long + * buffer between its position and limit. + * + *

More precisely, + *
{@code BitSet.valueOf(lb).get(n) == ((lb.get(lb.position()+n/64) & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * lb.remaining()}. + * + *

The long buffer is not modified by this method, and no + * reference to the buffer is retained by the bit set. + * + * @param lb a long buffer containing a little-endian representation + * of a sequence of bits between its position and limit, to be + * used as the initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the buffer in the + * specified range + * @since 1.7 + */ + public static BitSetRecyclable valueOf(LongBuffer lb) { + lb = lb.slice(); + int n; + for (n = lb.remaining(); n > 0 && lb.get(n - 1) == 0; n--) + ; + long[] words = new long[n]; + lb.get(words); + return new BitSetRecyclable(words); + } + + /** + * Returns a new bit set containing all the bits in the given byte array. + * + *

More precisely, + *
{@code BitSet.valueOf(bytes).get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bytes.length}. + * + *

This method is equivalent to + * {@code BitSet.valueOf(ByteBuffer.wrap(bytes))}. + * + * @param bytes a byte array containing a little-endian + * representation of a sequence of bits to be used as the + * initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the byte array + * @since 1.7 + */ + public static BitSetRecyclable valueOf(byte[] bytes) { + return BitSetRecyclable.valueOf(ByteBuffer.wrap(bytes)); + } + + /** + * Returns a new bit set containing all the bits in the given byte + * buffer between its position and limit. + * + *

More precisely, + *
{@code BitSet.valueOf(bb).get(n) == ((bb.get(bb.position()+n/8) & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bb.remaining()}. + * + *

The byte buffer is not modified by this method, and no + * reference to the buffer is retained by the bit set. + * + * @param bb a byte buffer containing a little-endian representation + * of a sequence of bits between its position and limit, to be + * used as the initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the buffer in the + * specified range + * @since 1.7 + */ + public static BitSetRecyclable valueOf(ByteBuffer bb) { + bb = bb.slice().order(ByteOrder.LITTLE_ENDIAN); + int n; + for (n = bb.remaining(); n > 0 && bb.get(n - 1) == 0; n--) + ; + long[] words = new long[(n + 7) / 8]; + bb.limit(n); + int i = 0; + while (bb.remaining() >= 8) + words[i++] = bb.getLong(); + for (int remaining = bb.remaining(), j = 0; j < remaining; j++) + words[i] |= (bb.get() & 0xffL) << (8 * j); + return new BitSetRecyclable(words); + } + + /** + * Returns a new byte array containing all the bits in this bit set. + * + *

More precisely, if + *
{@code byte[] bytes = s.toByteArray();} + *
then {@code bytes.length == (s.length()+7)/8} and + *
{@code s.get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bytes.length}. + * + * @return a byte array containing a little-endian representation + * of all the bits in this bit set + * @since 1.7 + */ + public byte[] toByteArray() { + int n = wordsInUse; + if (n == 0) + return new byte[0]; + int len = 8 * (n-1); + for (long x = words[n - 1]; x != 0; x >>>= 8) + len++; + byte[] bytes = new byte[len]; + ByteBuffer bb = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < n - 1; i++) + bb.putLong(words[i]); + for (long x = words[n - 1]; x != 0; x >>>= 8) + bb.put((byte) (x & 0xff)); + return bytes; + } + + /** + * Returns a new long array containing all the bits in this bit set. + * + *

More precisely, if + *
{@code long[] longs = s.toLongArray();} + *
then {@code longs.length == (s.length()+63)/64} and + *
{@code s.get(n) == ((longs[n/64] & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * longs.length}. + * + * @return a long array containing a little-endian representation + * of all the bits in this bit set + * @since 1.7 + */ + public long[] toLongArray() { + return Arrays.copyOf(words, wordsInUse); + } + + /** + * Ensures that the BitSet can hold enough words. + * @param wordsRequired the minimum acceptable number of words. + */ + private void ensureCapacity(int wordsRequired) { + if (words.length < wordsRequired) { + // Allocate larger of doubled size or required size + int request = Math.max(2 * words.length, wordsRequired); + words = Arrays.copyOf(words, request); + sizeIsSticky = false; + } + } + + /** + * Ensures that the BitSet can accommodate a given wordIndex, + * temporarily violating the invariants. The caller must + * restore the invariants before returning to the user, + * possibly using recalculateWordsInUse(). + * @param wordIndex the index to be accommodated. + */ + private void expandTo(int wordIndex) { + int wordsRequired = wordIndex+1; + if (wordsInUse < wordsRequired) { + ensureCapacity(wordsRequired); + wordsInUse = wordsRequired; + } + } + + /** + * Checks that fromIndex ... toIndex is a valid range of bit indices. + */ + private static void checkRange(int fromIndex, int toIndex) { + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + if (toIndex < 0) + throw new IndexOutOfBoundsException("toIndex < 0: " + toIndex); + if (fromIndex > toIndex) + throw new IndexOutOfBoundsException("fromIndex: " + fromIndex + + " > toIndex: " + toIndex); + } + + /** + * Sets the bit at the specified index to the complement of its + * current value. + * + * @param bitIndex the index of the bit to flip + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public void flip(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + expandTo(wordIndex); + + words[wordIndex] ^= (1L << bitIndex); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets each bit from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to the complement of its current + * value. + * + * @param fromIndex index of the first bit to flip + * @param toIndex index after the last bit to flip + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void flip(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + int startWordIndex = wordIndex(fromIndex); + int endWordIndex = wordIndex(toIndex - 1); + expandTo(endWordIndex); + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] ^= (firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] ^= firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] ^= WORD_MASK; + + // Handle last word + words[endWordIndex] ^= lastWordMask; + } + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets the bit at the specified index to {@code true}. + * + * @param bitIndex a bit index + * @throws IndexOutOfBoundsException if the specified index is negative + * @since JDK1.0 + */ + public void set(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + expandTo(wordIndex); + + words[wordIndex] |= (1L << bitIndex); // Restores invariants + + checkInvariants(); + } + + /** + * Sets the bit at the specified index to the specified value. + * + * @param bitIndex a bit index + * @param value a boolean value to set + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public void set(int bitIndex, boolean value) { + if (value) + set(bitIndex); + else + clear(bitIndex); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to {@code true}. + * + * @param fromIndex index of the first bit to be set + * @param toIndex index after the last bit to be set + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void set(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + // Increase capacity if necessary + int startWordIndex = wordIndex(fromIndex); + int endWordIndex = wordIndex(toIndex - 1); + expandTo(endWordIndex); + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] |= (firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] |= firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] = WORD_MASK; + + // Handle last word (restores invariants) + words[endWordIndex] |= lastWordMask; + } + + checkInvariants(); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to the specified value. + * + * @param fromIndex index of the first bit to be set + * @param toIndex index after the last bit to be set + * @param value value to set the selected bits to + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void set(int fromIndex, int toIndex, boolean value) { + if (value) + set(fromIndex, toIndex); + else + clear(fromIndex, toIndex); + } + + /** + * Sets the bit specified by the index to {@code false}. + * + * @param bitIndex the index of the bit to be cleared + * @throws IndexOutOfBoundsException if the specified index is negative + * @since JDK1.0 + */ + public void clear(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + if (wordIndex >= wordsInUse) + return; + + words[wordIndex] &= ~(1L << bitIndex); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to {@code false}. + * + * @param fromIndex index of the first bit to be cleared + * @param toIndex index after the last bit to be cleared + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void clear(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + int startWordIndex = wordIndex(fromIndex); + if (startWordIndex >= wordsInUse) + return; + + int endWordIndex = wordIndex(toIndex - 1); + if (endWordIndex >= wordsInUse) { + toIndex = length(); + endWordIndex = wordsInUse - 1; + } + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] &= ~(firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] &= ~firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] = 0; + + // Handle last word + words[endWordIndex] &= ~lastWordMask; + } + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets all of the bits in this BitSet to {@code false}. + * + * @since 1.4 + */ + public void clear() { + while (wordsInUse > 0) + words[--wordsInUse] = 0; + } + + /** + * Returns the value of the bit with the specified index. The value + * is {@code true} if the bit with the index {@code bitIndex} + * is currently set in this {@code BitSet}; otherwise, the result + * is {@code false}. + * + * @param bitIndex the bit index + * @return the value of the bit with the specified index + * @throws IndexOutOfBoundsException if the specified index is negative + */ + public boolean get(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + checkInvariants(); + + int wordIndex = wordIndex(bitIndex); + return (wordIndex < wordsInUse) + && ((words[wordIndex] & (1L << bitIndex)) != 0); + } + + /** + * Returns a new {@code BitSet} composed of bits from this {@code BitSet} + * from {@code fromIndex} (inclusive) to {@code toIndex} (exclusive). + * + * @param fromIndex index of the first bit to include + * @param toIndex index after the last bit to include + * @return a new {@code BitSet} from a range of this {@code BitSet} + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public BitSetRecyclable get(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + checkInvariants(); + + int len = length(); + + // If no set bits in range return empty bitset + if (len <= fromIndex || fromIndex == toIndex) + return new BitSetRecyclable(0); + + // An optimization + if (toIndex > len) + toIndex = len; + + BitSetRecyclable result = new BitSetRecyclable(toIndex - fromIndex); + int targetWords = wordIndex(toIndex - fromIndex - 1) + 1; + int sourceIndex = wordIndex(fromIndex); + boolean wordAligned = ((fromIndex & BIT_INDEX_MASK) == 0); + + // Process all words but the last word + for (int i = 0; i < targetWords - 1; i++, sourceIndex++) + result.words[i] = wordAligned ? words[sourceIndex] : + (words[sourceIndex] >>> fromIndex) | + (words[sourceIndex+1] << -fromIndex); + + // Process the last word + long lastWordMask = WORD_MASK >>> -toIndex; + result.words[targetWords - 1] = + ((toIndex-1) & BIT_INDEX_MASK) < (fromIndex & BIT_INDEX_MASK) + ? /* straddles source words */ + ((words[sourceIndex] >>> fromIndex) | + (words[sourceIndex+1] & lastWordMask) << -fromIndex) + : + ((words[sourceIndex] & lastWordMask) >>> fromIndex); + + // Set wordsInUse correctly + result.wordsInUse = targetWords; + result.recalculateWordsInUse(); + result.checkInvariants(); + + return result; + } + + /** + * Returns the index of the first bit that is set to {@code true} + * that occurs on or after the specified starting index. If no such + * bit exists then {@code -1} is returned. + * + *

To iterate over the {@code true} bits in a {@code BitSet}, + * use the following loop: + * + *

 {@code
+     * for (int i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i+1)) {
+     *     // operate on index i here
+     *     if (i == Integer.MAX_VALUE) {
+     *         break; // or (i+1) would overflow
+     *     }
+     * }}
+ * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next set bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public int nextSetBit(int fromIndex) { + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return -1; + + long word = words[u] & (WORD_MASK << fromIndex); + + while (true) { + if (word != 0) + return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word); + if (++u == wordsInUse) + return -1; + word = words[u]; + } + } + + /** + * Returns the index of the first bit that is set to {@code false} + * that occurs on or after the specified starting index. + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next clear bit + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public int nextClearBit(int fromIndex) { + // Neither spec nor implementation handle bitsets of maximal length. + // See 4816253. + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return fromIndex; + + long word = ~words[u] & (WORD_MASK << fromIndex); + + while (true) { + if (word != 0) + return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word); + if (++u == wordsInUse) + return wordsInUse * BITS_PER_WORD; + word = ~words[u]; + } + } + + /** + * Returns the index of the nearest bit that is set to {@code true} + * that occurs on or before the specified starting index. + * If no such bit exists, or if {@code -1} is given as the + * starting index, then {@code -1} is returned. + * + *

To iterate over the {@code true} bits in a {@code BitSet}, + * use the following loop: + * + *

 {@code
+     * for (int i = bs.length(); (i = bs.previousSetBit(i-1)) >= 0; ) {
+     *     // operate on index i here
+     * }}
+ * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the previous set bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is less + * than {@code -1} + * @since 1.7 + */ + public int previousSetBit(int fromIndex) { + if (fromIndex < 0) { + if (fromIndex == -1) + return -1; + throw new IndexOutOfBoundsException( + "fromIndex < -1: " + fromIndex); + } + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return length() - 1; + + long word = words[u] & (WORD_MASK >>> -(fromIndex+1)); + + while (true) { + if (word != 0) + return (u+1) * BITS_PER_WORD - 1 - Long.numberOfLeadingZeros(word); + if (u-- == 0) + return -1; + word = words[u]; + } + } + + /** + * Returns the index of the nearest bit that is set to {@code false} + * that occurs on or before the specified starting index. + * If no such bit exists, or if {@code -1} is given as the + * starting index, then {@code -1} is returned. + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the previous clear bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is less + * than {@code -1} + * @since 1.7 + */ + public int previousClearBit(int fromIndex) { + if (fromIndex < 0) { + if (fromIndex == -1) + return -1; + throw new IndexOutOfBoundsException( + "fromIndex < -1: " + fromIndex); + } + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return fromIndex; + + long word = ~words[u] & (WORD_MASK >>> -(fromIndex+1)); + + while (true) { + if (word != 0) + return (u+1) * BITS_PER_WORD -1 - Long.numberOfLeadingZeros(word); + if (u-- == 0) + return -1; + word = ~words[u]; + } + } + + /** + * Returns the "logical size" of this {@code BitSet}: the index of + * the highest set bit in the {@code BitSet} plus one. Returns zero + * if the {@code BitSet} contains no set bits. + * + * @return the logical size of this {@code BitSet} + * @since 1.2 + */ + public int length() { + if (wordsInUse == 0) + return 0; + + return BITS_PER_WORD * (wordsInUse - 1) + + (BITS_PER_WORD - Long.numberOfLeadingZeros(words[wordsInUse - 1])); + } + + /** + * Returns true if this {@code BitSet} contains no bits that are set + * to {@code true}. + * + * @return boolean indicating whether this {@code BitSet} is empty + * @since 1.4 + */ + public boolean isEmpty() { + return wordsInUse == 0; + } + + /** + * Returns true if the specified {@code BitSet} has any bits set to + * {@code true} that are also set to {@code true} in this {@code BitSet}. + * + * @param set {@code BitSet} to intersect with + * @return boolean indicating whether this {@code BitSet} intersects + * the specified {@code BitSet} + * @since 1.4 + */ + public boolean intersects(BitSetRecyclable set) { + for (int i = Math.min(wordsInUse, set.wordsInUse) - 1; i >= 0; i--) + if ((words[i] & set.words[i]) != 0) + return true; + return false; + } + + /** + * Returns the number of bits set to {@code true} in this {@code BitSet}. + * + * @return the number of bits set to {@code true} in this {@code BitSet} + * @since 1.4 + */ + public int cardinality() { + int sum = 0; + for (int i = 0; i < wordsInUse; i++) + sum += Long.bitCount(words[i]); + return sum; + } + + /** + * Performs a logical AND of this target bit set with the + * argument bit set. This bit set is modified so that each bit in it + * has the value {@code true} if and only if it both initially + * had the value {@code true} and the corresponding bit in the + * bit set argument also had the value {@code true}. + * + * @param set a bit set + */ + public void and(BitSetRecyclable set) { + if (this == set) + return; + + while (wordsInUse > set.wordsInUse) + words[--wordsInUse] = 0; + + // Perform logical AND on words in common + for (int i = 0; i < wordsInUse; i++) + words[i] &= set.words[i]; + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Performs a logical OR of this bit set with the bit set + * argument. This bit set is modified so that a bit in it has the + * value {@code true} if and only if it either already had the + * value {@code true} or the corresponding bit in the bit set + * argument has the value {@code true}. + * + * @param set a bit set + */ + public void or(BitSetRecyclable set) { + if (this == set) + return; + + int wordsInCommon = Math.min(wordsInUse, set.wordsInUse); + + if (wordsInUse < set.wordsInUse) { + ensureCapacity(set.wordsInUse); + wordsInUse = set.wordsInUse; + } + + // Perform logical OR on words in common + for (int i = 0; i < wordsInCommon; i++) + words[i] |= set.words[i]; + + // Copy any remaining words + if (wordsInCommon < set.wordsInUse) + System.arraycopy(set.words, wordsInCommon, + words, wordsInCommon, + wordsInUse - wordsInCommon); + + // recalculateWordsInUse() is unnecessary + checkInvariants(); + } + + /** + * Performs a logical XOR of this bit set with the bit set + * argument. This bit set is modified so that a bit in it has the + * value {@code true} if and only if one of the following + * statements holds: + *
    + *
  • The bit initially has the value {@code true}, and the + * corresponding bit in the argument has the value {@code false}. + *
  • The bit initially has the value {@code false}, and the + * corresponding bit in the argument has the value {@code true}. + *
+ * + * @param set a bit set + */ + public void xor(BitSetRecyclable set) { + int wordsInCommon = Math.min(wordsInUse, set.wordsInUse); + + if (wordsInUse < set.wordsInUse) { + ensureCapacity(set.wordsInUse); + wordsInUse = set.wordsInUse; + } + + // Perform logical XOR on words in common + for (int i = 0; i < wordsInCommon; i++) + words[i] ^= set.words[i]; + + // Copy any remaining words + if (wordsInCommon < set.wordsInUse) + System.arraycopy(set.words, wordsInCommon, + words, wordsInCommon, + set.wordsInUse - wordsInCommon); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Clears all of the bits in this {@code BitSet} whose corresponding + * bit is set in the specified {@code BitSet}. + * + * @param set the {@code BitSet} with which to mask this + * {@code BitSet} + * @since 1.2 + */ + public void andNot(BitSetRecyclable set) { + // Perform logical (a & !b) on words in common + for (int i = Math.min(wordsInUse, set.wordsInUse) - 1; i >= 0; i--) + words[i] &= ~set.words[i]; + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Returns the hash code value for this bit set. The hash code depends + * only on which bits are set within this {@code BitSet}. + * + *

The hash code is defined to be the result of the following + * calculation: + *

 {@code
+     * public int hashCode() {
+     *     long h = 1234;
+     *     long[] words = toLongArray();
+     *     for (int i = words.length; --i >= 0; )
+     *         h ^= words[i] * (i + 1);
+     *     return (int)((h >> 32) ^ h);
+     * }}
+ * Note that the hash code changes if the set of bits is altered. + * + * @return the hash code value for this bit set + */ + public int hashCode() { + long h = 1234; + for (int i = wordsInUse; --i >= 0; ) + h ^= words[i] * (i + 1); + + return (int)((h >> 32) ^ h); + } + + /** + * Returns the number of bits of space actually in use by this + * {@code BitSet} to represent bit values. + * The maximum element in the set is the size - 1st element. + * + * @return the number of bits currently in this bit set + */ + public int size() { + return words.length * BITS_PER_WORD; + } + + /** + * Compares this object against the specified object. + * The result is {@code true} if and only if the argument is + * not {@code null} and is a {@code Bitset} object that has + * exactly the same set of bits set to {@code true} as this bit + * set. That is, for every nonnegative {@code int} index {@code k}, + *
((BitSet)obj).get(k) == this.get(k)
+ * must be true. The current sizes of the two bit sets are not compared. + * + * @param obj the object to compare with + * @return {@code true} if the objects are the same; + * {@code false} otherwise + * @see #size() + */ + public boolean equals(Object obj) { + if (!(obj instanceof BitSetRecyclable)) + return false; + if (this == obj) + return true; + + BitSetRecyclable set = (BitSetRecyclable) obj; + + checkInvariants(); + set.checkInvariants(); + + if (wordsInUse != set.wordsInUse) + return false; + + // Check words in use by both BitSets + for (int i = 0; i < wordsInUse; i++) + if (words[i] != set.words[i]) + return false; + + return true; + } + + /** + * Cloning this {@code BitSet} produces a new {@code BitSet} + * that is equal to it. + * The clone of the bit set is another bit set that has exactly the + * same bits set to {@code true} as this bit set. + * + * @return a clone of this bit set + * @see #size() + */ + public Object clone() { + if (! sizeIsSticky) + trimToSize(); + + try { + BitSetRecyclable result = (BitSetRecyclable) super.clone(); + result.words = words.clone(); + result.checkInvariants(); + return result; + } catch (CloneNotSupportedException e) { + throw new InternalError(e); + } + } + + /** + * Attempts to reduce internal storage used for the bits in this bit set. + * Calling this method may, but is not required to, affect the value + * returned by a subsequent call to the {@link #size()} method. + */ + private void trimToSize() { + if (wordsInUse != words.length) { + words = Arrays.copyOf(words, wordsInUse); + checkInvariants(); + } + } + + /** + * Save the state of the {@code BitSet} instance to a stream (i.e., + * serialize it). + */ + private void writeObject(ObjectOutputStream s) + throws IOException { + + checkInvariants(); + + if (! sizeIsSticky) + trimToSize(); + + ObjectOutputStream.PutField fields = s.putFields(); + fields.put("bits", words); + s.writeFields(); + } + + /** + * Reconstitute the {@code BitSet} instance from a stream (i.e., + * deserialize it). + */ + private void readObject(ObjectInputStream s) + throws IOException, ClassNotFoundException { + + ObjectInputStream.GetField fields = s.readFields(); + words = (long[]) fields.get("bits", null); + + // Assume maximum length then find real length + // because recalculateWordsInUse assumes maintenance + // or reduction in logical size + wordsInUse = words.length; + recalculateWordsInUse(); + sizeIsSticky = (words.length > 0 && words[words.length-1] == 0L); // heuristic + checkInvariants(); + } + + /** + * Returns a string representation of this bit set. For every index + * for which this {@code BitSet} contains a bit in the set + * state, the decimal representation of that index is included in + * the result. Such indices are listed in order from lowest to + * highest, separated by ", " (a comma and a space) and + * surrounded by braces, resulting in the usual mathematical + * notation for a set of integers. + * + *

Example: + *

+     * BitSet drPepper = new BitSet();
+ * Now {@code drPepper.toString()} returns "{@code {}}". + *
+     * drPepper.set(2);
+ * Now {@code drPepper.toString()} returns "{@code {2}}". + *
+     * drPepper.set(4);
+     * drPepper.set(10);
+ * Now {@code drPepper.toString()} returns "{@code {2, 4, 10}}". + * + * @return a string representation of this bit set + */ + public String toString() { + checkInvariants(); + + int numBits = (wordsInUse > 128) ? + cardinality() : wordsInUse * BITS_PER_WORD; + StringBuilder b = new StringBuilder(6*numBits + 2); + b.append('{'); + + int i = nextSetBit(0); + if (i != -1) { + b.append(i); + while (true) { + if (++i < 0) break; + if ((i = nextSetBit(i)) < 0) break; + int endOfRun = nextClearBit(i); + do { b.append(", ").append(i); } + while (++i != endOfRun); + } + } + + b.append('}'); + return b.toString(); + } + + public BitSetRecyclable resetWords(long[] words) { + int n; + for (n = words.length; n > 0 && words[n - 1] == 0; n--) + ; + long[] longs = Arrays.copyOf(words, n); + this.words = longs; + this.wordsInUse = longs.length; + checkInvariants(); + return this; + } + + private Handle recyclerHandle = null; private static final Recycler RECYCLER = new Recycler() { protected BitSetRecyclable newObject(Recycler.Handle recyclerHandle) { @@ -37,7 +1192,7 @@ protected BitSetRecyclable newObject(Recycler.Handle recyclerH }; private BitSetRecyclable(Handle recyclerHandle) { - super(); + this(); this.recyclerHandle = recyclerHandle; } @@ -46,7 +1201,9 @@ public static BitSetRecyclable create() { } public void recycle() { - this.clear(); - recyclerHandle.recycle(this); + if (recyclerHandle != null) { + this.clear(); + recyclerHandle.recycle(this); + } } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java similarity index 76% rename from pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java rename to pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java index 07155ad8a5f454..0f42f35608a76a 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java @@ -21,7 +21,7 @@ import org.testng.Assert; import org.testng.annotations.Test; -public class BitSetRecyclableTest { +public class BitSetRecyclableRecyclableTest { @Test public void testRecycle() { @@ -34,4 +34,15 @@ public void testRecycle() { Assert.assertFalse(bitset2.get(3)); Assert.assertNotSame(bitset3, bitset1); } + + @Test + public void testResetWords() { + BitSetRecyclable bitset1 = BitSetRecyclable.create(); + BitSetRecyclable bitset2 = BitSetRecyclable.create(); + bitset1.set(256); + bitset2.set(128); + bitset1.resetWords(bitset2.toLongArray()); + Assert.assertTrue(bitset1.get(128)); + Assert.assertFalse(bitset1.get(256)); + } }