From: <tho...@us...> - 2013-05-03 16:35:52
|
Revision: 7100 http://bigdata.svn.sourceforge.net/bigdata/?rev=7100&view=rev Author: thompsonbry Date: 2013-05-03 16:35:41 +0000 (Fri, 03 May 2013) Log Message: ----------- - RWStore.postHACommit(): Lock ordering problem was causing a deadlock in testABCMultiTransactionFollowerReads(). The allocation lock must be taken before the extension lock. Javadoc update. - RWStore.resetFromRootBlock(). The same lock ordering problem existed here. Javadoc update. - RWStore: Added / updated javadoc for the allocation and extension locks and reviewed all locking patterns in RWStore. - RWStore: getData() requires shared access to the allocators. That access must be MUTEX with allocation mutation in postHACommit(). Modified getData() to use the m_allocationReadLock for shared access (was using the m_extension.readLock() which is to make file IO MUTEX with file extension). Note: The m_allocationReadLock COULD be pushed down into physicalAddress() since that is the enter point for readers to translate. Should it? - RWStore: freeDeferrals(). Modified the code to not release historical commit points if the key range scan on the commit record index would have a toKey LT the fromKey. This was done to support the HA TXS use case. - RWStore.readRootBlock(). Modified to take the m_extensionLock.readLock() to protect against concurrent file extension. - RWStore.readFromLatchedAddress(). Modified to take the allocator read lock since accesses the allocators. Modified to take the m_extensionLock.readLock() to protect against concurrent file extension. This method is used by DumpJournal. DumpJournal can now be invoked from the NSS on a live Journal. - RWStore.writeRaw(). Modified to take the ReadLock of the extensionLock to protect against concurrent file extension. - RWStore.writeRawBuffer(). Removed the code that was taking the allocation lock. It is not required for file IO. - WORMStrategy.writeRawBuffer(). Removed the code that was taking the file extension lock since it is always taken by writeOnChannel. - RWStore.physicalAddress(). Must take the ReadLock of the allocationLock since this is a public method and it reads on the allocators. - RWStore.getFixedAllocatorCount(). Must take allocationReadLock. - RWStore.getAllocatedBlocks(). Must take allocationReadLock. - RWStore.getFileStorage(). Must take allocationReadLock. - RWStore.getAllocatorSlots(). Must take allocationReadLock. - RWStore.computeDigest(). Made it easier to change between the two digest methods (Old and Alt). - TestJournalRW is green. - TestWORMStrategy is green. - TestHA3JournalStrategy.testABCMultiTransactionFollowerReads() is green locally (I have other changes locally pertaining to how and when the releaseTime is updated). - Bug fix to TestHA3SnapshotPolicy. It needed to enabled online disaster recovery. - All HA test are passing. @see https://sourceforge.net/apps/trac/bigdata/ticket/530 (Journal HA) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-05-03 12:11:49 UTC (rev 7099) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-05-03 16:35:41 UTC (rev 7100) @@ -1167,7 +1167,7 @@ } - @Override + @Override public long getBlockSequence() { return lastBlockSequence; @@ -1238,8 +1238,8 @@ writeCacheService.close(); writeCacheService = newWriteCacheService(); } else { - writeCacheService.reset(); - writeCacheService.setExtent(extent); + writeCacheService.reset(); + writeCacheService.setExtent(extent); } } catch (InterruptedException e) { throw new RuntimeException(e); @@ -1545,10 +1545,10 @@ * @return the physical address of the offset provided */ private long offset2PhysicalAddress(final long offset) { - return offset + headerSize; - } + return offset + headerSize; + } - /** + /** * Read on the backing file. {@link ByteBuffer#remaining()} bytes will be * read into the caller's buffer, starting at the specified offset in the * backing file. @@ -1567,7 +1567,7 @@ final Lock readLock = extensionLock.readLock(); readLock.lock(); try { - final int startPos = dst.position(); + final int startPos = dst.position(); try { // the offset into the disk file. @@ -1635,23 +1635,23 @@ */ private FileChannel reopenChannel() throws IOException { - /* - * Note: This is basically a double-checked locking pattern. It is - * used to avoid synchronizing when the backing channel is already - * open. - */ - { - final RandomAccessFile tmp = raf; - if (tmp != null) { - final FileChannel channel = tmp.getChannel(); - if (channel.isOpen()) { - // The channel is still open. - return channel; - } - } - } + /* + * Note: This is basically a double-checked locking pattern. It is + * used to avoid synchronizing when the backing channel is already + * open. + */ + { + final RandomAccessFile tmp = raf; + if (tmp != null) { + final FileChannel channel = tmp.getChannel(); + if (channel.isOpen()) { + // The channel is still open. + return channel; + } + } + } - synchronized (opener) { + synchronized (opener) { assertOpen(); @@ -1862,9 +1862,9 @@ offset = getOffset(addr); - final long paddr = offset2PhysicalAddress(offset); + final long paddr = offset2PhysicalAddress(offset); - boolean wroteOnCache = false; + boolean wroteOnCache = false; if (writeCacheService != null) { if (!writeCacheService.write(paddr, data, chk)) throw new AssertionError(); @@ -1952,9 +1952,9 @@ */ private final ByteBuffer _checkbuf; -// private HARebuildRequest m_rebuildRequest; +// private HARebuildRequest m_rebuildRequest; // -// private int m_rebuildSequence; +// private int m_rebuildSequence; /** * Make sure that the file is large enough to accept a write of @@ -2431,13 +2431,13 @@ super.closeForWrites(); // do not discard the write cache, just reset it to preserve - // read cache + // read cache // releaseWriteCache(); try { - writeCacheService.reset(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + writeCacheService.reset(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } @@ -2466,7 +2466,7 @@ * of this method are ignored. */ @Override - public void delete(final long addr) { + public void delete(final long addr) { if (writeCacheService != null) { @@ -2517,44 +2517,49 @@ final int limit = bb.limit(); bb.position(limit); - // Flush the write in the write cache to the backing store. - final Lock readLock = extensionLock.readLock(); - readLock.lock(); - try { + /* Flush the write in the write cache to the backing store. + * + * Note: writeOnChannel() takes the extensionLock for us. + */ +// final Lock readLock = extensionLock.readLock(); +// readLock.lock(); +// try { + writeCache.flush(false/* force */); - - // install reads into readCache (if any) - writeCacheService.installReads(writeCache); - } finally { - readLock.unlock(); - } +// } finally { +// readLock.unlock(); +// } + + // install reads into readCache (if any) + writeCacheService.installReads(writeCache); + } @Override public Future<Void> sendHALogBuffer(final IHALogRequest req, - final IHAWriteMessage msg, final IBufferAccess b) - throws IOException, InterruptedException { + final IHAWriteMessage msg, final IBufferAccess b) + throws IOException, InterruptedException { - // read direct from store - final ByteBuffer clientBuffer = b.buffer(); - final int nbytes = msg.getSize(); - clientBuffer.position(0); - clientBuffer.limit(nbytes); + // read direct from store + final ByteBuffer clientBuffer = b.buffer(); + final int nbytes = msg.getSize(); + clientBuffer.position(0); + clientBuffer.limit(nbytes); readRaw(/*nbytes, */msg.getFirstOffset(), clientBuffer); - - assert clientBuffer.remaining() > 0 : "Empty buffer: " + clientBuffer; + + assert clientBuffer.remaining() > 0 : "Empty buffer: " + clientBuffer; - @SuppressWarnings("unchecked") - final QuorumPipeline<HAPipelineGlue> quorumMember = (QuorumPipeline<HAPipelineGlue>) quorum - .getMember(); + @SuppressWarnings("unchecked") + final QuorumPipeline<HAPipelineGlue> quorumMember = (QuorumPipeline<HAPipelineGlue>) quorum + .getMember(); - final Future<Void> remoteWriteFuture = quorumMember.replicate(req, msg, - clientBuffer); + final Future<Void> remoteWriteFuture = quorumMember.replicate(req, msg, + clientBuffer); - return remoteWriteFuture; - } + return remoteWriteFuture; + } @Override public Future<Void> sendRawBuffer(final IHARebuildRequest req, @@ -2863,38 +2868,38 @@ } } - @Override - public void writeRawBuffer(HARebuildRequest req, IHAWriteMessage msg, - ByteBuffer transfer) throws IOException { -// if (m_rebuildRequest == null) -// throw new IllegalStateException("Store is not in rebuild state"); -// -// if (m_rebuildSequence != msg.getSequence()) -// throw new IllegalStateException("Invalid sequence number for rebuild, expected: " + m_rebuildSequence + ", actual: " + msg.getSequence()); + @Override + public void writeRawBuffer(HARebuildRequest req, IHAWriteMessage msg, + ByteBuffer transfer) throws IOException { +// if (m_rebuildRequest == null) +// throw new IllegalStateException("Store is not in rebuild state"); +// +// if (m_rebuildSequence != msg.getSequence()) +// throw new IllegalStateException("Invalid sequence number for rebuild, expected: " + m_rebuildSequence + ", actual: " + msg.getSequence()); - FileChannelUtility.writeAll(this.opener, transfer, msg.getFirstOffset()); - -// m_rebuildSequence++; - } + FileChannelUtility.writeAll(this.opener, transfer, msg.getFirstOffset()); + +// m_rebuildSequence++; + } -// @Override -// public void prepareForRebuild(HARebuildRequest req) { -// assert m_rebuildRequest == null; -// -// m_rebuildRequest = req; -// m_rebuildSequence = 0; -// } +// @Override +// public void prepareForRebuild(HARebuildRequest req) { +// assert m_rebuildRequest == null; +// +// m_rebuildRequest = req; +// m_rebuildSequence = 0; +// } // -// @Override -// public void completeRebuild(final HARebuildRequest req, final IRootBlockView rbv) { -// assert m_rebuildRequest != null; -// -// assert m_rebuildRequest.equals(req); -// -// // TODO: reinit from file -// this.resetFromHARootBlock(rbv); -// -// m_rebuildRequest = null; -// } - +// @Override +// public void completeRebuild(final HARebuildRequest req, final IRootBlockView rbv) { +// assert m_rebuildRequest != null; +// +// assert m_rebuildRequest.equals(req); +// +// // TODO: reinit from file +// this.resetFromHARootBlock(rbv); +// +// m_rebuildRequest = null; +// } + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-05-03 12:11:49 UTC (rev 7099) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-05-03 16:35:41 UTC (rev 7100) @@ -53,7 +53,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; @@ -267,27 +266,27 @@ */ public interface Options { - /** - * Option defines the Allocation block sizes for the RWStore. The values - * defined are multiplied by 64 to provide the actual allocations. The - * list of allocations should be ',' delimited and in increasing order. - * This array is written into the store so changing the values does not - * break older stores. For example, - * - * <pre> - * "1,2,4,8,116,32,64" - * </pre> - * - * defines allocations from 64 to 4K in size. It is a good to define - * block sizes on 4K boundaries as soon as possible to optimize IO. This - * is particularly relevant for SSDs. A 1K boundary is expressed as - * <code>16</code> in the allocation sizes, so a 4K boundary is - * expressed as <code>64</code> and an 8k boundary as <code>128</code>. - * <p> - * The default allocations are {@value #DEFAULT_ALLOCATION_SIZES}. - * - * @see #DEFAULT_ALLOCATION_SIZES - */ + /** + * Option defines the Allocation block sizes for the RWStore. The values + * defined are multiplied by 64 to provide the actual allocations. The + * list of allocations should be ',' delimited and in increasing order. + * This array is written into the store so changing the values does not + * break older stores. For example, + * + * <pre> + * "1,2,4,8,116,32,64" + * </pre> + * + * defines allocations from 64 to 4K in size. It is a good to define + * block sizes on 4K boundaries as soon as possible to optimize IO. This + * is particularly relevant for SSDs. A 1K boundary is expressed as + * <code>16</code> in the allocation sizes, so a 4K boundary is + * expressed as <code>64</code> and an 8k boundary as <code>128</code>. + * <p> + * The default allocations are {@value #DEFAULT_ALLOCATION_SIZES}. + * + * @see #DEFAULT_ALLOCATION_SIZES + */ String ALLOCATION_SIZES = RWStore.class.getName() + ".allocationSizes"; /** @@ -335,12 +334,12 @@ String DEFAULT_FREE_BITS_THRESHOLD = "300"; - /** - * When <code>true</code>, scattered writes which are strictly ascending - * will be coalesced within a buffer and written out as a single IO - * (default {@value #DEFAULT_DOUBLE_BUFFER_WRITES}). This improves write - * performance for SATA, SAS, and even SSD. - */ + /** + * When <code>true</code>, scattered writes which are strictly ascending + * will be coalesced within a buffer and written out as a single IO + * (default {@value #DEFAULT_DOUBLE_BUFFER_WRITES}). This improves write + * performance for SATA, SAS, and even SSD. + */ String DOUBLE_BUFFER_WRITES = RWStore.class.getName() + ".doubleBuffer"; String DEFAULT_DOUBLE_BUFFER_WRITES = "true"; @@ -370,19 +369,19 @@ private static final String ERR_WRITE_CACHE_CREATE = "Unable to create write cache service"; - /** - * The fixed size of any allocator on the disk in bytes. The #of allocations - * managed by an allocator is this value times 8 because each slot uses one - * bit in the allocator. When an allocator is allocated, the space on the - * persistent heap is reserved for all slots managed by that allocator. - * However, the {@link FixedAllocator} only incrementally allocates the - * {@link AllocBlock}s. - */ - static private final int ALLOC_BLOCK_SIZE = 1024; - -// // from 32 bits, need 13 to hold max offset of 8 * 1024, leaving 19 for number of blocks: 256K -// static final int BLOCK_INDEX_BITS = 19; /** + * The fixed size of any allocator on the disk in bytes. The #of allocations + * managed by an allocator is this value times 8 because each slot uses one + * bit in the allocator. When an allocator is allocated, the space on the + * persistent heap is reserved for all slots managed by that allocator. + * However, the {@link FixedAllocator} only incrementally allocates the + * {@link AllocBlock}s. + */ + static private final int ALLOC_BLOCK_SIZE = 1024; + +// // from 32 bits, need 13 to hold max offset of 8 * 1024, leaving 19 for number of blocks: 256K +// static final int BLOCK_INDEX_BITS = 19; + /** * The #of low bits in a latched address that encode the offset of the bit * in a {@link FixedAllocator}. The {@link FixedAllocator} will map the bit * onto an allocation slot. @@ -392,107 +391,107 @@ * order in which it was created. This is used to index into * {@link #m_allocs}, which are the {@link FixedAllocator}s. */ - static final int OFFSET_BITS = 13; - static final int OFFSET_BITS_MASK = 0x1FFF; // was 0xFFFF - - static final int ALLOCATION_SCALEUP = 16; // multiplier to convert allocations based on minimum allocation of 32k - static private final int META_ALLOCATION = 8; // 8 * 32K is size of meta Allocation + static final int OFFSET_BITS = 13; + static final int OFFSET_BITS_MASK = 0x1FFF; // was 0xFFFF + + static final int ALLOCATION_SCALEUP = 16; // multiplier to convert allocations based on minimum allocation of 32k + static private final int META_ALLOCATION = 8; // 8 * 32K is size of meta Allocation - // If required, then allocate 1M direct buffers - private static final int cDirectBufferCapacity = 1024 * 1024; + // If required, then allocate 1M direct buffers + private static final int cDirectBufferCapacity = 1024 * 1024; - private int cMaxDirectBuffers = 20; // 20M of direct buffers - static final int cDirectAllocationOffset = 64 * 1024; + private int cMaxDirectBuffers = 20; // 20M of direct buffers + static final int cDirectAllocationOffset = 64 * 1024; - // /////////////////////////////////////////////////////////////////////////////////////// - // RWStore Data - // /////////////////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////////////////// + // RWStore Data + // /////////////////////////////////////////////////////////////////////////////////////// - private final File m_fd; -// private RandomAccessFile m_raf; -// protected FileMetadata m_metadata; -// protected int m_transactionCount; -// private boolean m_committing; + private final File m_fd; +// private RandomAccessFile m_raf; +// protected FileMetadata m_metadata; +// protected int m_transactionCount; +// private boolean m_committing; // /** // * When <code>true</code> the allocations will not actually be recycled // * until after a store restart. When <code>false</code>, the allocations are // * recycled once they satisfy the history retention requirement. // */ -// private boolean m_preserveSession = false; -// private boolean m_readOnly; +// private boolean m_preserveSession = false; +// private boolean m_readOnly; - /** - * The UUID of the backing store. - * - * @see #initfromRootBlock(IRootBlockView) - * @see IRawStore#getUUID() - */ - private UUID m_storeUUID; - - /** - * lists of total alloc blocks. - * - * @todo examine concurrency and lock usage for {@link #m_alloc} and the - * rest of these lists. - */ - private final ArrayList<FixedAllocator> m_allocs; + /** + * The UUID of the backing store. + * + * @see #initfromRootBlock(IRootBlockView) + * @see IRawStore#getUUID() + */ + private UUID m_storeUUID; + + /** + * lists of total alloc blocks. + * + * @todo examine concurrency and lock usage for {@link #m_alloc} and the + * rest of these lists. + */ + private final ArrayList<FixedAllocator> m_allocs; - /** - * A fixed length array of lists of free {@link FixedAllocator}s with one - * entry in the array for each configured allocator size. An allocator is - * put onto this free list when it is initially created. When the store is - * opened, it will be added to this list if {@link Allocator#hasFree()} - * returns true. It will be removed when it has no free space remaining. It - * will be added back to the free list when its free slots exceeds a - * configured threshold. - */ - private ArrayList<FixedAllocator> m_freeFixed[]; - -// /** lists of free blob allocators. */ - // private final ArrayList<BlobAllocator> m_freeBlobs; + /** + * A fixed length array of lists of free {@link FixedAllocator}s with one + * entry in the array for each configured allocator size. An allocator is + * put onto this free list when it is initially created. When the store is + * opened, it will be added to this list if {@link Allocator#hasFree()} + * returns true. It will be removed when it has no free space remaining. It + * will be added back to the free list when its free slots exceeds a + * configured threshold. + */ + private ArrayList<FixedAllocator> m_freeFixed[]; + +// /** lists of free blob allocators. */ + // private final ArrayList<BlobAllocator> m_freeBlobs; - /** lists of blocks requiring commitment. */ - private final ArrayList<FixedAllocator> m_commitList; + /** lists of blocks requiring commitment. */ + private final ArrayList<FixedAllocator> m_commitList; -// private WriteBlock m_writes; - - private final Quorum<?,?> m_quorum; - - /** - * The #of buffers that will be used by the {@link WriteCacheService}. - * - * @see com.bigdata.journal.Options#WRITE_CACHE_BUFFER_COUNT - */ - private final int m_writeCacheBufferCount; +// private WriteBlock m_writes; + + private final Quorum<?,?> m_quorum; + + /** + * The #of buffers that will be used by the {@link WriteCacheService}. + * + * @see com.bigdata.journal.Options#WRITE_CACHE_BUFFER_COUNT + */ + private final int m_writeCacheBufferCount; - /** - * @see com.bigdata.journal.Options#WRITE_CACHE_MIN_CLEAN_LIST_SIZE - */ + /** + * @see com.bigdata.journal.Options#WRITE_CACHE_MIN_CLEAN_LIST_SIZE + */ private final int m_minCleanListSize; - /** - * The #of read buffers that will be used by the {@link WriteCacheService}. - * - * @see com.bigdata.journal.Options#READ_CACHE_BUFFER_COUNT - */ - private final int m_readCacheBufferCount; + /** + * The #of read buffers that will be used by the {@link WriteCacheService}. + * + * @see com.bigdata.journal.Options#READ_CACHE_BUFFER_COUNT + */ + private final int m_readCacheBufferCount; /** * @see com.bigdata.journal.Options#WRITE_CACHE_COMPACTION_THRESHOLD */ - private final int m_compactionThreshold; - + private final int m_compactionThreshold; + /** * @see com.bigdata.journal.Options#HOT_CACHE_THRESHOLD */ - private final int m_hotCacheThreshold; - + private final int m_hotCacheThreshold; + /** * @see com.bigdata.journal.Options#HOT_CACHE_SIZE */ - private final int m_hotCacheSize; - + private final int m_hotCacheSize; + /** * Note: This is not final because we replace the {@link WriteCacheService} * during {@link #reset(long)} in order to propagate the then current quorum @@ -500,79 +499,110 @@ */ RWWriteCacheService m_writeCacheService; - /** - * The actual allocation sizes as read from the store. - * - * @see #DEFAULT_ALLOCATION_SIZES - */ - private int[] m_allocSizes; + /** + * The actual allocation sizes as read from the store. + * + * @see #DEFAULT_ALLOCATION_SIZES + */ + private int[] m_allocSizes; - /** - * The maximum allocation size (bytes). - */ + /** + * The maximum allocation size (bytes). + */ final int m_maxFixedAlloc; /** * The minimum allocation size (bytes). */ final int m_minFixedAlloc; - + /** * We allow blob headers so the maximum blob size is Integer.MAX_VALUE. */ final int m_maxBlobAllocSize = Integer.MAX_VALUE; - + /** - * This lock is used to exclude readers when the extent of the backing file - * is about to be changed. + * This lock is used to exclude readers/writers performing IOs against the + * backing file when the extent of the backing file is about to be changed. + * Readers and writers take the {@link ReadLock}. The {@link WriteLock} is + * taken when the file extent must be changed. This is a workaround for an + * old (an unresolved as of February 2010) Sun bug. * <p> - * At present we use synchronized (this) for alloc/commitChanges and - * getData, since only alloc and commitChanges can cause a file extend, and - * only getData can read. + * Note: Any public method that ONLY takes the extensionLock MUST NOT make + * calls that could take the {@link #m_allocationLock}. This would cause a + * lock ordering problem. If both locks must be taken, then the + * {@link #m_allocationLock} MUST be taken first. + * + * @see http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6371642 + * @see #m_allocationLock + */ + final private ReentrantReadWriteLock m_extensionLock = new ReentrantReadWriteLock(); + + /** + * An explicit allocation lock supports exclusive access for allocator + * mutation and shared access for readers. * <p> - * By using an explicit extensionLock we can unsure that that the taking of - * the lock is directly related to the functionality, plus we can support - * concurrent reads. + * Note: You must hold the {@link #m_allocationReadLock} to read the + * allocators. * <p> - * You MUST hold the {@link #m_allocationLock} before acquiring the - * {@link ReentrantReadWriteLock#writeLock()} of the - * {@link #m_extensionLock}. + * Note: You must hold the {@link #m_allocationWriteLock} while allocating + * or clearing allocations. + * <p> + * Note: It is only when an allocation triggers a file extension that the + * {@link WriteLock} of the {@link #m_extensionLock} needs to be taken. + * + * TODO: There is scope to take advantage of the different allocator sizes + * and provide allocation locks on the fixed allocators. We will still need + * a store-wide allocation lock when creating new allocation areas, but + * significant contention may be avoided. */ - final private ReentrantReadWriteLock m_extensionLock = new ReentrantReadWriteLock(); - - /** - * An explicit allocation lock allows for reads concurrent with allocation - * requests. You must hold the allocation lock while allocating or clearing - * allocations. It is only when an allocation triggers a file extension that - * the write extensionLock needs to be taken. - * - * TODO: There is scope to take advantage of the different allocator sizes - * and provide allocation locks on the fixed allocators. We will still need - * a store-wide allocation lock when creating new allocation areas, but - * significant contention may be avoided. - */ final private ReentrantReadWriteLock m_allocationLock = new ReentrantReadWriteLock(); + /** + * Lock used for exclusive access to the allocators. + * <p> + * Note: Historically, this lock was only required for mutation and readers + * did not content for a lock. + */ final private WriteLock m_allocationWriteLock = m_allocationLock.writeLock(); + /** + * Lock used for shared access to allocators. + * <p> + * Note: Historically the allocators were unprotected for shared acccess + * (readers) and protected by a single lock for mutation (writes). Shared + * access by readers was safe since (a) old allocators were never replaced; + * and (b) readers had access only to committed data. + * <p> + * This situation was changed when the {@link #postHACommit(IRootBlockView)} + * method was introduced since it could replace allocators in a manner that + * was not safe for shared access by readers. Methods that were historically + * using unprotected shared access now require protected shared access using + * this lock. + * + * @see #postHACommit(IRootBlockView) + * @see #getData(long, int) + * @see #getData(long, byte[]) + * @see #getData(long, byte[], int, int) + */ final private ReadLock m_allocationReadLock = m_allocationLock.readLock(); - /** - * The deferredFreeList is simply an array of releaseTime,freeListAddrs - * stored at commit. - * <p> - * Note that when the deferredFreeList is saved, ONLY thefreeListAddrs - * are stored, NOT the releaseTime. This is because on any open of - * the store, all deferredFrees can be released immediately. This - * mechanism may be changed in the future to enable explicit history - * retention, but if so a different header structure would be used since - * it would not be appropriate to retain a simple header linked to - * thousands if not millions of commit points. - */ + /** + * The deferredFreeList is simply an array of releaseTime,freeListAddrs + * stored at commit. + * <p> + * Note that when the deferredFreeList is saved, ONLY thefreeListAddrs + * are stored, NOT the releaseTime. This is because on any open of + * the store, all deferredFrees can be released immediately. This + * mechanism may be changed in the future to enable explicit history + * retention, but if so a different header structure would be used since + * it would not be appropriate to retain a simple header linked to + * thousands if not millions of commit points. + */ // * // * If the current txn list exceeds the MAX_DEFERRED_FREE then it is // * incrementally saved and a new list begun. The master list itself // * serves as a BLOB header when there is more than a single entry with // * the same txReleaseTime. -// private static final int MAX_DEFERRED_FREE = 4094; // fits in 16k block +// private static final int MAX_DEFERRED_FREE = 4094; // fits in 16k block private final long m_minReleaseAge; /** @@ -582,25 +612,25 @@ */ private int m_activeTxCount = 0; - private volatile long m_lastDeferredReleaseTime = 0L; -// private final ArrayList<Integer> m_currentTxnFreeList = new ArrayList<Integer>(); - private final PSOutputStream m_deferredFreeOut; + private volatile long m_lastDeferredReleaseTime = 0L; +// private final ArrayList<Integer> m_currentTxnFreeList = new ArrayList<Integer>(); + private final PSOutputStream m_deferredFreeOut; /** * Used to transparently re-open the backing channel if it has been closed * by an interrupt during an IO. */ - private final ReopenFileChannel m_reopener; + private final ReopenFileChannel m_reopener; - private volatile BufferedWrite m_bufferedWrite; - - /** - * Our StoreageStats objects - */ - private StorageStats m_storageStats; - private long m_storageStatsAddr = 0; - + private volatile BufferedWrite m_bufferedWrite; + /** + * Our StoreageStats objects + */ + private StorageStats m_storageStats; + private long m_storageStatsAddr = 0; + + /** * <code>true</code> iff the backing store is open. */ private volatile boolean m_open = true; @@ -619,7 +649,7 @@ // private ConcurrentHashMap<Integer, String> m_blacklist = null; private ConcurrentHashMap<Integer, Long> m_lockAddresses = null; - class WriteCacheImpl extends WriteCache.FileChannelScatteredWriteCache { + class WriteCacheImpl extends WriteCache.FileChannelScatteredWriteCache { public WriteCacheImpl(final IBufferAccess buf, final boolean useChecksum, final boolean bufferHasData, @@ -661,23 +691,23 @@ // Added to enable debug of rare problem // FIXME: disable by removal once solved protected void registerWriteStatus(long offset, int length, char action) { - m_writeCacheService.debugAddrs(offset, length, action); + m_writeCacheService.debugAddrs(offset, length, action); } - + @Override protected void addAddress(int latchedAddr, int size) { - // No longer valid - // RWStore.this.addAddress(latchedAddr, size); + // No longer valid + // RWStore.this.addAddress(latchedAddr, size); } @Override protected void removeAddress(int latchedAddr) { - // No longer valid - // RWStore.this.removeAddress(latchedAddr); + // No longer valid + // RWStore.this.removeAddress(latchedAddr); } - }; - + }; + /** * The ALLOC_SIZES must be initialized from either the file or the * properties associated with the fileMetadataView @@ -722,50 +752,50 @@ } m_metaBits = new int[m_metaBitsSize]; - - m_metaTransientBits = new int[m_metaBitsSize]; - - + + m_metaTransientBits = new int[m_metaBitsSize]; + + m_quorum = quorum; - - m_fd = fileMetadata.file; - - // initialize striped performance counters for this store. + + m_fd = fileMetadata.file; + + // initialize striped performance counters for this store. this.storeCounters.set(new StoreCounters(10/* batchSize */)); - final IRootBlockView m_rb = fileMetadata.rootBlock; + final IRootBlockView m_rb = fileMetadata.rootBlock; - m_commitList = new ArrayList<FixedAllocator>(); + m_commitList = new ArrayList<FixedAllocator>(); - m_allocs = new ArrayList<FixedAllocator>(); - - // m_freeBlobs = new ArrayList<BlobAllocator>(); + m_allocs = new ArrayList<FixedAllocator>(); + + // m_freeBlobs = new ArrayList<BlobAllocator>(); - try { - final RandomAccessFile m_raf = fileMetadata.getRandomAccessFile(); - m_reopener = new ReopenFileChannel(m_fd, m_raf, "rw"); - } catch (IOException e1) { - throw new RuntimeException(e1); - } + try { + final RandomAccessFile m_raf = fileMetadata.getRandomAccessFile(); + m_reopener = new ReopenFileChannel(m_fd, m_raf, "rw"); + } catch (IOException e1) { + throw new RuntimeException(e1); + } - if (Boolean.valueOf(fileMetadata.getProperty( - Options.DOUBLE_BUFFER_WRITES, - Options.DEFAULT_DOUBLE_BUFFER_WRITES))) { - try { - m_bufferedWrite = new BufferedWrite(this); - } catch (InterruptedException e1) { - m_bufferedWrite = null; - } - } else { - m_bufferedWrite = null; - } + if (Boolean.valueOf(fileMetadata.getProperty( + Options.DOUBLE_BUFFER_WRITES, + Options.DEFAULT_DOUBLE_BUFFER_WRITES))) { + try { + m_bufferedWrite = new BufferedWrite(this); + } catch (InterruptedException e1) { + m_bufferedWrite = null; + } + } else { + m_bufferedWrite = null; + } - m_writeCacheBufferCount = fileMetadata.writeCacheBufferCount; - - m_readCacheBufferCount = Integer.valueOf(fileMetadata.getProperty( + m_writeCacheBufferCount = fileMetadata.writeCacheBufferCount; + + m_readCacheBufferCount = Integer.valueOf(fileMetadata.getProperty( com.bigdata.journal.Options.READ_CACHE_BUFFER_COUNT, com.bigdata.journal.Options.DEFAULT_READ_CACHE_BUFFER_COUNT)); - + if (log.isInfoEnabled()) log.info(com.bigdata.journal.Options.WRITE_CACHE_BUFFER_COUNT + "=" + m_writeCacheBufferCount); @@ -804,87 +834,87 @@ // m_writeCache = newWriteCache(); - try { + try { if (m_rb.getNextOffset() == 0) { // if zero then new file - setAllocations(fileMetadata); + setAllocations(fileMetadata); /* * FIXME Martyn, the code paths here are crazy complicated. * defaultInit() is also invoked from initFromRootBlock(). * Simplify this. BBT */ - m_storeUUID = m_rb.getUUID(); + m_storeUUID = m_rb.getUUID(); - defaultInit(); - - m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; - m_minFixedAlloc = m_allocSizes[0]*64; - - m_storageStats = new StorageStats(m_allocSizes); + defaultInit(); + + m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; + m_minFixedAlloc = m_allocSizes[0]*64; + + m_storageStats = new StorageStats(m_allocSizes); -// // Check for overwrite option and set overwrite buffer if -// // required -// if (Boolean.valueOf(fileMetadata.getProperty( -// Options.OVERWRITE_DELETE, -// Options.DEFAULT_OVERWRITE_DELETE))) { -// m_writeCache.setOverwriteBuffer(m_maxFixedAlloc); -// } - } else { - - initfromRootBlock(m_rb); - - m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; - m_minFixedAlloc = m_allocSizes[0]*64; +// // Check for overwrite option and set overwrite buffer if +// // required +// if (Boolean.valueOf(fileMetadata.getProperty( +// Options.OVERWRITE_DELETE, +// Options.DEFAULT_OVERWRITE_DELETE))) { +// m_writeCache.setOverwriteBuffer(m_maxFixedAlloc); +// } + } else { + + initfromRootBlock(m_rb); + + m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; + m_minFixedAlloc = m_allocSizes[0]*64; - if (m_storageStatsAddr != 0) { - final long statsAddr = m_storageStatsAddr >> 16; - final int statsLen = ((int) m_storageStatsAddr) & 0xFFFF; - final byte[] stats = new byte[statsLen + 4]; // allow for checksum - getData(statsAddr, stats); - final DataInputStream instr = new DataInputStream(new ByteArrayInputStream(stats)); - m_storageStats = new StorageStats(instr); - - for (FixedAllocator fa: m_allocs) { - m_storageStats.register(fa); - } - } else { - m_storageStats = new StorageStats(m_allocSizes); - } - + if (m_storageStatsAddr != 0) { + final long statsAddr = m_storageStatsAddr >> 16; + final int statsLen = ((int) m_storageStatsAddr) & 0xFFFF; + final byte[] stats = new byte[statsLen + 4]; // allow for checksum + getData(statsAddr, stats); + final DataInputStream instr = new DataInputStream(new ByteArrayInputStream(stats)); + m_storageStats = new StorageStats(instr); + + for (FixedAllocator fa: m_allocs) { + m_storageStats.register(fa); + } + } else { + m_storageStats = new StorageStats(m_allocSizes); + } + if (log.isTraceEnabled()) { final StringBuilder str = new StringBuilder(); this.showAllocators(str); log.trace(str); } - } - - // Maximum theoretically addressable file size is determined by the - // maximum allocator slot size multiplied by Integer.MAX_VALUE + } + + // Maximum theoretically addressable file size is determined by the + // maximum allocator slot size multiplied by Integer.MAX_VALUE // FIXME: do we want to constrain this as a system property? - m_maxFileSize = ((long) Integer.MAX_VALUE) * m_maxFixedAlloc; + m_maxFileSize = ((long) Integer.MAX_VALUE) * m_maxFixedAlloc; - // setup write cache AFTER init to ensure filesize is correct! + // setup write cache AFTER init to ensure filesize is correct! - m_writeCacheService = newWriteCache(); + m_writeCacheService = newWriteCache(); - final int maxBlockLessChk = m_maxFixedAlloc-4; + final int maxBlockLessChk = m_maxFixedAlloc-4; - assert m_maxFixedAlloc > 0; - - m_deferredFreeOut = PSOutputStream.getNew(this, m_maxFixedAlloc, null); + assert m_maxFixedAlloc > 0; + + m_deferredFreeOut = PSOutputStream.getNew(this, m_maxFixedAlloc, null); -// if (Boolean.valueOf(fileMetadata.getProperty( -// Options.MAINTAIN_BLACKLIST, -// Options.DEFAULT_MAINTAIN_BLACKLIST))) { -// m_blacklist = new ConcurrentHashMap<Integer, String>(); -// m_lockAddresses = new ConcurrentHashMap<Integer, Long>(); -// } +// if (Boolean.valueOf(fileMetadata.getProperty( +// Options.MAINTAIN_BLACKLIST, +// Options.DEFAULT_MAINTAIN_BLACKLIST))) { +// m_blacklist = new ConcurrentHashMap<Integer, String>(); +// m_lockAddresses = new ConcurrentHashMap<Integer, Long>(); +// } - } catch (IOException e) { - throw new StorageTerminalError("Unable to initialize store", e); - } - } + } catch (IOException e) { + throw new StorageTerminalError("Unable to initialize store", e); + } + } /** * Called from WriteCache.resetRecordMapFromBuffer @@ -901,59 +931,59 @@ * <i>latchedAddr</i> but the address itself should not yet be * allocated. */ - void addAddress(final int latchedAddr, final int size) { - // ignore zero address - if (latchedAddr == 0) - return; + void addAddress(final int latchedAddr, final int size) { + // ignore zero address + if (latchedAddr == 0) + return; - m_allocationWriteLock.lock(); - try { - FixedAllocator alloc = null; - try { - alloc = getBlock(latchedAddr); - } catch (final PhysicalAddressResolutionException par) { - // Must create new allocator - } - final int size2 = size < 0 ? -size : size; - if (alloc == null) { - final int i = fixedAllocatorIndex(size2); - final int block = 64 * m_allocSizes[i]; - final ArrayList<FixedAllocator> list = m_freeFixed[i]; - if (log.isTraceEnabled()) - log.trace("Creating new Allocator for address: " - + latchedAddr); + m_allocationWriteLock.lock(); + try { + FixedAllocator alloc = null; + try { + alloc = getBlock(latchedAddr); + } catch (final PhysicalAddressResolutionException par) { + // Must create new allocator + } + final int size2 = size < 0 ? -size : size; + if (alloc == null) { + final int i = fixedAllocatorIndex(size2); + final int block = 64 * m_allocSizes[i]; + final ArrayList<FixedAllocator> list = m_freeFixed[i]; + if (log.isTraceEnabled()) + log.trace("Creating new Allocator for address: " + + latchedAddr); - final FixedAllocator allocator = new FixedAllocator(this, block); + final FixedAllocator allocator = new FixedAllocator(this, block); - allocator.setFreeList(list); - allocator.setIndex(m_allocs.size()); + allocator.setFreeList(list); + allocator.setIndex(m_allocs.size()); - m_allocs.add(allocator); + m_allocs.add(allocator); - // Check correctly synchronized creation - assert allocator == getBlock(latchedAddr); + // Check correctly synchronized creation + assert allocator == getBlock(latchedAddr); - alloc = allocator; - } + alloc = allocator; + } - assert size2 <= alloc.getSlotSize(); + assert size2 <= alloc.getSlotSize(); - if (size > 0) { + if (size > 0) { - /* - * This is a real allocation. - */ + /* + * This is a real allocation. + */ - alloc.setAddressExternal(latchedAddr); + alloc.setAddressExternal(latchedAddr); - } + } - } finally { + } finally { - m_allocationWriteLock.unlock(); + m_allocationWriteLock.unlock(); - } - } + } + } /** * Called from WriteCache.resetRecordMapFromBuffer @@ -962,40 +992,40 @@ * * @param latchedAddr */ - void removeAddress(final int latchedAddr) { - // ignore zero address - if (latchedAddr == 0) - return; + void removeAddress(final int latchedAddr) { + // ignore zero address + if (latchedAddr == 0) + return; - m_allocationWriteLock.lock(); - try { - // assert m_commitList.size() == 0; + m_allocationWriteLock.lock(); + try { + // assert m_commitList.size() == 0; - final FixedAllocator alloc = getBlockByAddress(latchedAddr); + final FixedAllocator alloc = getBlockByAddress(latchedAddr); - assert alloc != null; + assert alloc != null; - final int addrOffset = getOffset(latchedAddr); - if (alloc == null) { - throw new IllegalArgumentException( - "Invalid address provided to immediateFree: " - + latchedAddr); - } - final long pa = alloc.getPhysicalAddress(addrOffset); + final int addrOffset = getOffset(latchedAddr); + if (alloc == null) { + throw new IllegalArgumentException( + "Invalid address provided to immediateFree: " + + latchedAddr); + } + final long pa = alloc.getPhysicalAddress(addrOffset); - if (log.isTraceEnabled()) - log.trace("Freeing allocation at " + latchedAddr - + ", physical address: " + pa); + if (log.isTraceEnabled()) + log.trace("Freeing allocation at " + latchedAddr + + ", physical address: " + pa); - alloc.free(latchedAddr, 0, false); + alloc.free(latchedAddr, 0, false); - // assert m_commitList.size() == 0; - } finally { - m_allocationWriteLock.unlock(); - } - } - - /** + // assert m_commitList.size() == 0; + } finally { + m_allocationWriteLock.unlock(); + } + } + + /** * Create and return a new {@link RWWriteCacheService} instance. The caller * is responsible for closing out the old one and must be holding the * appropriate locks when it switches in the new instance. @@ -1033,7 +1063,7 @@ } } - private void setAllocations(final FileMetadata fileMetadata) + private void setAllocations(final FileMetadata fileMetadata) throws IOException { final String buckets = fileMetadata.getProperty( @@ -1052,30 +1082,30 @@ } private void defaultInit() throws IOException { - final int numFixed = m_allocSizes.length; + final int numFixed = m_allocSizes.length; - m_freeFixed = new ArrayList[numFixed]; + m_freeFixed = new ArrayList[numFixed]; - for (int i = 0; i < numFixed; i++) { - m_freeFixed[i] = new ArrayList<FixedAllocator>(); - } + for (int i = 0; i < numFixed; i++) { + m_freeFixed[i] = new ArrayList<FixedAllocator>(); + } - m_fileSize = convertFromAddr(m_fd.length()); - - // make space for meta-allocators - m_metaBits[0] = -1; - m_metaTransientBits[0] = -1; - m_nextAllocation = -(1 + META_ALLOCATION); // keep on a minimum 8K boundary - m_committedNextAllocation = m_nextAllocation; - - if (m_fileSize > m_nextAllocation) { - m_fileSize = m_nextAllocation; - } - - if (log.isInfoEnabled()) - log.info("Set default file extent " + convertAddr(m_fileSize)); - - m_reopener.raf.setLength(convertAddr(m_fileSize)); + m_fileSize = convertFromAddr(m_fd.length()); + + // make space for meta-allocators + m_metaBits[0] = -1; + m_metaTransientBits[0] = -1; + m_nextAllocation = -(1 + META_ALLOCATION); // keep on a minimum 8K boundary + m_committedNextAllocation = m_nextAllocation; + + if (m_fileSize > m_nextAllocation) { + m_fileSize = m_nextAllocation; + } + + if (log.isInfoEnabled()) + log.info("Set default file extent " + convertAddr(m_fileSize)); + + m_reopener.raf.setLength(convertAddr(m_fileSize)); } @@ -1104,71 +1134,71 @@ } } - /** - * Basic check on key root block validity - * - * @param rbv - */ - private void checkRootBlock(final IRootBlockView rbv) { - final long nxtOffset = rbv.getNextOffset(); - final int nxtalloc = -(int) (nxtOffset >> 32); + /** + * Basic check on key root block validity + * + * @param rbv + */ + private void checkRootBlock(final IRootBlockView rbv) { + final long nxtOffset = rbv.getNextOffset(); + final int nxtalloc = -(int) (nxtOffset >> 32); - final int metaBitsAddr = -(int) nxtOffset; + final int metaBitsAddr = -(int) nxtOffset; - final long metaAddr = rbv.getMetaStartAddr(); - final long rawMetaBitsAddr = rbv.getMetaBitsAddr(); - if (metaAddr == 0 || rawMetaBitsAddr == 0) { + final long metaAddr = rbv.getMetaStartAddr(); + final long rawMetaBitsAddr = rbv.getMetaBitsAddr(); + if (metaAddr == 0 || rawMetaBitsAddr == 0) { /* * possible when rolling back to empty file. */ - log.warn("No meta allocation data included in root block for RWStore"); - } - - if (log.isTraceEnabled()) { + log.warn("No meta allocation data included in root block for RWStore"); + } + + if (log.isTraceEnabled()) { final int commitRecordAddr = (int) (rbv.getCommitRecordAddr() >> 32); log.trace("CommitRecord " + rbv.getCommitRecordAddr() + " at physical address: " + physicalAddress(commitRecordAddr)); } - - final long commitCounter = rbv.getCommitCounter(); + + final long commitCounter = rbv.getCommitCounter(); -// final int metaStartAddr = (int) -(metaAddr >> 32); // void -// final int fileSize = (int) -(metaAddr & 0xFFFFFFFF); +// final int metaStartAddr = (int) -(metaAddr >> 32); // void +// final int fileSize = (int) -(metaAddr & 0xFFFFFFFF); - if (log.isTraceEnabled()) - log.trace("m_allocation: " + nxtalloc + ", m_metaBitsAddr: " - + metaBitsAddr + ", m_commitCounter: " + commitCounter); - - } - - /* - * Utility to encapsulate RootBlock interpreation - */ - static class RootBlockInfo { - - static int nextAllocation(final IRootBlockView rb) { - final long nxtOffset = rb.getNextOffset(); + if (log.isTraceEnabled()) + log.trace("m_allocation: " + nxtalloc + ", m_metaBitsAddr: " + + metaBitsAddr + ", m_commitCounter: " + commitCounter); + + } + + /* + * Utility to encapsulate RootBlock interpreation + */ + static class RootBlockInfo { + + static int nextAllocation(final IRootBlockView rb) { + final long nxtOffset = rb.getNextOffset(); - // next allocation to be made (in -32K units). - final int ret = -(int) (nxtOffset >> 32); - + // next allocation to be made (in -32K units). + final int ret = -(int) (nxtOffset >> 32); + /* * Skip the first 32K in the file. The root blocks live here but * nothing else. */ - return ret == 0 ? -(1 + META_ALLOCATION) : ret; - } - - /* - * Meta-Allocations stored as {int address; int[8] bits}, so each block - * holds 8*32=256 allocation slots of 1K totaling 256K. - * - * The returned int array is a flattened list of these int[9] blocks - */ - static int[] metabits(final IRootBlockView rb, final ReopenFileChannel reopener) throws IOException { - final long rawmbaddr = rb.getMetaBitsAddr(); - + return ret == 0 ? -(1 + META_ALLOCATION) : ret; + } + + /* + * Meta-Allocations stored as {int address; int[8] bits}, so each block + * holds 8*32=256 allocation slots of 1K totaling 256K. + * + * The returned int array is a flattened list of these int[9] blocks + */ + static int[] metabits(final IRootBlockView rb, final ReopenFileChannel reopener) throws IOException { + final long rawmbaddr = rb.getMetaBitsAddr(); + /* * The #of int32 values in the metabits region. * @@ -1176,86 +1206,86 @@ * gives the #of int32 values in the metabits regions (up to 64k * int32 values). */ - final int metaBitsStore = (int) (rawmbaddr & 0xFFFF); - - - // The byte offset of the metabits region in the file. - final long pmaddr = rawmbaddr >> 16; - + final int metaBitsStore = (int) (rawmbaddr & 0xFFFF); + + + // The byte offset of the metabits region in the file. + final long pmaddr = rawmbaddr >> 16; + /* * Read the metabits block, including a header and the int32[] * that encodes both startAddrs and bit vectors. */ final byte[] buf = new byte[metaBitsStore * 4]; - FileChannelUtility.readAll(reopener, ByteBuffer.wrap(buf), pmaddr); - - final DataInputStream strBuf = new DataInputStream(new ByteArrayInputStream(buf)); - - // Can handle minor store version incompatibility - strBuf.readInt(); // STORE VERSION - strBuf.readLong(); // Last Deferred Release Time - strBuf.readInt(); // cDefaultMetaBitsSize - - final int allocBlocks = strBuf.readInt(); - strBuf.readLong(); // m_storageStatsAddr + FileChannelUtility.readAll(reopener, ByteBuffer.wrap(buf), pmaddr); + + final DataInputStream strBuf = new DataInputStream(new ByteArrayInputStream(buf)); + + // Can handle minor store version incompatibility + strBuf.readInt(); // STORE VERSION + strBuf.readLong(); // Last Deferred Release Time + strBuf.readInt(); // cDefaultMetaBitsSize + + final int allocBlocks = strBuf.readInt(); + strBuf.readLong(); // m_storageStatsAddr - // step over those reserved ints + // step over those reserved ints for (int i = 0; i < cReservedMetaBits; i++) { strBuf.readInt(); } // step over the allocSizes - for (int i = 0; i < allocBlocks; i++) { - strBuf.readInt(); - } - final int metaBitsSize = metaBitsStore - allocBlocks - cMetaHdrFields; // allow for header fields - - // Must be multiple of 9 - assert metaBitsSize % 9 == 0; - - int[] ret = new int[metaBitsSize]; - for (int i = 0; i < metaBitsSize; i++) { - ret[i] = strBuf.readInt(); - } + for (int i = 0; i < allocBlocks; i++) { + strBuf.readInt(); + } + final int metaBitsSize = metaBitsStore - allocBlocks - cMetaHdrFields; // allow for header fields + + // Must be multiple of 9 + assert metaBitsSize % 9 == 0; + + int[] ret = new int[metaBitsSize]; + for (int i = 0; i < metaBitsSize; i++) { + ret[i] = strBuf.readInt(); + } - /* - * Meta-Allocations stored as {int address; int[8] bits}, so each block - * holds 8*32=256 allocation slots of 1K totaling 256K. - */ - return ret; - } - } - - /** - * Should be called where previously initFileSpec was used. - * - * Rather than reading from file, instead reads from the current root block. - * - * We use the rootBlock fields, nextOffset, metaStartAddr, metaBitsAddr. - * - * metaBitsAddr indicates where the meta allocation bits are. - * - * metaStartAddr is the offset in the file where the allocation blocks are - * allocated the long value also indicates the size of the allocation, such - * that the address plus the size is the "filesize". - * - * Note that metaBitsAddr must be an absolute address, with the low order 16 - * bits used to indicate the size. - * - * @throws IOException - */ - private void initfromRootBlock(final IRootBlockView rb) throws IOException { - // m_rb = m_fmv.getRootBlock(); - assert(rb != null); + /* + * Meta-Allocations stored as {int address; int[8] bits}, so each block + * holds 8*32=256 allocation slots of 1K totaling 256K. + */ + return ret; + } + } + + /** + * Should be called where previously initFileSpec was used. + * + * Rather than reading from file, instead reads from the current root block. + * + * We use the rootBlock fields, nextOffset, metaStartAddr, metaBitsAddr. + * + * metaBitsAddr indicates where the meta allocation bits are. + * + * metaStartAddr is the offset in the file where the allocation blocks are + * allocated the long value also indicates the size of the allocation, such + * that the address plus the size is the "filesize". + * + * Note that metaBitsAddr must be an absolute address, with the low order 16 + * bits used to indicate the size. + * + * @throws IOException + */ + private void initfromRootBlock(final IRootBlockView rb) throws IOException { + // m_rb = m_fmv.getRootBlock(); + assert(rb != null); - m_storeUUID = rb.getUUID(); - - if (rb.getNextOffset() == 0) { + m_storeUUID = rb.getUUID(); + + if (rb.getNextOffset() == 0) { - defaultInit(); - - } else { + defaultInit(); + + } else { /* * The RWStore stores in IRootBlock.getNextOffset() two distinct @@ -1271,52 +1301,52 @@ * FixedAllocators in order to turn it into a byte offset on the * file. */ - final long nxtOffset = rb.getNextOffset(); + final long nxtOffset = rb.getNextOffset(); - // next allocation to be made (in -32K units). - m_nextAllocation = -(int) (nxtOffset >> 32); - - if (m_nextAllocation == 0) { + // next allocation to be made (in -32K units). + m_nextAllocation = -(int) (nxtOffset >> 32); + + if (m_nextAllocation == 0) { /* * Skip the first 32K in the file. The root blocks live here but * nothing else. */ - - m_nextAllocation = -(1 + META_ALLOCATION); - - } - - m_committedNextAllocation = m_nextAllocation; - - // latched offset of the metabits region. - m_metaBitsAddr = -(int) nxtOffset; - - if (log.isInfoEnabled()) { - log.info("MetaBitsAddr: " + m_metaBitsAddr); - } + + m_nextAllocation = -(1 + META_ALLOCATION); + + } + + ... [truncated message content] |