From: <tho...@us...> - 2010-11-03 18:53:56
|
Revision: 3884 http://bigdata.svn.sourceforge.net/bigdata/?rev=3884&view=rev Author: thompsonbry Date: 2010-11-03 18:53:49 +0000 (Wed, 03 Nov 2010) Log Message: ----------- Handoff to Martyn. There are some issues with abort(), but everything else is looking quite good. Modified Paths: -------------- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/FileMetadata.java branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/IBufferStrategy.java branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RWStrategy.java branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/rwstore/RWStore.java branches/JOURNAL_HA_BRANCH/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java Added Paths: ----------- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RootBlockUtility.java Modified: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/FileMetadata.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/FileMetadata.java 2010-11-03 18:00:50 UTC (rev 3883) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/FileMetadata.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -974,51 +974,56 @@ * Check root blocks (magic, timestamps), choose root block, read * constants (slotSize, segmentId). */ - { - - final ChecksumUtility checker = validateChecksum ? ChecksumUtility.threadChk - .get() - : null; + final RootBlockUtility tmp = new RootBlockUtility(opener, file, + validateChecksum, alternateRootBlock); + this.rootBlock0 = tmp.rootBlock0; + this.rootBlock1 = tmp.rootBlock1; + this.rootBlock = tmp.rootBlock; +// { +// +// final ChecksumUtility checker = validateChecksum ? ChecksumUtility.threadChk +// .get() +// : null; +// +// // final FileChannel channel = raf.getChannel(); +// final ByteBuffer tmp0 = ByteBuffer.allocate(RootBlockView.SIZEOF_ROOT_BLOCK); +// final ByteBuffer tmp1 = ByteBuffer.allocate(RootBlockView.SIZEOF_ROOT_BLOCK); +// FileChannelUtility.readAll(opener, tmp0, OFFSET_ROOT_BLOCK0); +// FileChannelUtility.readAll(opener, tmp1, OFFSET_ROOT_BLOCK1); +// tmp0.position(0); // resets the position. +// tmp1.position(0); +// try { +// rootBlock0 = new RootBlockView(true, tmp0, checker); +// } catch (RootBlockException ex) { +// log.warn("Bad root block zero: " + ex); +// } +// try { +// rootBlock1 = new RootBlockView(false, tmp1, checker); +// } catch (RootBlockException ex) { +// log.warn("Bad root block one: " + ex); +// } +// if (rootBlock0 == null && rootBlock1 == null) { +// throw new RuntimeException( +// "Both root blocks are bad - journal is not usable: " +// + file); +// } +// if (alternateRootBlock) +// log.warn("Using alternate root block"); +// /* +// * Choose the root block based on the commit counter. +// * +// * Note: The commit counters MAY be equal. This will happen if +// * we rollback the journal and override the current root block +// * with the alternate root block. +// */ +// final long cc0 = rootBlock0==null?-1L:rootBlock0.getCommitCounter(); +// final long cc1 = rootBlock1==null?-1L:rootBlock1.getCommitCounter(); +// this.rootBlock = (cc0 > cc1 ? (alternateRootBlock ? rootBlock1 +// : rootBlock0) : (alternateRootBlock ? rootBlock0 +// : rootBlock1)); +// +// } - // final FileChannel channel = raf.getChannel(); - final ByteBuffer tmp0 = ByteBuffer.allocate(RootBlockView.SIZEOF_ROOT_BLOCK); - final ByteBuffer tmp1 = ByteBuffer.allocate(RootBlockView.SIZEOF_ROOT_BLOCK); - FileChannelUtility.readAll(opener, tmp0, OFFSET_ROOT_BLOCK0); - FileChannelUtility.readAll(opener, tmp1, OFFSET_ROOT_BLOCK1); - tmp0.position(0); // resets the position. - tmp1.position(0); - try { - rootBlock0 = new RootBlockView(true, tmp0, checker); - } catch (RootBlockException ex) { - log.warn("Bad root block zero: " + ex); - } - try { - rootBlock1 = new RootBlockView(false, tmp1, checker); - } catch (RootBlockException ex) { - log.warn("Bad root block one: " + ex); - } - if (rootBlock0 == null && rootBlock1 == null) { - throw new RuntimeException( - "Both root blocks are bad - journal is not usable: " - + file); - } - if (alternateRootBlock) - log.warn("Using alternate root block"); - /* - * Choose the root block based on the commit counter. - * - * Note: The commit counters MAY be equal. This will happen if - * we rollback the journal and override the current root block - * with the alternate root block. - */ - final long cc0 = rootBlock0.getCommitCounter(); - final long cc1 = rootBlock1.getCommitCounter(); - this.rootBlock = (cc0 > cc1 ? (alternateRootBlock ? rootBlock1 - : rootBlock0) : (alternateRootBlock ? rootBlock0 - : rootBlock1)); - - } - this.bufferMode = BufferMode.getDefaultBufferMode(rootBlock.getStoreType()); if (bufferMode.isFullyBuffered()) { @@ -1113,7 +1118,7 @@ } - } + } /** * Used to re-open the {@link FileChannel} in this class. Modified: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/IBufferStrategy.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/IBufferStrategy.java 2010-11-03 18:00:50 UTC (rev 3883) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/IBufferStrategy.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -218,7 +218,8 @@ * data. For most strategies the action is void since the client WORM DISK * strategy writes data as allocated. For the Read Write Strategy more data * must be managed as part of the protocol outside of the RootBlock, and - * this is the method that triggers that management. + * this is the method that triggers that management. The caller MUST provide + * appropriate synchronization. * * @param abstractJournal */ Modified: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RWStrategy.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RWStrategy.java 2010-11-03 18:00:50 UTC (rev 3883) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RWStrategy.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -29,7 +29,6 @@ import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.util.UUID; -import java.util.concurrent.locks.ReentrantLock; import org.apache.log4j.Logger; @@ -56,10 +55,9 @@ * * @author Martyn Cutcher * - * @todo review life cycle state changes and refusal of methods when the backing - * store is closed. - * - * @todo Implement use of IByteArraySlice as alternative to ByteBuffer + * FIXME Review life cycle state changes and refusal of methods when the + * backing store is closed. m_open should probably be moved into RWStore + * which could then expose an isOpen() method to be used by this class. */ public class RWStrategy extends AbstractRawStore implements IBufferStrategy, IHABufferStrategy { @@ -87,21 +85,12 @@ */ final private long m_initialExtent; - /** - * @todo The use of this lock is suspicious. It is only used by - * {@link #commit(IJournal)} and that method is invoked by the - * {@link AbstractJournal#commitNow(long)} which is already protected - * by a lock. - */ - private final ReentrantLock m_commitLock = new ReentrantLock(); - /** - * It is important to ensure that the RWStrategy keeps a check on the physical root blocks and uses - * to manage re-opening of the store. * * @param fileMetadata + * @param quorum */ - RWStrategy(final FileMetadata fileMetadata, final Quorum<?,?> quorum) { + RWStrategy(final FileMetadata fileMetadata, final Quorum<?, ?> quorum) { m_uuid = fileMetadata.rootBlock.getUUID(); @@ -143,6 +132,7 @@ m_store.getData(rwaddr, buf); return ByteBuffer.wrap(buf, 0, sze); + } public long write(final ByteBuffer data) { @@ -161,7 +151,7 @@ if (data.hasArray() && data.arrayOffset() != 0) { /* - * FIXME [data] is not always backed by an array, the array may not + * @todo [data] is not always backed by an array, the array may not * be visible (read-only), the array offset may not be zero, etc. * Try to drive the ByteBuffer into the RWStore.alloc() method * instead. @@ -200,7 +190,9 @@ } private int decodeSize(final long addr) { - return (int) (addr & 0xFFFFFFFF); + + return (int) (addr & 0xFFFFFFFF); + } public void delete(final long addr) { @@ -246,9 +238,10 @@ } /** - * FIXME Define and implement support for counters. The pattern for this - * method is to always return a new object so it may be attached to various - * points in hierarchies belonging to the caller. + * @todo Define and implement support for counters. The pattern for this + * method is to always return a new object so it may be attached to + * various points in hierarchies belonging to the caller. See the + * {@link WORMStrategy} for examples. */ public CounterSet getCounters() { @@ -363,18 +356,10 @@ } - /** - * Commit must use a commit lock to synchronize the rootBlock with the commit. - * - * Must pass in earliestTxnTime to commitChanges to enable - */ - public void commit(final IJournal journal) { - m_commitLock.lock(); - try { - m_store.commitChanges((Journal) journal); // includes a force(false) - } finally { - m_commitLock.unlock(); - } + public void commit(final IJournal journal) { + + m_store.commitChanges((Journal) journal); // includes a force(false) + } /** @@ -545,7 +530,9 @@ * IHABufferStrategy */ - // FIXME writeRawBuffer + /** + * Operation is not supported. + */ public void writeRawBuffer(HAWriteMessage msg, ByteBuffer b) throws IOException, InterruptedException { @@ -553,7 +540,9 @@ } - // FIXME readFromLocalStore + /** + * Operation is not supported. + */ public ByteBuffer readFromLocalStore(final long addr) throws InterruptedException { Added: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RootBlockUtility.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RootBlockUtility.java (rev 0) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RootBlockUtility.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -0,0 +1,116 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2010. All rights reserved. + +Contact: + SYSTAP, LLC + 4501 Tower Road + Greensboro, NC 27410 + lic...@bi... + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; version 2 of the License. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA +*/ +/* + * Created on Nov 3, 2010 + */ + +package com.bigdata.journal; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +import org.apache.log4j.Logger; + +import com.bigdata.io.FileChannelUtility; +import com.bigdata.io.IReopenChannel; +import com.bigdata.util.ChecksumUtility; + +/** + * Utility class will read both root blocks of a file and indicate which one + * is current. + * + * @author <a href="mailto:tho...@us...">Bryan + * Thompson</a> + * @version $Id$ + */ +public class RootBlockUtility { + + private static final Logger log = Logger.getLogger(RootBlockUtility.class); + + /** + * The 1st root block. + */ + public IRootBlockView rootBlock0; + + /** + * The 2nd root block. + */ + public IRootBlockView rootBlock1; + + /** + * The current root block. For a new file, this is "rootBlock0". For an + * existing file it is based on an examination of both root blocks. + */ + public final IRootBlockView rootBlock; + + public RootBlockUtility(final IReopenChannel<FileChannel> opener, + final File file, final boolean validateChecksum, + final boolean alternateRootBlock) throws IOException { + + final ChecksumUtility checker = validateChecksum ? ChecksumUtility.threadChk + .get() + : null; + + final ByteBuffer tmp0 = ByteBuffer + .allocate(RootBlockView.SIZEOF_ROOT_BLOCK); + final ByteBuffer tmp1 = ByteBuffer + .allocate(RootBlockView.SIZEOF_ROOT_BLOCK); + FileChannelUtility.readAll(opener, tmp0, FileMetadata.OFFSET_ROOT_BLOCK0); + FileChannelUtility.readAll(opener, tmp1, FileMetadata.OFFSET_ROOT_BLOCK1); + tmp0.position(0); // resets the position. + tmp1.position(0); + try { + rootBlock0 = new RootBlockView(true, tmp0, checker); + } catch (RootBlockException ex) { + log.warn("Bad root block zero: " + ex); + } + try { + rootBlock1 = new RootBlockView(false, tmp1, checker); + } catch (RootBlockException ex) { + log.warn("Bad root block one: " + ex); + } + if (rootBlock0 == null && rootBlock1 == null) { + throw new RuntimeException( + "Both root blocks are bad - journal is not usable: " + file); + } + if (alternateRootBlock) + log.warn("Using alternate root block"); + /* + * Choose the root block based on the commit counter. + * + * Note: The commit counters MAY be equal. This will happen if we + * rollback the journal and override the current root block with the + * alternate root block. + */ + final long cc0 = rootBlock0 == null ? -1L : rootBlock0 + .getCommitCounter(); + final long cc1 = rootBlock1 == null ? -1L : rootBlock1 + .getCommitCounter(); + this.rootBlock = (cc0 > cc1 ? (alternateRootBlock ? rootBlock1 + : rootBlock0) : (alternateRootBlock ? rootBlock0 : rootBlock1)); + } + +} Property changes on: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/journal/RootBlockUtility.java ___________________________________________________________________ Added: svn:keywords + Id Date Revision Author HeadURL Modified: branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2010-11-03 18:00:50 UTC (rev 3883) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -64,6 +64,7 @@ import com.bigdata.journal.Journal; import com.bigdata.journal.JournalTransactionService; import com.bigdata.journal.Options; +import com.bigdata.journal.RootBlockUtility; import com.bigdata.journal.RootBlockView; import com.bigdata.quorum.Quorum; import com.bigdata.util.ChecksumUtility; @@ -236,7 +237,7 @@ // /////////////////////////////////////////////////////////////////////////////////////// private final File m_fd; - private RandomAccessFile m_raf; +// private RandomAccessFile m_raf; // protected FileMetadata m_metadata; // protected int m_transactionCount; // private boolean m_committing; @@ -392,7 +393,7 @@ // private final FileMetadataView m_fmv; - private IRootBlockView m_rb; +// private volatile IRootBlockView m_rb; // volatile private long m_commitCounter; @@ -417,18 +418,18 @@ m_metaBitsSize = cDefaultMetaBitsSize; m_metaBits = new int[m_metaBitsSize]; + m_metaTransientBits = new int[m_metaBitsSize]; - + + // @todo Review maximum file size constraints - is this old stuff? m_maxFileSize = 2 * 1024 * 1024; // 1gb max (mult by 128)!! m_quorum = quorum; m_fd = fileMetadata.file; - m_raf = fileMetadata.getRandomAccessFile(); + final IRootBlockView m_rb = fileMetadata.rootBlock; - m_rb = fileMetadata.rootBlock; - m_commitList = new ArrayList<Allocator>(); m_allocs = new ArrayList<Allocator>(); @@ -436,6 +437,7 @@ 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); @@ -453,7 +455,7 @@ log.info("RWStore using writeCacheService with buffers: " + buffers); try { - m_writeCache = new RWWriteCacheService(buffers, m_raf.length(), + m_writeCache = new RWWriteCacheService(buffers, m_fd.length(), m_reopener, m_quorum) { public WriteCache newWriteCache(final ByteBuffer buf, @@ -511,7 +513,7 @@ m_fileSize = m_nextAllocation; } - m_raf.setLength(convertAddr(m_fileSize)); + m_reopener.raf.setLength(convertAddr(m_fileSize)); m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; m_minFixedAlloc = m_allocSizes[0]*64; @@ -520,7 +522,7 @@ } else { - initfromRootBlock(); + initfromRootBlock(m_rb); m_maxFixedAlloc = m_allocSizes[m_allocSizes.length-1]*64; m_minFixedAlloc = m_allocSizes[0]*64; @@ -541,7 +543,7 @@ m_bufferedWrite = null; } m_writeCache.close(); - m_raf.close(); + m_reopener.raf.close(); } catch (Throwable t) { throw new RuntimeException(t); } @@ -583,12 +585,12 @@ log.trace("m_allocation: " + nxtalloc + ", m_metaBitsAddr: " + metaBitsAddr + ", m_commitCounter: " + commitCounter); - /** - * Ensure rootblock is in sync with external request - * - * FIXME No side-effect please. - */ - m_rb = rbv; +// /** +// * Ensure rootblock is in sync with external request +// * +// * FIXME No side-effect please. +// */ +// m_rb = rbv; } /** @@ -609,7 +611,7 @@ * * @throws IOException */ - private void initfromRootBlock() throws IOException { + private void initfromRootBlock(final IRootBlockView m_rb) throws IOException { // m_rb = m_fmv.getRootBlock(); assert(m_rb != null); @@ -1334,7 +1336,7 @@ } } - int fixedAllocatorIndex(final int size) { + private int fixedAllocatorIndex(final int size) { int i = 0; int cmp = m_minFixedAlloc; @@ -1355,17 +1357,17 @@ return PSOutputStream.getNew(this, m_maxFixedAlloc, null); } - - /**************************************************************************** - * Called by PSOutputStream to make to actual allocation or directly by lower - * level API clients. - * <p> - * If the allocation is for greater than MAX_FIXED_ALLOC, then a PSOutputStream - * is used to manage the chained buffers. - * - * TODO: Instead of using PSOutputStream instead manage allocations written - * to the WriteCacheService, building BlobHeader as you go. - **/ + + /**************************************************************************** + * Called by PSOutputStream to make to actual allocation or directly by + * lower level API clients. + * <p> + * If the allocation is for greater than MAX_FIXED_ALLOC, then a + * PSOutputStream is used to manage the chained buffers. + * + * TODO: Instead of using PSOutputStream, manage allocations written to the + * WriteCacheService, building BlobHeader as you go. + **/ public long alloc(final byte buf[], final int size, final IAllocationContext context) { if (size > (m_maxFixedAlloc-4)) { if (size > (BLOB_FIXED_ALLOCS * (m_maxFixedAlloc-4))) @@ -1474,8 +1476,15 @@ } m_allocationLock.lock(); try { - checkRootBlock(m_rb); - m_commitList.clear(); + + final RootBlockUtility tmp = new RootBlockUtility(m_reopener, m_fd, + true/* validateChecksum */, false/* alternateRootBlock */); + + final IRootBlockView rootBlock = tmp.rootBlock; + + checkRootBlock(rootBlock); + + m_commitList.clear(); m_allocs.clear(); m_freeBlobs.clear(); @@ -1491,7 +1500,7 @@ throw new RuntimeException(e); } - initfromRootBlock(); + initfromRootBlock(rootBlock); // notify of current file length. m_writeCache.setExtent(convertAddr(m_fileSize)); @@ -1551,80 +1560,6 @@ static final float s_version = 3.0f; -// /** -// * This must now update the root block which is managed by FileMetadata in -// * almost guaranteed secure manner. -// * -// * It is not the responsibility of the store to write this out, this is -// * handled by whatever is managing the FileMetadata that this RWStore was -// * initialised from and should be forced by newRootBlockView. -// * -// * It should now only be called by extend file to ensure that the metaBits -// * are set correctly. -// * -// * In order to ensure that the new block is the one that would be chosen, we need to -// * duplicate the rootBlock. This does mean that we lose the ability to roll -// * back the commit. It also means that until that point there is an invalid store state. -// * Both rootBlocks would be valid but with different extents. This is fine at -// * that moment, but subsequent writes would effectively cause the initial rootBlock -// * to reference invalid allocation blocks. -// * -// * In any event we need to duplicate the rootblocks since any rootblock that references -// * the old allocation area will be invalid. -// * -// * TODO: Should be replaced with specific updateExtendedMetaData that will -// * simply reset the metaBitsAddr -// * @throws IOException -// */ -// protected void writeFileSpec() throws IOException { -// -// m_rb = m_fmv.newRootBlockView(// -// !m_rb.isRootBlock0(), // -// m_rb.getOffsetBits(), // -// getNextOffset(), // -// m_rb.getFirstCommitTime(),// -// m_rb.getLastCommitTime(), // -// m_rb.getCommitCounter(), // -// m_rb.getCommitRecordAddr(),// -// m_rb.getCommitRecordIndexAddr(), // -// getMetaStartAddr(),// -// getMetaBitsAddr(), // -// m_rb.getLastCommitTime()// -// ); -// -// m_fmv.getFileMetadata().writeRootBlock(m_rb, ForceEnum.Force); -// -// } - -// float m_vers = 0.0f; -// -// protected void readFileSpec() { -// if (true) { -// throw new Error("Unexpected old format initialisation called"); -// } -// -// try { -// m_raf.seek(0); -// m_curHdrAddr = m_raf.readLong(); -// -// m_fileSize = m_raf.readInt(); -// m_metaStartAddr = m_raf.readInt(); -// -// m_vers = m_raf.readFloat(); -// -// if (m_vers != s_version) { -// String msg = "Incorrect store version : " + m_vers + " expects : " + s_version; -// -// throw new IOException(msg); -// } else { -// m_headerSize = m_raf.readInt(); -// } -// -// } catch (IOException e) { -// throw new StorageTerminalError("Unable to read file spec", e); -// } -// } - public String getVersionString() { return "RWStore " + s_version; } @@ -1696,7 +1631,7 @@ // m_commitCallback.commitComplete(); // } - m_raf.getChannel().force(false); // TODO, check if required! + m_reopener.reopenChannel().force(false); // TODO, check if required! } catch (IOException e) { throw new StorageTerminalError("Unable to commit transaction", e); } finally { @@ -2088,7 +2023,8 @@ if (log.isInfoEnabled()) log.info("Extending file to: " + toAddr); - m_raf.setLength(toAddr); + m_reopener.reopenChannel(); + m_reopener.raf.setLength(toAddr); if (log.isInfoEnabled()) log.info("Extend file done"); } catch (Throwable t) { @@ -2588,7 +2524,8 @@ } /** - * Simple implementation for a {@link RandomAccessFile} to handle the direct backing store. + * Simple implementation for a {@link RandomAccessFile} to handle the direct + * backing store. */ private static class ReopenFileChannel implements IReopenChannel<FileChannel> { @@ -3107,18 +3044,6 @@ * Note: This uses the [opener] to automatically retry the operation * in case concurrent readers are interrupting, causing an * asynchronous close of the backing channel. - * - * @todo Consider using the read lock vs the write lock of the - * extensionLock here. The advantage of the read lock is higher - * concurrency. The advantage of the write lock is that it locks out - * readers when we are writing the root blocks, which could help to - * ensure timely updates of the root blocks even if readers are - * behaving badly (lots of interrupts). - * - * FIXME Modify AbstractInterruptsTestCase to test for correct - * handling of root block writes where concurrent readers cause the - * backing store to be closed asynchronously. This code block SHOULD - * cause the root block write to eventually succeed. */ final Lock lock = m_extensionLock.readLock(); lock.lock(); @@ -3133,13 +3058,6 @@ * to the disk when we change the file size (unless the file * system updates other aspects of file metadata during normal * writes). - * - * @todo make sure the journal has already forced the writes, - * that forcing an empty cache buffer is a NOP, and that we want - * to just force the channel after we write the root blocks - * since writes were already forced on each node in the quorum - * before we wrote the root blocks and the root blocks are - * transmitted using RMI not the write pipeline. */ // sync the disk. Modified: branches/JOURNAL_HA_BRANCH/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java =================================================================== --- branches/JOURNAL_HA_BRANCH/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java 2010-11-03 18:00:50 UTC (rev 3883) +++ branches/JOURNAL_HA_BRANCH/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java 2010-11-03 18:53:49 UTC (rev 3884) @@ -1023,10 +1023,10 @@ Journal store = (Journal) getStore(); - RWStrategy bs = (RWStrategy) store.getBufferStrategy(); + final RWStrategy bs = (RWStrategy) store.getBufferStrategy(); - RWStore rw = bs.getRWStore(); - long realAddr = 0; + final RWStore rw = bs.getRWStore(); +// long realAddr = 0; try { // allocBatch(store, 1, 32, 650, 100000000); allocBatch(store, 1, 32, 650, 5000000); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |