From: <tho...@us...> - 2013-03-18 15:03:37
|
Revision: 7007 http://bigdata.svn.sourceforge.net/bigdata/?rev=7007&view=rev Author: thompsonbry Date: 2013-03-18 15:03:28 +0000 (Mon, 18 Mar 2013) Log Message: ----------- Added awaitHAReady() methods on AbstractJournal - this is not yet integrated into HAJournalServer. This is exposed through HAGlue.awaitHAReady(). Added status?HA to obtain an HTTP status message that is HA specific. These additions are not yet tested. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-18 13:40:26 UTC (rev 7006) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-18 15:03:28 UTC (rev 7007) @@ -29,6 +29,7 @@ import java.security.DigestException; import java.security.NoSuchAlgorithmException; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import com.bigdata.ha.msg.IHADigestRequest; @@ -41,6 +42,8 @@ import com.bigdata.journal.AbstractJournal; import com.bigdata.journal.ITransactionService; import com.bigdata.journal.Journal; +import com.bigdata.quorum.AsynchronousQuorumCloseException; +import com.bigdata.quorum.QuorumException; import com.bigdata.service.IService; /** @@ -89,6 +92,29 @@ */ public Future<Void> enterErrorState() throws IOException; + /** + * Await the service being ready to partitipate in an HA quorum. The + * preconditions include: + * <ol> + * <li>receiving notice of the quorum token via + * {@link #setQuorumToken(long)}</li> + * <li>The service is joined with the met quorum for that token</li> + * <li>If the service is a follower and it's local root blocks were at + * <code>commitCounter:=0</code>, then the root blocks from the leader have + * been installed on the follower.</li> + * <ol> + * + * @param timeout + * The timeout to await this condition. + * @param units + * The units for that timeout. + * + * @return the quorum token for which the service became HA ready. + */ + public long awaitHAReady(long timeout, TimeUnit unit) throws IOException, + InterruptedException, TimeoutException, QuorumException, + AsynchronousQuorumCloseException; + /* * Synchronization. * Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-18 13:40:26 UTC (rev 7006) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-18 15:03:28 UTC (rev 7007) @@ -30,10 +30,9 @@ import java.security.NoSuchAlgorithmException; import java.util.UUID; import java.util.concurrent.Future; -import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.msg.IHA2PhaseAbortMessage; import com.bigdata.ha.msg.IHA2PhaseCommitMessage; import com.bigdata.ha.msg.IHA2PhasePrepareMessage; @@ -56,6 +55,8 @@ import com.bigdata.ha.msg.IHAWriteSetStateRequest; import com.bigdata.ha.msg.IHAWriteSetStateResponse; import com.bigdata.journal.ValidationError; +import com.bigdata.quorum.AsynchronousQuorumCloseException; +import com.bigdata.quorum.QuorumException; /** * Delegation pattern. @@ -247,4 +248,11 @@ return delegate.getHAWriteSetState(req); } + @Override + public long awaitHAReady(long timeout, TimeUnit unit) throws IOException, + InterruptedException, QuorumException, + AsynchronousQuorumCloseException, TimeoutException { + return delegate.awaitHAReady(timeout, unit); + } + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-18 13:40:26 UTC (rev 7006) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-18 15:03:28 UTC (rev 7007) @@ -58,6 +58,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; @@ -130,8 +131,10 @@ import com.bigdata.journal.jini.ha.HAJournal; import com.bigdata.mdi.IResourceMetadata; import com.bigdata.mdi.JournalMetadata; +import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; import com.bigdata.quorum.QuorumActor; +import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.QuorumMember; import com.bigdata.rawstore.IAllocationContext; import com.bigdata.rawstore.IAllocationManagerStore; @@ -4930,6 +4933,11 @@ * we have to wait until we observe that to cast a new vote. */ + haReadyToken = Quorum.NO_QUORUM; + + // signal HAReady condition. + haReadyCondition.signalAll(); + } else if (didMeet) { quorumToken = newValue; @@ -4938,46 +4946,61 @@ final QuorumService<HAGlue> localService = quorum.getClient(); boolean installedRBs = false; - - if (_rootBlock.getCommitCounter() == 0L - && localService.isFollower(quorumToken)) { - /* - * Take the root blocks from the quorum leader and use them. - */ - - // Remote interface for the quorum leader. - final HAGlue leader = localService.getLeader(quorumToken); + if (localService.isFollower(quorumToken)) { - log.info("Fetching root block from leader."); - final IRootBlockView leaderRB; - try { - leaderRB = leader.getRootBlock( - new HARootBlockRequest(null/* storeUUID */)) - .getRootBlock(); - } catch (IOException e) { - throw new RuntimeException(e); - } - - if (leaderRB.getCommitCounter() == 0L) { - + if (_rootBlock.getCommitCounter() == 0L) { + /* - * Installs the root blocks and does a local abort. - * - * Note: This code path is only taken when both the - * leader and the follower are at commitCounter==0L. - * This prevents us from accidentally laying down on a - * follower the root blocks corresponding to a leader - * that already has committed write sets. + * Take the root blocks from the quorum leader and use + * them. */ - localService.installRootBlocks( - leaderRB.asRootBlock(true/* rootBlock0 */), - leaderRB.asRootBlock(false/* rootBlock0 */)); - installedRBs = true; + // Remote interface for the quorum leader. + final HAGlue leader = localService + .getLeader(quorumToken); + log.info("Fetching root block from leader."); + final IRootBlockView leaderRB; + try { + leaderRB = leader + .getRootBlock( + new HARootBlockRequest(null/* storeUUID */)) + .getRootBlock(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + if (leaderRB.getCommitCounter() == 0L) { + + /* + * Installs the root blocks and does a local abort. + * + * Note: This code path is only taken when both the + * leader and the follower are at commitCounter==0L. + * This prevents us from accidentally laying down on + * a follower the root blocks corresponding to a + * leader that already has committed write sets. + */ + localService + .installRootBlocks( + leaderRB.asRootBlock(true/* rootBlock0 */), + leaderRB.asRootBlock(false/* rootBlock0 */)); + + installedRBs = true; + + } + } + // ready as follower. + haReadyToken = newValue; + + } else if (localService.isLeader(quorumToken)) { + + // ready as leader. + haReadyToken = newValue; + } if (!installedRBs) { @@ -4995,6 +5018,9 @@ } + // signal HAReady condition. + haReadyCondition.signalAll(); + } else { throw new AssertionError(); @@ -5008,8 +5034,98 @@ } } + private final Condition haReadyCondition = _fieldReadWriteLock.writeLock().newCondition(); + private volatile long haReadyToken = Quorum.NO_QUORUM; + + /** + * Await the service being ready to partitipate in an HA quorum. The + * preconditions include: + * <ol> + * <li>receiving notice of the quorum token via + * {@link #setQuorumToken(long)}</li> + * <li>The service is joined with the met quorum for that token</li> + * <li>If the service is a follower and it's local root blocks were at + * <code>commitCounter:=0</code>, then the root blocks from the leader have + * been installed on the follower.</li> + * <ol> + * + * @return the quorum token for which the service became HA ready. + */ + final public long awaitHAReady() throws InterruptedException, + AsynchronousQuorumCloseException, QuorumException { + final WriteLock lock = _fieldReadWriteLock.writeLock(); + lock.lock(); + try { + long t = Quorum.NO_QUORUM; + while (((t = haReadyToken) != Quorum.NO_QUORUM) + && getQuorum().getClient() != null) { + haReadyCondition.await(); + } + final QuorumService<?> client = getQuorum().getClient(); + if (client == null) + throw new AsynchronousQuorumCloseException(); + if (!client.isJoinedMember(t)) { + throw new QuorumException(); + } + return t; + } finally { + lock.unlock(); + } + } /** + * Await the service being ready to partitipate in an HA quorum. The + * preconditions include: + * <ol> + * <li>receiving notice of the quorum token via + * {@link #setQuorumToken(long)}</li> + * <li>The service is joined with the met quorum for that token</li> + * <li>If the service is a follower and it's local root blocks were at + * <code>commitCounter:=0</code>, then the root blocks from the leader have + * been installed on the follower.</li> + * <ol> + * + * @param timeout + * The timeout to await this condition. + * @param units + * The units for that timeout. + * + * @return the quorum token for which the service became HA ready. + */ + final public long awaitHAReady(final long timeout, final TimeUnit units) + throws InterruptedException, TimeoutException, + AsynchronousQuorumCloseException { + final WriteLock lock = _fieldReadWriteLock.writeLock(); + final long begin = System.nanoTime(); + final long nanos = units.toNanos(timeout); + long remaining = nanos; + if (!lock.tryLock(remaining, TimeUnit.NANOSECONDS)) + throw new TimeoutException(); + try { + // remaining = nanos - (now - begin) [aka elapsed] + remaining = nanos - (System.nanoTime() - begin); + long t = Quorum.NO_QUORUM; + while (((t = haReadyToken) != Quorum.NO_QUORUM) + && getQuorum().getClient() != null && remaining > 0) { + if (!haReadyCondition.await(remaining, TimeUnit.NANOSECONDS)) + throw new TimeoutException(); + remaining = nanos - (System.nanoTime() - begin); + } + final QuorumService<?> client = getQuorum().getClient(); + if (client == null) + throw new AsynchronousQuorumCloseException(); + if (remaining <= 0) + throw new TimeoutException(); + if (!client.isJoinedMember(t)) { + throw new QuorumException(); + } + return t; + } finally { + lock.unlock(); + } + } + + /** * Install identical root blocks on the journal. This is used for a few * different conditions in HA. * <ol> @@ -5352,6 +5468,15 @@ } @Override + public long awaitHAReady(final long timeout, final TimeUnit units) + throws AsynchronousQuorumCloseException, InterruptedException, + TimeoutException { + + return AbstractJournal.this.awaitHAReady(timeout, units); + + } + + @Override public IHADigestResponse computeDigest(final IHADigestRequest req) throws IOException, NoSuchAlgorithmException, DigestException { Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-18 13:40:26 UTC (rev 7006) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-18 15:03:28 UTC (rev 7007) @@ -27,6 +27,8 @@ import java.io.IOException; import java.io.PrintWriter; import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -39,6 +41,8 @@ import com.bigdata.journal.IIndexManager; import com.bigdata.journal.jini.ha.HAJournal; import com.bigdata.journal.jini.ha.SnapshotManager; +import com.bigdata.quorum.AsynchronousQuorumCloseException; +import com.bigdata.quorum.Quorum; import com.bigdata.quorum.zk.ZKQuorumImpl; import com.bigdata.zookeeper.DumpZookeeper; @@ -378,5 +382,71 @@ return -1; } - + + /** + * Special reporting request for HA status. + * + * @param req + * @param resp + * @throws TimeoutException + * @throws InterruptedException + * @throws AsynchronousQuorumCloseException + * @throws IOException + */ + public void doHAStatus(HttpServletRequest req, HttpServletResponse resp) + throws IOException { + + if (!(indexManager instanceof HAJournal)) + return; + + final HAJournal journal = (HAJournal) indexManager; + + final ZKQuorumImpl<HAGlue, QuorumService<HAGlue>> quorum = (ZKQuorumImpl<HAGlue, QuorumService<HAGlue>>) journal + .getQuorum(); + +// // The current token. +// final long quorumToken = quorum.token(); +// +// // The last valid token. +// final long lastValidToken = quorum.lastValidToken(); +// +// final int njoined = quorum.getJoined().length; + + final QuorumService<HAGlue> quorumService = quorum.getClient(); + + // check, but do not wait. + long haReadyToken = Quorum.NO_QUORUM; + try { + haReadyToken = journal.awaitHAReady(0/* timeout */, + TimeUnit.NANOSECONDS/* units */); + } catch (TimeoutException ex) { + // ignore. + } catch (AsynchronousQuorumCloseException e) { + // ignore. + } catch (InterruptedException e) { + // propagate the interrupt. + Thread.currentThread().interrupt(); + } + + final String content; + if (haReadyToken != Quorum.NO_QUORUM) { + if (quorumService.isLeader(haReadyToken)) { + content = "Leader"; + } else if (quorumService.isFollower(haReadyToken)) { + content = "Follower"; + } else { + content = "NotReady"; + } + } else { + content = "NotReady"; + } + // TODO Alternatively "max-age=1" for max-age in seconds. + resp.addHeader("Cache-Control", "no-cache"); + BigdataRDFServlet.buildResponse(resp, BigdataRDFServlet.HTTP_OK, + BigdataRDFServlet.MIME_TEXT_PLAIN, content); + + return; + + } + } Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java 2013-03-18 13:40:26 UTC (rev 7006) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java 2013-03-18 15:03:28 UTC (rev 7007) @@ -154,6 +154,13 @@ static final String SNAPSHOT = "snapshot"; /** + * Special HA status request designed for clients that poll to determine the + * status of an HAJournalServer. This option is exclusive of other + * parameters. + */ + static final String HA = "HA"; + + /** * Handles CANCEL requests (terminate a running query). */ @Override @@ -286,6 +293,15 @@ protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) throws IOException { + if (req.getParameter(HA) != null + && getIndexManager() instanceof AbstractJournal + && ((AbstractJournal) getIndexManager()).isHighlyAvailable()) { + + new HAStatusServletUtil(getIndexManager()).doHAStatus(req, resp); + + return; + } + // IRunningQuery objects currently running on the query controller. final boolean showQueries = req.getParameter(SHOW_QUERIES) != null; This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-03-18 20:25:51
|
Revision: 7010 http://bigdata.svn.sourceforge.net/bigdata/?rev=7010&view=rev Author: thompsonbry Date: 2013-03-18 20:25:42 +0000 (Mon, 18 Mar 2013) Log Message: ----------- I have added IHABufferStrategy.writeOnStore(). The caller is responsible for taking a read-lock, opening the output stream, and then writing the root blocks onto the stream. This method then writes the data (starting immediately after the root blocks) onto the stream. The current approach writes the journal's data onto a GZIPOutputStream, thus the snapshot is compressed. This is integrated into HAJournal using a temporary file rename pattern. The snapshot is written onto a temporary file in the snapshot directory. Once the file has been written, flushed, and closed it is renamed onto the target snapshot file name. If the process is interrupted, a temporary file will exist in the snapshot directory. That temporary file may be removed. Note that it is not necessary for this pattern for the file rename to be atomic. If the file winds up renamed, then it is well and good. If the file does not wind up renamed, then the snapshot was is invalid and will not be used since it will have the wrong name. I have written implementations of this writeOnChannel() method for the WORMStrategy and RWStrategy. These implementations have not yet been tested. We still need to write the test suites for the HA Backup / restore mechanisms. See https://sourceforge.net/apps/trac/bigdata/ticket/645 (HA Backup) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RWStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -28,12 +28,15 @@ package com.bigdata.journal; import java.io.IOException; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.security.DigestException; import java.security.MessageDigest; import java.util.UUID; import java.util.concurrent.Future; +import com.bigdata.ha.HAGlue; +import com.bigdata.ha.QuorumService; import com.bigdata.ha.msg.HARebuildRequest; import com.bigdata.ha.msg.IHALogRequest; import com.bigdata.ha.msg.IHARebuildRequest; @@ -42,6 +45,7 @@ import com.bigdata.io.writecache.WriteCache; import com.bigdata.io.writecache.WriteCacheService; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; /** * A highly available {@link IBufferStrategy}. @@ -208,16 +212,34 @@ void writeRawBuffer(HARebuildRequest req, IHAWriteMessage msg, ByteBuffer transfer) throws IOException; -// /** -// * Sets strategy to be ready to rebuild store -// * @param req -// */ -// void prepareForRebuild(HARebuildRequest req); -// -// /** -// * Informs strategy to reload from rebuilt store -// * @param req -// */ -// void completeRebuild(HARebuildRequest req, IRootBlockView rbv); + /** + * Write a consistent snapshot of the committed state of the backing store. + * This method writes all data starting after the root blocks. The caller is + * responsible for putting down the root blocks themselves. + * <p> + * Note: The caller is able to obtain both root blocks atomically, while the + * strategy may not be aware of the root blocks or may not be able to + * coordinate their atomic capture. + * <p> + * Note: The caller must ensure that the resulting snapshot will be + * consistent either by ensuring that no writes occur or by taking a + * read-lock that will prevent overwrites of committed state during this + * operation. + * + * @param os + * Where to write the data. + * @param quorum + * The {@link Quorum}. + * @param token + * The token that must remain valid during this operation. + * + * @throws IOException + * @throws QuorumException + * if the service is not joined with the met quorum for that + * token at any point during the operation. + */ + void writeOnStream(OutputStream os, + Quorum<HAGlue, QuorumService<HAGlue>> quorum, long token) + throws IOException, QuorumException; } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RWStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RWStrategy.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RWStrategy.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -27,6 +27,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.security.DigestException; @@ -38,7 +39,9 @@ import com.bigdata.cache.ConcurrentWeakValueCache; import com.bigdata.counters.CounterSet; +import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumRead; +import com.bigdata.ha.QuorumService; import com.bigdata.ha.msg.HARebuildRequest; import com.bigdata.ha.msg.IHALogRequest; import com.bigdata.ha.msg.IHARebuildRequest; @@ -46,6 +49,7 @@ import com.bigdata.io.IBufferAccess; import com.bigdata.mdi.IResourceMetadata; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; import com.bigdata.rawstore.AbstractRawStore; import com.bigdata.rawstore.IAddressManager; import com.bigdata.rawstore.IAllocationContext; @@ -668,7 +672,16 @@ sequence, quorumToken, fileExtent, offset, nbytes, b); } + + @Override + public void writeOnStream(final OutputStream os, + final Quorum<HAGlue, QuorumService<HAGlue>> quorum, final long token) + throws IOException, QuorumException { + m_store.writeOnStream(os, quorum, token); + + } + @Override public Object snapshotAllocators() { // TODO Auto-generated method stub Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -25,6 +25,7 @@ import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.nio.channels.Channel; @@ -46,9 +47,11 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.Instrument; import com.bigdata.counters.striped.StripedCounters; +import com.bigdata.ha.HAGlue; import com.bigdata.ha.HAPipelineGlue; import com.bigdata.ha.QuorumPipeline; import com.bigdata.ha.QuorumRead; +import com.bigdata.ha.QuorumService; import com.bigdata.ha.msg.HARebuildRequest; import com.bigdata.ha.msg.HAWriteMessage; import com.bigdata.ha.msg.IHALogRequest; @@ -63,6 +66,7 @@ import com.bigdata.io.writecache.WriteCacheCounters; import com.bigdata.io.writecache.WriteCacheService; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; import com.bigdata.rawstore.IRawStore; import com.bigdata.util.ChecksumError; import com.bigdata.util.ChecksumUtility; @@ -2566,6 +2570,127 @@ } + @Override + public void writeOnStream(final OutputStream os, + final Quorum<HAGlue, QuorumService<HAGlue>> quorum, final long token) + throws IOException, QuorumException { + + IBufferAccess buf = null; + try { + + try { + // Acquire a buffer. + buf = DirectBufferPool.INSTANCE.acquire(); + } catch (InterruptedException ex) { + // Wrap and re-throw. + throw new IOException(ex); + } + + // The backing ByteBuffer. + final ByteBuffer b = buf.buffer(); + + // The capacity of that buffer (typically 1MB). + final int bufferCapacity = b.capacity(); + + // A big enough byte[]. + final byte[] a = new byte[bufferCapacity]; + + // The size of the root blocks (which we skip). + final int headerSize = FileMetadata.headerSize0; + + /* + * The size of the file at the moment we begin. We will not + * replicate data on new extensions of the file. Those data will + * be captured by HALog files that are replayed by the service + * that is doing the rebuild. + */ + final long fileExtent = getExtent(); + + // The #of bytes to be transmitted. + final long totalBytes = fileExtent - headerSize; + + // The #of bytes remaining. + long remaining = totalBytes; + + // The offset from which data is retrieved. + long offset = headerSize; + + long sequence = 0L; + + if (log.isInfoEnabled()) + log.info("Writing on stream: nbytes=" + totalBytes); + + while (remaining > 0) { + + int nbytes = (int) Math.min((long) bufferCapacity, + remaining); + + if (sequence == 0L && nbytes == bufferCapacity + && remaining > bufferCapacity) { + + /* + * Adjust the first block so the remainder will be + * aligned on the bufferCapacity boundaries (IO + * efficiency). + */ + nbytes -= headerSize; + + } + + if (log.isDebugEnabled()) + log.debug("Writing block: sequence=" + sequence + + ", offset=" + offset + ", nbytes=" + nbytes); + + // read direct from store + final ByteBuffer clientBuffer = b; + clientBuffer.position(0); + clientBuffer.limit(nbytes); + + readRaw(/*nbytes,*/ offset, clientBuffer); + + assert clientBuffer.remaining() > 0 : "Empty buffer: " + clientBuffer; + + if (BytesUtil + .toArray(clientBuffer, false/* forceCopy */, a/* dst */) != a) { + + // Should have copied into our array. + throw new AssertionError(); + + } + + // write onto the stream. + os.write(a, 0/* off */, nbytes/* len */); + + remaining -= nbytes; + + offset += nbytes; + + sequence++; + + if (!quorum.getClient().isJoinedMember(token)) + throw new QuorumException(); + + } + + if (log.isInfoEnabled()) + log.info("Wrote on stream: #blocks=" + sequence + ", #bytes=" + + (fileExtent - headerSize)); + + } finally { + + if (buf != null) { + try { + // Release the direct buffer. + buf.release(); + } catch (InterruptedException e) { + log.warn(e); + } + } + + } + + } + @Override public void setExtentForLocalStore(final long extent) throws IOException, InterruptedException { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -1,7 +1,6 @@ package com.bigdata.rwstore; import com.bigdata.journal.IBufferStrategy; -import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.RWStrategy; import com.bigdata.rawstore.IAllocationManagerStore; import com.bigdata.rwstore.sector.MemStrategy; Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -29,6 +29,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.nio.channels.Channel; @@ -65,8 +66,10 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.Instrument; import com.bigdata.counters.striped.StripedCounters; +import com.bigdata.ha.HAGlue; import com.bigdata.ha.HAPipelineGlue; import com.bigdata.ha.QuorumPipeline; +import com.bigdata.ha.QuorumService; import com.bigdata.ha.msg.HAWriteMessage; import com.bigdata.ha.msg.IHALogRequest; import com.bigdata.ha.msg.IHARebuildRequest; @@ -93,6 +96,7 @@ import com.bigdata.journal.RootBlockView; import com.bigdata.journal.StoreTypeEnum; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; import com.bigdata.rawstore.IAllocationContext; import com.bigdata.rawstore.IPSOutputStream; import com.bigdata.rawstore.IRawStore; @@ -5336,6 +5340,130 @@ } /** + * @see IHABufferStrategy#writeOnStream(OutputStream, Quorum, long) + */ + public void writeOnStream(final OutputStream os, + final Quorum<HAGlue, QuorumService<HAGlue>> quorum, final long token) + throws IOException, QuorumException { + + IBufferAccess buf = null; + try { + + try { + // Acquire a buffer. + buf = DirectBufferPool.INSTANCE.acquire(); + } catch (InterruptedException ex) { + // Wrap and re-throw. + throw new IOException(ex); + } + + // The backing ByteBuffer. + final ByteBuffer b = buf.buffer(); + + // The capacity of that buffer (typically 1MB). + final int bufferCapacity = b.capacity(); + + // A big enough byte[]. + final byte[] a = new byte[bufferCapacity]; + + // The size of the root blocks (which we skip). + final int headerSize = FileMetadata.headerSize0; + + /* + * The size of the file at the moment we begin. We will not + * replicate data on new extensions of the file. Those data will + * be captured by HALog files that are replayed by the service + * that is doing the rebuild. + */ +// final long fileExtent = getExtent(); + final long fileExtent = getStoreFile().length(); + + // The #of bytes to be transmitted. + final long totalBytes = fileExtent - headerSize; + + // The #of bytes remaining. + long remaining = totalBytes; + + // The offset from which data is retrieved. + long offset = headerSize; + + long sequence = 0L; + + if (log.isInfoEnabled()) + log.info("Writing on stream: nbytes=" + totalBytes); + + while (remaining > 0) { + + int nbytes = (int) Math.min((long) bufferCapacity, + remaining); + + if (sequence == 0L && nbytes == bufferCapacity + && remaining > bufferCapacity) { + + /* + * Adjust the first block so the remainder will be + * aligned on the bufferCapacity boundaries (IO + * efficiency). + */ + nbytes -= headerSize; + + } + + if (log.isDebugEnabled()) + log.debug("Writing block: sequence=" + sequence + + ", offset=" + offset + ", nbytes=" + nbytes); + + // read direct from store + final ByteBuffer clientBuffer = b; + clientBuffer.position(0); + clientBuffer.limit(nbytes); + + readRaw(/*nbytes,*/ offset, clientBuffer); + + assert clientBuffer.remaining() > 0 : "Empty buffer: " + clientBuffer; + + if (BytesUtil + .toArray(clientBuffer, false/* forceCopy */, a/* dst */) != a) { + + // Should have copied into our array. + throw new AssertionError(); + + } + + // write onto the stream. + os.write(a, 0/* off */, nbytes/* len */); + + remaining -= nbytes; + + offset += nbytes; + + sequence++; + + if (!quorum.getClient().isJoinedMember(token)) + throw new QuorumException(); + + } + + if (log.isInfoEnabled()) + log.info("Wrote on stream: #blocks=" + sequence + ", #bytes=" + + (fileExtent - headerSize)); + + } finally { + + if (buf != null) { + try { + // Release the direct buffer. + buf.release(); + } catch (InterruptedException e) { + log.warn(e); + } + } + + } + + } + + /** * 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. Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-18 17:40:12 UTC (rev 7009) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-18 20:25:42 UTC (rev 7010) @@ -26,8 +26,9 @@ import java.io.File; import java.io.FileFilter; import java.io.FileNotFoundException; +import java.io.FileOutputStream; import java.io.IOException; -import java.io.RandomAccessFile; +import java.io.OutputStream; import java.io.Serializable; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -48,6 +49,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.zip.GZIPOutputStream; import net.jini.config.Configuration; import net.jini.config.ConfigurationException; @@ -59,6 +61,7 @@ import org.apache.log4j.Logger; +import com.bigdata.btree.BytesUtil; import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumService; @@ -573,6 +576,7 @@ private class SnapshotTask implements Callable<IRootBlockView> { public SnapshotTask() { + } public IRootBlockView call() throws Exception { @@ -613,165 +617,100 @@ final IRootBlockView currentRootBlock = RootBlockUtility .chooseRootBlock(rootBlocks[0], rootBlocks[1]); - // TODO SNAPSHOT: It is a problem if this file exists and is not - // (logically) empty. final File file = getSnapshotManager().getSnapshotFile( currentRootBlock.getCommitCounter()); - // FIXME SNAPSHOT: Modify to use GZip on outputStream. - IBufferAccess buf = null; - RandomAccessFile raf = null; - try { + if (file.exists() && file.length() != 0L) { - try { - // Acquire a buffer. - buf = DirectBufferPool.INSTANCE.acquire(); - } catch (InterruptedException ex) { - // Wrap and re-throw. - throw new IOException(ex); - } - - raf = new RandomAccessFile(file, "rw"); - - // The backing ByteBuffer. - final ByteBuffer b = buf.buffer(); - - // The capacity of that buffer (typically 1MB). - final int bufferCapacity = b.capacity(); - - // The size of the root blocks (which we skip). - final int headerSize = FileMetadata.headerSize0; - /* - * The size of the file at the moment we begin. We will not - * replicate data on new extensions of the file. Those data will - * be captured by HALog files that are replayed by the service - * that is doing the rebuild. + * Snapshot exists and is not (logically) empty. + * + * TODO Make sure that we do not attempt to re-generate a + * snapshot if we already have one for the same commit point. If + * somebody wants to generate a new one for that commit point, + * they need to remove the old one first. */ - final long fileExtent = getBufferStrategy().getExtent(); - - // The #of bytes to be transmitted. - final long totalBytes = fileExtent - headerSize; - // The #of bytes remaining. - long remaining = totalBytes; + throw new IOException("File exists: " + file); - // The offset from which data is retrieved. - long offset = headerSize; - - long sequence = 0L; - - if (haLog.isInfoEnabled()) - haLog.info("Writing snapshot: file=" + file + ", nbytes=" - + totalBytes); + } - while (remaining > 0) { + /* + * Create a temporary file. We will write the snapshot here. The + * file will be renamed onto the target file name iff the snapshot + * is successfully written. + */ + final File tmp = File.createTempFile("snapshot", ".tmp", + file.getParentFile()); - int nbytes = (int) Math.min((long) bufferCapacity, - remaining); - - if (sequence == 0L && nbytes == bufferCapacity - && remaining > bufferCapacity) { - - /* - * Adjust the first block so the remainder will be - * aligned on the bufferCapacity boundaries (IO - * efficiency). - */ - nbytes -= headerSize; + OutputStream os = null; + boolean success = false; + try { - } + os = new GZIPOutputStream(new FileOutputStream(tmp)); - if (haLog.isDebugEnabled()) - haLog.debug("Sending block: sequence=" + sequence - + ", offset=" + offset + ", nbytes=" + nbytes); + // root block 0 + os.write(BytesUtil.toArray(rootBlocks[0].asReadOnlyBuffer())); - if (!getQuorum().getClient().isJoinedMember(token)) { - /* - * Abort the snapshot if the service leaves the quorum - * or if the quorum breaks. - */ - throw new QuorumException( - "Snapshot aborted: service not joined with met quorum."); - } - - if(true) { - /* - * FIXME SNAPSHOT: We have to actually read the block - * off of the backing store and then write it onto the - * file channel. - * - * FIXME SNAPSHOT: Make sure to write out empty root - * blocks first. - * - * FIXME SNAPSHOT: Skip over the root blocks before - * writing the data blocks. - * - * FIXME SNAPSHOT: Write the root blocks afterwards, - * ideally with the checksum of the data blocks. - * - * FIXME SNAPSHOT: Add compression. - */ - throw new UnsupportedOperationException(); - } - -// FileChannelUtility.writeAll(opener, data, pos); -// -// final Future<?> snd = getBufferStrategy().sendRawBuffer(req, sequence, -// quorumToken, fileExtent, offset, nbytes, b); -// -// snd.get(); // wait for data sent! + // root block 1 + os.write(BytesUtil.toArray(rootBlocks[1].asReadOnlyBuffer())); - remaining -= nbytes; - - offset += nbytes; + // and the data. + ((IHABufferStrategy) getBufferStrategy()).writeOnStream(os, + getQuorum(), token); - sequence++; - - } + // flush the output stream. + os.flush(); - /* - * FIXME SNAPSHOT: Lay down the root blocks. - */ + // done. + success = true; + } finally { + + // Release the read lock. + abort(txId); + + if (os != null) { + try { + os.close(); + } finally { + // ignore. + } + } + + } + + /* + * Either rename the temporary file onto the target filename or + * delete the tempoary file. The snapshot is not considered to + * be valid until it is found under the appropriate name. + */ + if (success) { + if (!getQuorum().getClient().isJoinedMember(token)) { // Verify before putting down the root blocks. throw new QuorumException( "Snapshot aborted: service not joined with met quorum."); } - if (haLog.isInfoEnabled()) - haLog.info("Sent store file: #blocks=" + sequence - + ", #bytes=" + (fileExtent - headerSize)); + if (!tmp.renameTo(file)) { - // Done. - return currentRootBlock; + log.error("Could not rename " + tmp + " as " + file); - } finally { - - if (buf != null) { - try { - // Release the direct buffer. - buf.release(); - } catch (InterruptedException e) { - haLog.warn(e); - } } - - if (raf != null) { - try { - raf.close(); - } finally { - // ignore. - } + } else { + + if (!tmp.delete()) { + + log.warn("Could not delete temporary file: " + tmp); } - - // Release the read lock. - abort(txId); } + + // Done. + return currentRootBlock; } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-03-18 21:11:44
|
Revision: 7013 http://bigdata.svn.sourceforge.net/bigdata/?rev=7013&view=rev Author: thompsonbry Date: 2013-03-18 21:11:38 +0000 (Mon, 18 Mar 2013) Log Message: ----------- The routine to read the root blocks of the snapshot needs to be modified. They are now compressed (GZip) files. Also, the file header needs to include not just the root blocks but also the magic and file version fields. It is NOT just sizeof(rootBlock)*2. See FileMetadata and RootBlockUtility. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/FileMetadata.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/FileMetadata.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/FileMetadata.java 2013-03-18 20:39:17 UTC (rev 7012) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/FileMetadata.java 2013-03-18 21:11:38 UTC (rev 7013) @@ -71,15 +71,22 @@ /** * Magic value for journal (the root blocks have their own magic value). */ - final int MAGIC = 0xe6b4c275; + public static final int MAGIC = 0xe6b4c275; /** - * Journal version number (version 1). @todo What relation is there between - * this and version info in the root blocks? + * The original journal version number (version 1). */ - final int VERSION1 = 0x1; + public static final int VERSION1 = 0x1; /** + * The current journal version number. + * <p> + * Note: This could be used to create major versioning changes of the file + * structure. There is only one version to date. + */ + public static final int CURRENT_VERSION = VERSION1; + + /** * The file that was opened. */ public final File file; @@ -706,7 +713,7 @@ magic = MAGIC; - version = VERSION1; + version = CURRENT_VERSION; if (!temporary) { @@ -722,7 +729,7 @@ */ raf.seek(0); raf.writeInt(MAGIC); - raf.writeInt(VERSION1); + raf.writeInt(version); } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-18 20:39:17 UTC (rev 7012) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-18 21:11:38 UTC (rev 7013) @@ -23,6 +23,7 @@ */ package com.bigdata.journal.jini.ha; +import java.io.DataOutputStream; import java.io.File; import java.io.FileFilter; import java.io.FileNotFoundException; @@ -655,13 +656,24 @@ os = new GZIPOutputStream(new FileOutputStream(tmp)); - // root block 0 + // Write out the file header. + { + final DataOutputStream os2 = new DataOutputStream(os); + try { + os2.writeInt(FileMetadata.MAGIC); + os2.writeInt(FileMetadata.CURRENT_VERSION); + os2.flush(); + } finally { + os2.close(); + } + } + + // write out the root blocks. + os.write(BytesUtil.toArray(rootBlocks[0].asReadOnlyBuffer())); - - // root block 1 os.write(BytesUtil.toArray(rootBlocks[1].asReadOnlyBuffer())); - // and the data. + // write out the file data. ((IHABufferStrategy) getBufferStrategy()).writeOnStream(os, getQuorum(), token); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-18 20:39:17 UTC (rev 7012) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-18 21:11:38 UTC (rev 7013) @@ -23,16 +23,18 @@ */ package com.bigdata.journal.jini.ha; +import java.io.DataInputStream; import java.io.File; +import java.io.FileInputStream; import java.io.FilenameFilter; import java.io.IOException; -import java.io.RandomAccessFile; -import java.nio.channels.FileChannel; +import java.nio.ByteBuffer; import java.util.Formatter; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.zip.GZIPInputStream; import net.jini.config.Configuration; import net.jini.config.ConfigurationException; @@ -42,10 +44,11 @@ import com.bigdata.btree.IRangeQuery; import com.bigdata.btree.ITupleIterator; import com.bigdata.ha.halog.IHALogReader; -import com.bigdata.io.IReopenChannel; -import com.bigdata.io.NOPReopener; +import com.bigdata.journal.FileMetadata; import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.RootBlockUtility; +import com.bigdata.journal.RootBlockView; +import com.bigdata.util.ChecksumUtility; /** * Class to manage the snapshot files. @@ -256,27 +259,47 @@ static IRootBlockView getRootBlockForSnapshot(final File file) throws IOException { - final RandomAccessFile raf = new RandomAccessFile(file, "r"); + final byte[] b0 = new byte[RootBlockView.SIZEOF_ROOT_BLOCK]; + final byte[] b1 = new byte[RootBlockView.SIZEOF_ROOT_BLOCK]; + + final DataInputStream is = new DataInputStream(new GZIPInputStream( + new FileInputStream(file), FileMetadata.headerSize0)); try { - final IReopenChannel<FileChannel> opener = new NOPReopener( - raf.getChannel()); + final int magic = is.readInt(); - final RootBlockUtility util = new RootBlockUtility(opener, file, - true/* validateChecksum */, false/* alternateRootBlock */, - false/* ignoreBadRootBlock */); + if (magic != FileMetadata.MAGIC) + throw new IOException("Bad journal magic: expected=" + + FileMetadata.MAGIC + ", actual=" + magic); + + final int version = is.readInt(); + + if (version != FileMetadata.CURRENT_VERSION) + throw new RuntimeException("Bad journal version: expected=" + + FileMetadata.CURRENT_VERSION + ", actual=" + version); - final IRootBlockView currentRootBlock = util.chooseRootBlock(); + // read root blocks. + is.readFully(b0); + is.readFully(b1); - return currentRootBlock; - } finally { - raf.close(); + is.close(); } + final IRootBlockView rb0 = new RootBlockView(true, ByteBuffer.wrap(b0), + ChecksumUtility.getCHK()); + + final IRootBlockView rb1 = new RootBlockView(true, ByteBuffer.wrap(b1), + ChecksumUtility.getCHK()); + + final IRootBlockView currentRootBlock = RootBlockUtility + .chooseRootBlock(rb0, rb1); + + return currentRootBlock; + } void addSnapshot(final File file) throws IOException { This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-03-19 20:55:03
|
Revision: 7016 http://bigdata.svn.sourceforge.net/bigdata/?rev=7016&view=rev Author: thompsonbry Date: 2013-03-19 20:54:52 +0000 (Tue, 19 Mar 2013) Log Message: ----------- Added a test suite for the HA backup scenarios. This test suite is not yet complete. It verifies the ability to create snapshots, but does not yet validate the snapshots or the HARestore utility. Added HAGlue.takeSnapshot() method. This is in addition to the REST API for requesting a snapshot. Moved the takeSnapshot() logic entirely inside of the SnapshotManager. Added logic to the HA test suite harness to permit overrides of configuration parameters. This is used to selectively change the snapshotPolicy and retentionPolicy in the test suites which exercise that logic. HA test suite is good (worksheet updated). Reconciled an edit by MC for this commit. @see https://sourceforge.net/apps/trac/bigdata/ticket/645 (HA Backup) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java branches/READ_CACHE/src/resources/HAJournal/HAJournal.config Added Paths: ----------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotResponse.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotResponse.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -39,6 +39,8 @@ import com.bigdata.ha.msg.IHALogDigestResponse; import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; +import com.bigdata.ha.msg.IHASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.journal.AbstractJournal; import com.bigdata.journal.ITransactionService; import com.bigdata.journal.Journal; @@ -204,4 +206,17 @@ Future<Void> globalWriteLock(IHAGlobalWriteLockRequest req) throws IOException, TimeoutException, InterruptedException; + /** + * Request that the service take a snapshot. If there is already a snapshot + * in progress, then the {@link Future} for that request will be returned. + * + * @param req + * The request. + * + * @return A {@link Future} for the snapshot -or- <code>null</code> if no + * snapshot is running and none will be taken for that request. + */ + Future<IHASnapshotResponse> takeSnapshot(IHASnapshotRequest req) + throws IOException; + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -50,6 +50,8 @@ import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; @@ -244,15 +246,21 @@ @Override public IHAWriteSetStateResponse getHAWriteSetState( - IHAWriteSetStateRequest req) throws IOException { + final IHAWriteSetStateRequest req) throws IOException { return delegate.getHAWriteSetState(req); } @Override - public long awaitHAReady(long timeout, TimeUnit unit) throws IOException, - InterruptedException, QuorumException, + public long awaitHAReady(final long timeout, final TimeUnit unit) + throws IOException, InterruptedException, QuorumException, AsynchronousQuorumCloseException, TimeoutException { return delegate.awaitHAReady(timeout, unit); } + @Override + public Future<IHASnapshotResponse> takeSnapshot(final IHASnapshotRequest req) + throws IOException { + return delegate.takeSnapshot(req); + } + } Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotRequest.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -0,0 +1,55 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import java.io.Serializable; + +public class HASnapshotRequest implements IHASnapshotRequest, + Serializable { + + private static final long serialVersionUID = 1L; + + private final int percentLogSize; + + /** + * + * @param percentLogSize + * If the size on the disk of the HALog files is less than this + * percentage of the size on the disk of the journal, then the + * snapshot will not be taken. + */ + public HASnapshotRequest(final int percentLogSize) { + + this.percentLogSize = percentLogSize; + + } + + @Override + public int getPercentLogSize() { + + return percentLogSize; + + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotResponse.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -0,0 +1,60 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import java.io.Serializable; +import java.nio.ByteBuffer; + +import com.bigdata.btree.BytesUtil; +import com.bigdata.journal.IRootBlockView; +import com.bigdata.journal.RootBlockView; +import com.bigdata.util.ChecksumUtility; + +public class HASnapshotResponse implements IHASnapshotResponse, Serializable { + + private static final long serialVersionUID = 1L; + + final private boolean isRootBlock0; + final private byte[] data; + + public HASnapshotResponse(final IRootBlockView rootBlock) { + + if (rootBlock == null) + throw new IllegalArgumentException(); + + this.isRootBlock0 = rootBlock.isRootBlock0(); + + this.data = BytesUtil.toArray(rootBlock.asReadOnlyBuffer()); + + } + + @Override + public IRootBlockView getRootBlock() { + + return new RootBlockView(isRootBlock0, ByteBuffer.wrap(data), + new ChecksumUtility()); + + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotRequest.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -0,0 +1,40 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +/** + * Message requesting a snapshot. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHASnapshotRequest extends IHAMessage { + + /** + * If the size on the disk of the HALog files is less than this percentage + * of the size on the disk of the journal, then the snapshot will not be + * taken. + */ + public int getPercentLogSize(); + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotResponse.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -0,0 +1,42 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import com.bigdata.journal.IRootBlockView; + +/** + * The message when a snapshot has been taken. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHASnapshotResponse extends IHAMessage { + + /** + * The root block associated with the snapshot. + * + * @return The root block and never <code>null</code>. + */ + IRootBlockView getRootBlock(); + +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -118,6 +118,8 @@ import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; @@ -5503,6 +5505,14 @@ } + @Override + public Future<IHASnapshotResponse> takeSnapshot( + final IHASnapshotRequest req) throws IOException { + + throw new UnsupportedOperationException(); + + } + /** * Return a proxy object for a {@link Future} suitable for use in an RMI * environment (the default implementation returns its argument). Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -33,7 +33,9 @@ import com.bigdata.btree.BytesUtil; import com.bigdata.btree.Checkpoint; import com.bigdata.btree.DefaultTupleSerializer; +import com.bigdata.btree.IRangeQuery; import com.bigdata.btree.ITuple; +import com.bigdata.btree.ITupleIterator; import com.bigdata.btree.IndexMetadata; import com.bigdata.btree.keys.ASCIIKeyBuilderFactory; import com.bigdata.btree.keys.IKeyBuilder; @@ -290,6 +292,41 @@ } /** + * Find the commit counter for the most recent snapshot (if any). + * + * @return That commit counter -or- ZERO (0L) if there are no snapshots. + */ + public long getMostRecentSnapshotCommitCounter() { + + final long snapshotCommitCounter; + synchronized (this) { + + final ITupleIterator<IRootBlockView> itr = + rangeIterator(null/* fromKey */, null/* toKey */, + 1/* capacity */, IRangeQuery.DEFAULT + | IRangeQuery.REVERSE/* flags */, null/* filter */); + + if (itr.hasNext()) { + + final ITuple<IRootBlockView> t = itr.next(); + + final IRootBlockView rootBlock = t.getObject(); + + snapshotCommitCounter = rootBlock.getCommitCounter(); + + } else { + + snapshotCommitCounter = 0L; + + } + + } + + return snapshotCommitCounter; + + } + + /** * Encapsulates key and value formation. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> @@ -327,7 +364,7 @@ * Decodes the key as a commit time. */ @Override - public Long deserializeKey(ITuple tuple) { + public Long deserializeKey(final ITuple tuple) { return KeyBuilder .decodeLong(tuple.getKeyBuffer().array(), 0/* offset */); @@ -335,6 +372,20 @@ } /** + * De-serializes an object from the {@link ITuple#getValue() value} stored + * in the tuple (ignores the key stored in the tuple). + */ + public IRootBlockView deserialize(final ITuple tuple) { + + if (tuple == null) + throw new IllegalArgumentException(); + + return (IRootBlockView) new RootBlockView(false/* rootBlock0 */, + ByteBuffer.wrap(tuple.getValue()), ChecksumUtility.getCHK()); + + } + + /** * The initial version (no additional persistent state). */ private final static transient byte VERSION0 = 0; Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -30,7 +30,8 @@ import org.apache.log4j.Logger; -import com.bigdata.journal.IRootBlockView; +import com.bigdata.ha.msg.HASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; /** * Policy schedules a snapshot at the same time each day. A threshold is used to @@ -124,8 +125,8 @@ } // Conditionally start a snapshot. - final Future<IRootBlockView> f = jnl.getSnapshotManager() - .takeSnapshot(percentLogSize); + final Future<IHASnapshotResponse> f = jnl.getSnapshotManager() + .takeSnapshot(new HASnapshotRequest(percentLogSize)); if (f != null) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -23,13 +23,10 @@ */ package com.bigdata.journal.jini.ha; -import java.io.DataOutputStream; import java.io.File; import java.io.FileFilter; import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.OutputStream; import java.io.Serializable; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -50,7 +47,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.zip.GZIPOutputStream; import net.jini.config.Configuration; import net.jini.config.ConfigurationException; @@ -62,7 +58,6 @@ import org.apache.log4j.Logger; -import com.bigdata.btree.BytesUtil; import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumService; @@ -84,6 +79,8 @@ import com.bigdata.ha.msg.IHALogRootBlocksResponse; import com.bigdata.ha.msg.IHARebuildRequest; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.io.DirectBufferPool; @@ -95,13 +92,11 @@ import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.ITx; import com.bigdata.journal.Journal; -import com.bigdata.journal.RootBlockUtility; import com.bigdata.journal.ValidationError; import com.bigdata.journal.WriteExecutorService; import com.bigdata.journal.jini.ha.HAJournalServer.HAQuorumService; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; -import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.zk.ZKQuorumImpl; import com.bigdata.service.AbstractTransactionService; import com.bigdata.service.jini.JiniClient; @@ -541,211 +536,17 @@ } /** - * Take a snapshot. - * - * @param file - * Where to write the snapshot. - * - * @return The {@link Future} of the task that is taking the snapshot. The - * {@link Future} will evaluate to the closing - * {@link IRootBlockView} on the snapshot. - * - * @throws IOException - * @throws ExecutionException - * @throws InterruptedException + * {@inheritDoc} + * <p> + * Extended to expose this method to the {@link SnapshotManger}. */ - Future<IRootBlockView> takeSnapshotNow() { - - final FutureTask<IRootBlockView> ft = new FutureTaskMon<IRootBlockView>( - new SnapshotTask()); - - // Run task. - getExecutorService().submit(ft); - - return ft; - + @Override + public IRootBlockView[] getRootBlocks() { + + return super.getRootBlocks(); + } - /** - * Take a snapshot. - * - * TODO If possible, move this code to the {@link SnapshotManager} class. - * Both the {@link SnapshotManager} and the {@link HARestore} classes could - * be moved into the {@link Journal} package (and the {@link HARestore} - * class renamed since it would then be usable without the HAJournalServer). - */ - private class SnapshotTask implements Callable<IRootBlockView> { - - public SnapshotTask() { - - } - - public IRootBlockView call() throws Exception { - - // The quorum token (must remain valid through this operation). - final long token = getQuorumToken(); - - if (!getQuorum().getClient().isJoinedMember(token)) { - - /* - * Note: The service must be joined with a met quorum to take a - * snapshot. This is necessary in order to ensure that the - * snapshots are copies of a journal state that the quorum - * agrees on, otherwise we could later attempt to restore from - * an invalid state. - */ - - throw new QuorumException("Service not joined with met quorum"); - - } - - // Grab a read lock. - final long txId = newTx(ITx.READ_COMMITTED); - - /* - * Get both root blocks (atomically). - * - * Note: This is done AFTER we take the read-lock and BEFORE we - * copy the data from the backing store. These root blocks MUST - * be consistent for the leader's backing store because we are - * not recycling allocations (since the read lock has pinned - * them). The journal MIGHT go through a concurrent commit - * before we obtain these root blocks, but they are still valid - * for the data on the disk because of the read-lock. - */ - final IRootBlockView[] rootBlocks = getRootBlocks(); - - final IRootBlockView currentRootBlock = RootBlockUtility - .chooseRootBlock(rootBlocks[0], rootBlocks[1]); - - final File file = getSnapshotManager().getSnapshotFile( - currentRootBlock.getCommitCounter()); - - if (file.exists() && file.length() != 0L) { - - /* - * Snapshot exists and is not (logically) empty. - * - * Note: The SnapshotManager will not recommend taking a - * snapshot if a snapshot already exists for the current commit - * point since there is no committed delta that can be captured - * by the snapshot. - * - * This code makes sure that we do not attempt to overwrite a - * snapshot if we already have one for the same commit point. If - * you want to re-generate a snapshot for the same commit point - * (e.g., because the existing one is corrupt) then you MUST - * remove the pre-existing snapshot first. - */ - - throw new IOException("File exists: " + file); - - } - - /* - * Create a temporary file. We will write the snapshot here. The - * file will be renamed onto the target file name iff the snapshot - * is successfully written. - */ - final File tmp = File.createTempFile("snapshot", ".tmp", - file.getParentFile()); - - OutputStream os = null; - boolean success = false; - try { - - os = new GZIPOutputStream(new FileOutputStream(tmp)); - - // Write out the file header. - { - final DataOutputStream os2 = new DataOutputStream(os); - try { - os2.writeInt(FileMetadata.MAGIC); - os2.writeInt(FileMetadata.CURRENT_VERSION); - os2.flush(); - } finally { - os2.close(); - } - } - - // write out the root blocks. - - os.write(BytesUtil.toArray(rootBlocks[0].asReadOnlyBuffer())); - os.write(BytesUtil.toArray(rootBlocks[1].asReadOnlyBuffer())); - - // write out the file data. - ((IHABufferStrategy) getBufferStrategy()).writeOnStream(os, - getQuorum(), token); - - // flush the output stream. - os.flush(); - - // done. - success = true; - - } finally { - - // Release the read lock. - abort(txId); - - if (os != null) { - try { - os.close(); - } finally { - // ignore. - } - } - - } - - /* - * Either rename the temporary file onto the target filename or - * delete the tempoary file. The snapshot is not considered to - * be valid until it is found under the appropriate name. - */ - if (success) { - - if (!getQuorum().getClient().isJoinedMember(token)) { - // Verify before putting down the root blocks. - throw new QuorumException( - "Snapshot aborted: service not joined with met quorum."); - } - - if (!tmp.renameTo(file)) { - - log.error("Could not rename " + tmp + " as " + file); - - } else { - - // Add to the set of known snapshots. - getSnapshotManager().addSnapshot(file); - - /* - * FIXME This is where we need to see whether or not we can - * release an earlier snapshot and the intervening HALog - * files. - */ - - } - - } else { - - if (!tmp.delete()) { - - log.warn("Could not delete temporary file: " + tmp); - - } - - } - - // Done. - return currentRootBlock; - - } - - } // class SendStoreTask - - /** * {@inheritDoc} * <p> * Extended to expose this method to the {@link HAQuorumService}. @@ -806,10 +607,10 @@ * Note: If the quorum breaks, the service which was the leader will * invalidate all open transactions. This is handled in AbstractJournal. * - * FIXME We should really pair the quorum token with the transaction - * identifier in order to guarantee that the quorum token does not - * change (e.g., that the quorum does not break) across the scope of the - * transaction. That will require either changing the + * FIXME HA TXS: We should really pair the quorum token with the + * transaction identifier in order to guarantee that the quorum token + * does not change (e.g., that the quorum does not break) across the + * scope of the transaction. That will require either changing the * ITransactionService API and/or defining an HA variant of that API. */ @@ -1201,6 +1002,10 @@ * connections distinct KBs per the ticket below, then we will need to * have a different global write lock - perhaps via the * {@link WriteExecutorService}. + * <p> + * In fact, we could deprecate this method. It is no longer necessary to + * support backups since we can now take snapshots without suspending + * writers. * * @see https://sourceforge.net/apps/trac/bigdata/ticket/566 ( * Concurrent unisolated operations against multiple KBs on the @@ -1490,6 +1295,17 @@ } @Override + public Future<IHASnapshotResponse> takeSnapshot( + final IHASnapshotRequest req) throws IOException { + + final Future<IHASnapshotResponse> ft = getSnapshotManager() + .takeSnapshot(req); + + return ft == null ? null : getProxy(ft, true/* async */); + + } + + @Override public Future<Void> bounceZookeeperConnection() { final FutureTask<Void> ft = new FutureTaskMon<Void>( Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -2826,7 +2826,7 @@ final HALogWriter logWriter = journal.getHALogWriter(); - if (false && !isJoinedService) {//FIXME logRootBlock() + if (!isJoinedService) {//FIXME logRootBlock() /* * NOTE: Unless we are joined with the met quorum we will be Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -24,30 +24,41 @@ package com.bigdata.journal.jini.ha; import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Formatter; +import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; import net.jini.config.Configuration; import net.jini.config.ConfigurationException; import org.apache.log4j.Logger; -import com.bigdata.btree.IRangeQuery; -import com.bigdata.btree.ITupleIterator; +import com.bigdata.btree.BytesUtil; +import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.halog.IHALogReader; +import com.bigdata.ha.msg.HASnapshotResponse; +import com.bigdata.ha.msg.IHASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.journal.FileMetadata; +import com.bigdata.journal.IHABufferStrategy; import com.bigdata.journal.IRootBlockView; +import com.bigdata.journal.ITx; import com.bigdata.journal.RootBlockUtility; import com.bigdata.journal.RootBlockView; +import com.bigdata.quorum.QuorumException; import com.bigdata.util.ChecksumUtility; /** @@ -67,10 +78,20 @@ /** * The file extension for journal snapshots. */ - public final static String SNAPSHOT_EXT = ".snap"; + public final static String SNAPSHOT_EXT = ".jnl.zip"; + + /** + * The prefix for the temporary files used to generate snapshots. + */ + public final static String SNAPSHOT_TMP_PREFIX = "snapshot"; + + /** + * The suffix for the temporary files used to generate snapshots. + */ + public final static String SNAPSHOT_TMP_SUFFIX = ".tmp"; + + private final HAJournal journal; - private HAJournal journal; - /** * @see HAJournalServer.ConfigurationOptions#SNAPSHOT_DIR */ @@ -104,7 +125,7 @@ * <p> * This field is guarded by the {@link #lock}. */ - private Future<IRootBlockView> snapshotFuture = null; + private Future<IHASnapshotResponse> snapshotFuture = null; /** * Return the {@link ISnapshotPolicy}. @@ -200,18 +221,19 @@ private void populateSnapshotIndex() throws IOException { /* - * List the snapshot files for this service. + * Delete any temporary files that were left lying around in the + * snapshot directory. */ - final File[] files; { + final File[] files; final File snapshotDir = getSnapshotDir(); - + files = snapshotDir.listFiles(new FilenameFilter() { /** - * Return <code>true</code> iff the file is an HALog file - * that should be deleted. + * Return <code>true</code> iff the file is an HALog file that + * should be deleted. * * @param name * The name of that HALog file (encodes the @@ -220,6 +242,44 @@ @Override public boolean accept(final File dir, final String name) { + if (name.startsWith(SNAPSHOT_TMP_PREFIX) + && name.endsWith(SNAPSHOT_TMP_SUFFIX)) { + + // One of our temporary files. + return true; + + } + + return false; + + } + }); + + for(File file : files) { + + if(!file.delete()) { + + log.warn("Could not delete temporary file: "+file); + + } + + } + + } + + /* + * List the snapshot files for this service. + */ + final File[] files; + { + + final File snapshotDir = getSnapshotDir(); + + files = snapshotDir.listFiles(new FilenameFilter() { + + @Override + public boolean accept(final File dir, final String name) { + if (!name.endsWith(SNAPSHOT_EXT)) { // Not an snapshot file. return false; @@ -247,16 +307,12 @@ * Read the current root block out of the snapshot. * * @param file - * @return + * the file. + * @return The current root block from that file. + * * @throws IOException - * - * FIXME DETECT EMPTY SNAPSHOTS!!!! (root blocks are as if for - * an empty journal) EMPTY SNAPSHOTS SHOULD BE REMOVED ON - * STARTUP AND ARE NOT VALID AND SHOULD NOT BE USED WHEN - * CHECKING FOR THE PREVIOUS SNAPSHOT, etc. VERIFY THE - * ROOT BLOCK BEFORE RELYING ON A SNAPSHOT. */ - static IRootBlockView getRootBlockForSnapshot(final File file) + static public IRootBlockView getRootBlockForSnapshot(final File file) throws IOException { final byte[] b0 = new byte[RootBlockView.SIZEOF_ROOT_BLOCK]; @@ -272,11 +328,11 @@ if (magic != FileMetadata.MAGIC) throw new IOException("Bad journal magic: expected=" + FileMetadata.MAGIC + ", actual=" + magic); - + final int version = is.readInt(); - + if (version != FileMetadata.CURRENT_VERSION) - throw new RuntimeException("Bad journal version: expected=" + throw new IOException("Bad journal version: expected=" + FileMetadata.CURRENT_VERSION + ", actual=" + version); // read root blocks. @@ -304,6 +360,21 @@ void addSnapshot(final File file) throws IOException { + /* + * Validate the snapshot. + * + * TODO If the root blocks are bad, then this will throw an + * IOException and that will prevent the startup of the + * HAJournalServer. However, if we start up the server with a known + * bad snapshot *and* the snapshot is the earliest snapshot, then we + * can not restore commit points which depend on that earliest + * snapshot. + * + * TODO A similar problem exists if any of the HALog files GTE the + * earliest snapshot are missing, have bad root blocks, etc. We will + * not be able to restore the commit point associated with that + * HALog file unless it also happens to correspond to a snapshot. + */ final IRootBlockView currentRootBlock = getRootBlockForSnapshot(file); synchronized (snapshotIndex) { @@ -350,7 +421,7 @@ + commitTime + ", snapshot=" + currentRootBlock - + ",indexRootBlock=" + tmp); + + ", indexRootBlock=" + tmp); return false; @@ -378,7 +449,7 @@ * @return The {@link Future} of the current snapshot operation -or- * <code>null</code> if there is no snapshot operation running. */ - public Future<IRootBlockView> getSnapshotFuture() { + public Future<IHASnapshotResponse> getSnapshotFuture() { lock.lock(); @@ -418,8 +489,11 @@ * @throws ExecutionException * @throws InterruptedException */ - public Future<IRootBlockView> takeSnapshot(final int percentLogSize) { + public Future<IHASnapshotResponse> takeSnapshot(final IHASnapshotRequest req) { + if (req == null) + throw new IllegalArgumentException(); + lock.lock(); try { @@ -442,7 +516,7 @@ * longer required. */ - if (!isReadyToSnapshot(percentLogSize)) { + if (!isReadyToSnapshot(req)) { // Pre-conditions are not met. return null; @@ -450,7 +524,7 @@ } // Take the snapshot, return Future but save a reference. - return snapshotFuture = journal.takeSnapshotNow(); + return snapshotFuture = takeSnapshotNow(); } finally { @@ -464,10 +538,29 @@ * Return the snapshot {@link File} associated with the commitCounter. * * @param commitCounter - * @return + * The commit counter for the current root block on the journal. + * + * @return The name of the corresponding snapshot file. */ public File getSnapshotFile(final long commitCounter) { + + return getSnapshotFile(snapshotDir, commitCounter); + + } + /** + * Return the snapshot {@link File} associated with the commitCounter. + * + * @param snapshotDir + * The directory in which the snapshot files are stored. + * @param commitCounter + * The commit counter for the current root block on the journal. + * + * @return The name of the corresponding snapshot file. + */ + public static File getSnapshotFile(final File snapshotDir, + final long commitCounter) { + /* * Format the name of the file. * @@ -495,39 +588,6 @@ } /** - * Find the commit counter for the most recent snapshot (if any). - * - * @return That commit counter -or- ZERO (0L) if there are no snapshots. - */ - private long getMostRecentSnapshotCommitCounter() { - - final long snapshotCommitCounter; - synchronized (snapshotIndex) { - - final ITupleIterator<IRootBlockView> itr = snapshotIndex - .rangeIterator(null/* fromKey */, null/* toKey */, - 1/* capacity */, IRangeQuery.DEFAULT - | IRangeQuery.REVERSE/* flags */, null/* filter */); - - if (itr.hasNext()) { - - final IRootBlockView rootBlock = itr.next().getObject(); - - snapshotCommitCounter = rootBlock.getCommitCounter(); - - } else { - - snapshotCommitCounter = 0L; - - } - - } - - return snapshotCommitCounter; - - } - - /** * Find the commit counter for the most recent snapshot (if any). Count up * the bytes on the disk for the HALog files GTE the commitCounter of that * snapshot. If the size(halogs) as a percentage of the size(journal) is LTE @@ -540,8 +600,11 @@ * journal state that the quorum agrees on, otherwise we could later attempt * to restore from an invalid state. */ - private boolean isReadyToSnapshot(final int percentLogSize) { + private boolean isReadyToSnapshot(final IHASnapshotRequest req) { + if(req == null) + throw new IllegalArgumentException(); + final long token = journal.getQuorum().token(); if (!journal.getQuorum().getClient().isJoinedMember(token)) { @@ -553,7 +616,7 @@ } - final long snapshotCommitCounter = getMostRecentSnapshotCommitCounter(); + final long snapshotCommitCounter = snapshotIndex.getMostRecentSnapshotCommitCounter(); if (journal.getRootBlockView().getCommitCounter() == snapshotCommitCounter) { @@ -651,17 +714,20 @@ final long journalSize = journal.size(); - final double percent = ((double) totalBytes) / ((double) journalSize); - - final boolean takeSnapshot = (percent > percentLogSize); + // size(HALogs)/size(journal) as percentage. + final int actualPercentLogSize = (int) (100 * (((double) totalBytes) / ((double) journalSize))); + final int thresholdPercentLogSize = req.getPercentLogSize(); + + final boolean takeSnapshot = (actualPercentLogSize >= thresholdPercentLogSize); + if (haLog.isInfoEnabled()) { haLog.info("There are " + files.length + " HALog files since the last snapshot occupying " + totalBytes + " bytes. The journal is currently " - + journalSize + " bytes. The HALogs are " + percent - + " of the journal on the disk. A new snapshot should " + + journalSize + " bytes. The HALogs are " + actualPercentLogSize + + " of the journal on the disk. A new snapshot should" + (takeSnapshot ? "" : " not") + " be taken"); } @@ -670,4 +736,218 @@ } + /** + * Take a snapshot. + * + * @return The {@link Future} of the task that is taking the snapshot. The + * {@link Future} will evaluate to {@link IHASnapshotResponse} + * containing the closing {@link IRootBlockView} on the snapshot. + * + * @throws IOException + * @throws ExecutionException + * @throws InterruptedException + */ + Future<IHASnapshotResponse> takeSnapshotNow() { + + final FutureTask<IHASnapshotResponse> ft = new FutureTaskMon<IHASnapshotResponse>( + new SnapshotTask(this)); + + // Run task. + journal.getExecutorService().submit(ft); + + return ft; + + } + + /** + * Take a snapshot. + */ + static private class SnapshotTask implements Callable<IHASnapshotResponse> { + + private final SnapshotManager snapshotManager; + private final HAJournal journal; + + public SnapshotTask(final SnapshotManager snapshotManager) { + + if (snapshotManager == null) + throw new IllegalArgumentException(); + + this.snapshotManager = snapshotManager; + + this.journal = snapshotManager.journal; + + } + + public IHASnapshotResponse call() throws Exception { + + // The quorum token (must remain valid through this operation). + final long token = journal.getQuorumToken(); + + if (!journal.getQuorum().getClient().isJoinedMember(token)) { + + /* + * Note: The service must be joined with a met quorum to take a + * snapshot. This is necessary in order to ensure that the + * snapshots are copies of a journal state that the quorum + * agrees on, otherwise we could later attempt to restore from + * an invalid state. + */ + + throw new QuorumException("Service not joined with met quorum"); + + } + + // Grab a read lock. + final long txId = journal.newTx(ITx.READ_COMMITTED); + + /* + * Get both root blocks (atomically). + * + * Note: This is done AFTER we take the read-lock and BEFORE we copy + * the data from the backing store. These root blocks MUST be + * consistent for the leader's backing store because we are not + * recycling allocations (since the read lock has pinned them). The + * journal MIGHT go through a concurrent commit before we obtain + * these root blocks, but they are still valid for the data on the + * disk because of the read-lock. + */ + final IRootBlockView[] rootBlocks = journal.getRootBlocks(); + + final IRootBlockView currentRootBlock = RootBlockUtility + .chooseRootBlock(rootBlocks[0], rootBlocks[1]); + + final File file = snapshotManager.getSnapshotFile(currentRootBlock + .getCommitCounter()); + + if (file.exists() && file.length() != 0L) { + + /* + * Snapshot exists and is not (logically) empty. + * + * Note: The SnapshotManager will not recommend taking a + * snapshot if a snapshot already exists for the current commit + * point since there is no committed delta that can be captured + * by the snapshot. + * + * This code makes sure that we do not attempt to overwrite a + * snapshot if we already have one for the same commit point. If + * you want to re-generate a snapshot for the same commit point + * (e.g., because the existing one is corrupt) then you MUST + * remove the pre-existing snapshot first. + */ + + throw new IOException("File exists: " + file); + + } + + /* + * Create a temporary file. We will write the snapshot here. The + * file will be renamed onto the target file name iff the snapshot + * is successfully written. + */ + final File tmp = File.createTempFile( + SnapshotManager.SNAPSHOT_TMP_PREFIX, + SnapshotManager.SNAPSHOT_TMP_SUFFIX, file.getParentFile()); + + DataOutputStream os = null; + boolean success = false; + try { + + os = new DataOutputStream(new GZIPOutputStream( + new FileOutputStream(tmp))); + + // Write out the file header. + os.writeInt(FileMetadata.MAGIC); + os.writeInt(FileMetadata.CURRENT_VERSION); + + // write out the root blocks. + os.write(BytesUtil.toArray(rootBlocks[0].asReadOnlyBuffer())); + os.write(BytesUtil.toArray(rootBlocks[1].asReadOnlyBuffer())); + + // write out the file data. + ((IHABufferStrategy) journal.getBufferStrategy()) + .writeOnStream(os, journal.getQuorum(), token); + + // flush the output stream. + os.flush(); + + // done. + success = true; + } catch (Throwable t) { + /* + * Log @ ERROR and launder throwable. + */ + log.error(t, t); + if (t instanceof Exception) + throw (Exception) t; + else + throw new RuntimeException(t); + } finally { + + // Release the read lock. + journal.abort(txId); + + if (os != null) { + try { + os.close(); + } finally { + // ignore. + } + } + + /* + * Either rename the temporary file onto the target filename or + * delete the tempoary file. The snapshot is not considered to + * be valid until it is found under the appropriate name. + */ + if (success) { + + if (!journal.getQuorum().getClient().isJoinedMember(token)) { + // Verify before putting down the root blocks. + throw new QuorumException( + "Snapshot aborted: service not joined with met quorum."); + } + + if (!tmp.renameTo(file)) { + + log.error("Could not rename " + tmp + " as " + file); + + } else { + + // Add to the set of known snapshots. + snapshotManager.addSnapshot(file); + + if (haLog.isInfoEnabled()) + haLog.info("Captured snapshot: " + file + + ", commitCounter=" + + currentRootBlock.getCommitCounter() + + ", length=" + file.length()); + + /* + * FIXME SNAPSHOTS: This is where we need to see whether + * or not we can release an earlier snapshot and the + * intervening HALog files. + */ + + } + + } else { + + if (!tmp.delete()) { + + log.warn("Could not delete temporary file: " + tmp); + + } + + } + + } + + // Done. + return new HASnapshotResponse(currentRootBlock); + + } + + } // class SendStoreTask + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -74,6 +74,7 @@ import com.bigdata.jini.start.config.ServiceConfiguration.AbstractServiceStarter; import com.bigdata.jini.start.config.ZookeeperClientConfig; import com.bigdata.jini.start.process.ProcessHelper; +import com.bigdata.jini.util.ConfigMath; import com.bigdata.jini.util.JiniUtil; import com.bigdata.journal.jini.ha.HAJournalServer.ConfigurationOptions; import com.bigdata.quorum.AbstractQuorumClient; @@ -89,6 +90,7 @@ import com.bigdata.zookeeper.DumpZookeeper; import com.bigdata.zookeeper.ZooHelper; import com.bigdata.zookeeper.ZooKeeperAccessor; +import com.sun.jini.config.ConfigUtil; /** * Class layers in support to start and stop the {@link HAJournalServer} @@ -190,6 +192,21 @@ } /** + * Return any overrides to be specified together with the basic + * {@link Configuration}. Each override is the fully qualified name + * of a {@link Configuration} parameter together with its value. For + * example: + * <pre> + * com.bigdata.journal.jini.ha.HAJournalServer.snapshotPolicy=new NoSnapshotPolicy(); + * </pre> + */ + protected String[] getOverrides() { + + return new String[]{}; + + } + + /** * The {@link Remote} interfaces for these services (if started and * successfully discovered). */ @@ -215,7 +232,7 @@ private ServiceDiscoveryManager serviceDiscoveryManager = null; private HAJournalDiscoveryClient discoveryClient = null; - + /** * The {@link ZooKeeperAccessor} used by the {@link #quorum}. */ @@ -500,44 +517,51 @@ /* * Utility methods to access service HALog file directories */ - protected File getServiceDir() { - return new File("benchmark/CI-HAJournal-1"); + + /** + * The directory that is the parent of each {@link HAJournalServer}'s + * individual service directory. + */ + protected File getTestDir() { + + return new File("benchmark/CI-HAJournal-1"); + } protected File getHAJournalFileA() { - return new File(getServiceDir(),"A/bigdata-ha.jnl"); + return new File(getTestDir(), "A/bigdata-ha.jnl"); } protected File getHAJournalFileB() { - return new File(getServiceDir(),"B/bigdata-ha.jnl"); + return new File(getTestDir(), "B/bigdata-ha.jnl"); } protected File getHAJournalFileC() { - return new File(getServiceDir(),"C/bigdata-ha.jnl"); + return new File(getTestDir(), "C/bigdata-ha.jnl"); } protected File getHALogDirA() { - return new File(getServiceDir(),"A/HALog"); + return new File(getTestDir(), "A/HALog"); } - + protected File getHALogDirB() { - return new File(getServiceDir(),"B/HALog"); + return new File(getTestDir(), "B/HALog"); } - + protected File getHALogDirC() { - return new File(getServiceDir(),"C/HALog"); + return new File(getTestDir(), "C/HALog"); } - + protected File getSnapshotDirA() { - return new File(getServiceDir(),"A/snapshot"); + return new File(getTestDir(), "A/snapshot"); } - + protected File getSnapshotDirB() { - return new File(getServiceDir(),"B/snapshot"); + return new File(getTestDir(), "B/snapshot"); } - + protected File getSnapshotDirC() { - return new File(getServiceDir(),"C/snapshot"); + return new File(getTestDir(), "C/snapshot"); } /** @@ -1240,12 +1264,22 @@ /* * Read jini configuration. */ - final Configuration config = ConfigurationProvider - .getInstance(new String[] { configFile }); + // Overrides for this test. + final String[] overrides = getOverrides(); + + // Config file + overrides from perspective of this JVM. + final String[] ourArgs = ConfigMath.concat(new String[] { configFile }, + overrides); + + // Config file + overrides from perspective of the child process. + final String[] childArgs = ConfigMath.concat( + new String[] { installedConfigFileName }, overrides); + + final Configuration config = ConfigurationProvider.getInstance(ourArgs); + final ServiceConfiguration serviceConfig = new HAJournalServerConfiguration( - name, config, serviceId, serviceDir, - new String[] { installedConfigFileName }); + name, config, serviceId, serviceDir, childArgs); final AbstractServiceStarter<?> serviceStarter = serviceConfig .newServiceStarter(serviceListener); Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-19 18:19:21 UTC (rev 7015) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-19 20:54:52 UTC (rev 7016) @@ -31,7 +31,6 @@ import java.io.IOException; import java.security.DigestException; import java.security.NoSuchAlgorithmException; -import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -54,8 +53,6 @@ import com.bigdata.ha.msg.HADigestRequest; import com.bigdata.ha.msg.HALogDigestRequest; import com.bigdata.io.TestCase3; -import com.bigdata.quorum.Quorum; -import com.bigdata.quorum.QuorumClient; import com.bigdata.rdf.sail.TestConcurrentKBCreate; import com.bigdata.rdf.sail.webapp.NanoSparqlServer; import com.bigdata.rdf.sail.webapp.client.ConnectOptions; @@ -232,9 +229,6 @@ void doNSSStatusRequest(final HAGlue haGlue) throws Exception { - // The port that is exposing the NSS. - final int NSSPort = haGlue.getNSSPort(); - // Client for talking to the NSS. final HttpClient httpClient = new DefaultHttpClient(ccm); @@ -248,8 +242,9 @@ * only running one HAJournalServer here. */ { + // The NSS service URL (NOT the SPARQL end point). - final String serviceURL = "http://localhost:" + NSSPort; + final String serviceURL = getNanoSparqlServerURL(haGlue); final ConnectOptions opts = new ConnectOptions(serviceURL + "/status"); @@ -273,8 +268,26 @@ } } - + /** + * Return the {@link NanoSparqlServer} end point (NOT the SPARQL end point) + * for the specified remote service. + * + * @param haGlue + * The remove service. + * + * @return The {@link NanoSparqlServer} end point. + * + * @throws IOException + */ + protected String getNanoSparqlServerURL(final HAGlue haGlue) + throws IOException { + + return "http://localhost:" + haGlue.getNSSPort(); + + } + + /** * Retur... [truncated message content] |
From: <tho...@us...> - 2013-03-20 13:48:58
|
Revision: 7020 http://bigdata.svn.sourceforge.net/bigdata/?rev=7020&view=rev Author: thompsonbry Date: 2013-03-20 13:48:48 +0000 (Wed, 20 Mar 2013) Log Message: ----------- Added an HAStatusEnum whose values are {Leader, Follower, NotReady}. This is reported by "GET .../status?HA". The value is also available in the "GET .../status" page. Found several places where HTTP exceptions were logged but not rethrown in the HA test suite. This could lead to the test suite progressing beyond an HTTP error unintentionally. Rewrote the logic to await the HA service being ready and added explicit tests for both awaitHAReady() and GET .../status?HA. AbstractJournal: Added non-blocking variant of awaitHAReady() and modified the code paths in setQuorumToken() to support that non-blocking method. The new methods is getHAReady(). Test suite behavior is unchanged locally. Committed to get feedback from CI. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Added Paths: ----------- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -4907,7 +4907,7 @@ * Both a meet and a break require an exclusive write lock. */ final WriteLock lock = _fieldReadWriteLock.writeLock(); - + lock.lock(); try { @@ -4935,13 +4935,14 @@ * we have to wait until we observe that to cast a new vote. */ - haReadyToken = Quorum.NO_QUORUM; + haReadyToken = Quorum.NO_QUORUM; // volatile write. - // signal HAReady condition. - haReadyCondition.signalAll(); + haReadyCondition.signalAll(); // signal ALL. } else if (didMeet) { + final long tmp; + quorumToken = newValue; // This quorum member. @@ -4996,13 +4997,18 @@ } // ready as follower. - haReadyToken = newValue; + tmp = newValue; } else if (localService.isLeader(quorumToken)) { // ready as leader. - haReadyToken = newValue; + tmp = newValue; + } else { + + // Not ready. + tmp = Quorum.NO_QUORUM; + } if (!installedRBs) { @@ -5020,9 +5026,9 @@ } - log.warn("SIGNAL READY TOKEN: " + haReadyToken); - // signal HAReady condition. - haReadyCondition.signalAll(); + this.haReadyToken = tmp; // volatile write. + + haReadyCondition.signalAll(); // signal ALL. } else { @@ -5130,6 +5136,25 @@ } /** + * Returns the current value of the <code>haReadyToken</code> + * (non-blocking). + */ + final public long getHAReady() { + + /* + * Note: In order for this operation to be non-blocking and still ensure + * proper visibility of the [haReadyToken], the token MUST be volatile, + * the setQuorumToken() method MUST NOT change the value of the + * [haReadyToken] until all internal actions have been taken. That is, + * until it is willing to do haReadyCondition.signalAll() and release + * the lock guarding that Condition. + */ + + return haReadyToken; + + } + + /** * Install identical root blocks on the journal. This is used for a few * different conditions in HA. * <ol> Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -83,14 +83,13 @@ import com.bigdata.quorum.QuorumClient; import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.zk.ZKQuorumImpl; -import com.bigdata.rdf.sail.webapp.client.HttpException; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.service.jini.JiniClientConfig; import com.bigdata.service.jini.RemoteDestroyAdmin; import com.bigdata.util.InnerCause; import com.bigdata.zookeeper.DumpZookeeper; import com.bigdata.zookeeper.ZooHelper; import com.bigdata.zookeeper.ZooKeeperAccessor; -import com.sun.jini.config.ConfigUtil; /** * Class layers in support to start and stop the {@link HAJournalServer} @@ -118,11 +117,6 @@ static final protected long cacheMissTimeout = 2000; /** - * The timeout used to await quorum meet or break. - */ - protected final static long awaitQuorumTimeout = 5000; - - /** * Implementation listens for the death of the child process and can be used * to decide when the child process is no longer executing. */ @@ -2048,49 +2042,34 @@ } /** - * Commits update transaction after awaiting quorum + * Commits update transaction after awaiting quorum. */ protected void simpleTransaction() throws IOException, Exception { + + // Await quorum meet. final long token = quorum.awaitQuorum(awaitQuorumTimeout, TimeUnit.MILLISECONDS); - - /* - * Now go through a commit point with a fully met quorum. The HALog - * files should be purged at that commit point. - */ - int retryCount = 5; - while (true) { - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // By awaiting for fully ready, should avoid any 404 errors for leader - // not being correctly setup - leader.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - try { - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - break; - } catch (HttpException httpe) { - log.warn("HTTP Error: " + httpe.getStatusCode()); - if (httpe.getStatusCode() == 404 && --retryCount > 0) - continue; - - throw httpe; - } - } + // Figure out which service is the leader. + final HAGlue leader = quorum.getClient().getLeader(token); + + // Wait until that service is ready to act as the leader. + assertEquals(HAStatusEnum.Leader, awaitNSSAndHAReady(leader)); + + final StringBuilder sb = new StringBuilder(); + sb.append("DROP ALL;\n"); + sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); + sb.append("INSERT DATA {\n"); + sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); + sb.append(" dc:creator \"A.N.Other\" .\n"); + sb.append("}\n"); + + final String updateStr = sb.toString(); + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -57,6 +57,8 @@ import com.bigdata.rdf.sail.webapp.NanoSparqlServer; import com.bigdata.rdf.sail.webapp.client.ConnectOptions; import com.bigdata.rdf.sail.webapp.client.DefaultClientConnectionManagerFactory; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; +import com.bigdata.rdf.sail.webapp.client.HttpException; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; import com.bigdata.util.InnerCause; import com.bigdata.util.concurrent.DaemonThreadFactory; @@ -95,6 +97,11 @@ static final protected String TGT_PATH = "benchmark/CI-HAJournal-1/"; /** + * The timeout used to await quorum meet or break. + */ + protected final static long awaitQuorumTimeout = 5000; + + /** * A service used to run tasks for some tests. */ protected ExecutorService executorService = null; @@ -227,44 +234,128 @@ } } - void doNSSStatusRequest(final HAGlue haGlue) throws Exception { + /** + * Check the NSS API by issuing a "GET .../status" request. + * <p> + * Note: The NSS server will not allow reads or writes until the + * quorum meets. + */ + protected void doNSSStatusRequest(final HAGlue haGlue) throws Exception { // Client for talking to the NSS. final HttpClient httpClient = new DefaultHttpClient(ccm); + // The NSS service URL (NOT the SPARQL end point). + final String serviceURL = getNanoSparqlServerURL(haGlue); + + final ConnectOptions opts = new ConnectOptions(serviceURL + "/status"); + + opts.method = "GET"; + + try { + final HttpResponse response; + + RemoteRepository.checkResponseCode(response = doConnect(httpClient, + opts)); + + EntityUtils.consume(response.getEntity()); + + } catch (IOException ex) { + + log.error(ex, ex); + + throw ex; + + } + + } + + /** + * The zookeeper events corresponding to quorum state changes are delivered + * in the zk event thread. In response to those events the services need to + * queue tasks to handle the events. Thus, there can be a lag between when + * we notice a quorum state change and when a service has actually responded + * to that state change. + * <p> + * This method is designed to ensure that the service has noticed some + * internal state changes which mean that it is ready to participate in a + * met quorum (as a leader or follower) and further that the service's NSS + * end point is running (the start of the NSS is also asynchronous.) + * @throws Exception + */ + protected HAStatusEnum awaitNSSAndHAReady(final HAGlue haGlue) + throws Exception { /* - * Check the NSS API. It should be up. - * - * Note: The NSS server will not allow reads or writes until the - * quorum meets. - * - * Note: The quorum will not meet for this unit test since we are - * only running one HAJournalServer here. + * Wait for the service to report that it is ready as a leader or + * follower. */ - { + haGlue.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + /* + * Wait for the NSS to report the status of the service (this verifies + * that the NSS interface is running). + */ + int retryCount = 5; + while (true) { + try { + final HAStatusEnum s = getNSSHAStatus(haGlue); + switch (s) { + case NotReady: + continue; + } + return s; + } catch (HttpException httpe) { + log.warn("HTTP Error: " + httpe.getStatusCode()); + if (httpe.getStatusCode() == 404 && --retryCount > 0) { + Thread.sleep(200/* ms */); + continue; + } + throw httpe; + } + } + } - // The NSS service URL (NOT the SPARQL end point). - final String serviceURL = getNanoSparqlServerURL(haGlue); + /** + * Issue HTTP request to a service to request its HA status. + * + * @param haGlue + * The service. + * + * @throws Exception + * @throws IOException + * This can include a 404 if the REST API is not yet up or has + * been shutdown. + */ + protected HAStatusEnum getNSSHAStatus(final HAGlue haGlue) + throws Exception, IOException { - final ConnectOptions opts = new ConnectOptions(serviceURL - + "/status"); + // Client for talking to the NSS. + final HttpClient httpClient = new DefaultHttpClient(ccm); - opts.method = "GET"; + // The NSS service URL (NOT the SPARQL end point). + final String serviceURL = getNanoSparqlServerURL(haGlue); - try { - final HttpResponse response; + final ConnectOptions opts = new ConnectOptions(serviceURL + + "/status?HA"); - RemoteRepository.checkResponseCode(response = doConnect( - httpClient, opts)); - - EntityUtils.consume(response.getEntity()); - - } catch (IOException ex) { - - log.error(ex, ex); - - } + opts.method = "GET"; + try { + + final HttpResponse response; + + RemoteRepository.checkResponseCode(response = doConnect(httpClient, + opts)); + + final String s = EntityUtils.toString(response.getEntity()); + + return HAStatusEnum.valueOf(s); + + } catch (IOException ex) { + + log.error(ex, ex); + + throw ex; + } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -35,6 +35,7 @@ import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.journal.IRootBlockView; import com.bigdata.quorum.Quorum; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** @@ -78,11 +79,26 @@ final HAGlue serverA = startA(); final HAGlue serverB = startB(); - awaitMetQuorum(); + final long token = awaitMetQuorum(); + // Service is met in role around a quorum. + assertEquals(token, + serverA.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS)); + + // Service is met in role around a quorum. + assertEquals(token, + serverB.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS)); + + // Verify can access the REST API "status" page. doNSSStatusRequest(serverA); doNSSStatusRequest(serverB); + // Verify that service self-reports role via the REST API. + assertEquals(HAStatusEnum.Leader, getNSSHAStatus(serverA)); + + // Verify that service self-reports role via the REST API. + assertEquals(HAStatusEnum.Follower, getNSSHAStatus(serverB)); + // Wait until KB exists. awaitKBExists(serverA); @@ -143,6 +159,12 @@ // Wait until KB exists. awaitKBExists(serverA); + // Await [A] up and running as leader. + assertEquals(HAStatusEnum.Leader, awaitNSSAndHAReady(serverA)); + + // Await [B] up and running as follower. + assertEquals(HAStatusEnum.Follower, awaitNSSAndHAReady(serverB)); + // Verify binary equality on the journal files. assertDigestsEquals(new HAGlue[] { serverA, serverB }); @@ -235,6 +257,12 @@ // Wait until KB exists. awaitKBExists(serverA); + // Await [A] up and running as leader. + assertEquals(HAStatusEnum.Leader, awaitNSSAndHAReady(serverA)); + + // Await [B] up and running as follower. + assertEquals(HAStatusEnum.Follower, awaitNSSAndHAReady(serverB)); + // Verify binary equality on the journal files. assertDigestsEquals(new HAGlue[] { serverA, serverB }); Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -548,6 +548,8 @@ * The service. * * @throws Exception + * + * TODO Add percentLogSize query parameter option. */ private void doSnapshotRequest(final HAGlue haGlue) throws Exception { @@ -574,7 +576,9 @@ } catch (IOException ex) { log.error(ex, ex); - + + throw ex; + } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -34,6 +34,8 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.msg.HADigestRequest; +import com.bigdata.quorum.Quorum; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; /** * Life cycle and related tests for a single remote {@link HAJournalServer} out @@ -69,9 +71,20 @@ // ignore. } - + + // Service is not met in any role around a quorum. + try { + serverA.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException ex) { + // Ignore expected exception. + } + + // Verify can access the REST API "status" page. doNSSStatusRequest(serverA); + // Verify that service self-reports as NotReady via the REST API. + assertEquals(HAStatusEnum.NotReady, getNSSHAStatus(serverA)); + assertTrue(getHAJournalFileA().exists()); assertTrue(getHALogDirA().exists()); assertTrue(getSnapshotDirA().exists()); @@ -284,120 +297,4 @@ } -// /** -// * Create and destroy an {@link HAJournalServer}. -// * -// * @throws Exception -// */ -// public void test_createDestroyOneServer() throws Exception { -// -// HAJournalServer serverA = null; -// -// try { -// -// /* -// * Start the HAJournalServer. -// * -// * Note: Make sure that we do not do this with a server instance that -// * could have existing data (or simply that pre-exists)? -// * -// * Note: if we run these in the same JVM then they MUST have -// * distinct zookeeper sessions! -// */ -// -// final HAJournalServer tmp = serverA = new HAJournalServer( -// new String[] { SRC_PATH + "HAJournal-A.config" }, -// new FakeLifeCycle()); -// -// final FutureTask<Void> ft1 = new FutureTask<Void>(new Callable<Void>() { -// -// public Void call() throws Exception { -// -// try { -// -// // Start server. -// tmp.run(); -// -// // Server is down. -// return null; -// -// } catch (Throwable t) { -// -// log.error(t, t); -// -// throw new RuntimeException(t); -// -// } -// } -// }); -// -// // Run task -// executorService.execute(ft1); -// -// /* -// * Wait for the service to start. -// */ -// assertCondition(new Runnable() { -// public void run() { -// switch (tmp.getRunState()) { -// case Start: -// fail(); // wait until the service starts. -// } -// } -// }); -// -// // Should be running. -// assertEquals(RunState.Running, tmp.getRunState()); -// -// // The Remote interface for (A). -// final HAGlue remoteA = (HAGlue) serverA.getProxy(); -// -// doNSSStatusRequest(remoteA); -// -// // Destroy the service using its remote interface. -// ((RemoteDestroyAdmin)remoteA).destroy(); -// -// /* -// * Wait until the server acknowledges that it is shutting down. -// */ -// assertCondition(new Runnable() { -// public void run() { -// switch (tmp.getRunState()) { -// case Shutdown: -// case ShuttingDown: -// return; -// } -// fail(); -// } -// }); -// -// // Wait for future/error. -// ft1.get(5000, TimeUnit.MILLISECONDS); -// -//// // Wait until shutdown. -//// assertCondition(new Runnable() { -//// public void run() { -//// switch (tmp.getRunState()) { -//// case Shutdown: -//// return; -//// } -//// fail(); -//// } -//// }); -// -// // Should be stopped. -// assertEquals(RunState.Shutdown, tmp.getRunState()); -// -// } finally { -// -// if (serverA != null) { -// -// serverA.shutdownNow(true/*destroy*/); -// -// } -// -// } -// -// } - } Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-20 12:02:47 UTC (rev 7019) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -27,7 +27,6 @@ import java.io.IOException; import java.io.PrintWriter; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import javax.servlet.http.HttpServletRequest; @@ -45,6 +44,7 @@ import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; import com.bigdata.quorum.zk.ZKQuorumImpl; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.zookeeper.DumpZookeeper; /** @@ -123,6 +123,9 @@ p.text("logicalServiceId=" + quorumService.getLogicalServiceId()) .node("br").close(); + // Note: This is the *local* value of getHAStatus(). + p.text("HAStatus: " + getHAStatus(journal)).node("br").close(); + /* * Report on the Service. */ @@ -141,7 +144,7 @@ { final File file = journal.getFile(); if (file != null) { - p.text("journal: file=" + file + ", nbytes=" + p.text("HAJournal: file=" + file + ", nbytes=" + journal.size()).node("br").close(); } } @@ -385,6 +388,65 @@ } /** + * Return the {@link HAStatusEnum} for <em>this</em> {@link HAJournal}. + * + * @param journal + * The {@link HAJournal}. + * + * @return The {@link HAStatusEnum}. + */ + public HAStatusEnum getHAStatus(final HAJournal journal) { + + final ZKQuorumImpl<HAGlue, QuorumService<HAGlue>> quorum = (ZKQuorumImpl<HAGlue, QuorumService<HAGlue>>) journal + .getQuorum(); + + final QuorumService<HAGlue> quorumService = quorum.getClient(); + + // check, but do not wait. + final long haReadyToken = journal.getHAReady(); + + final HAStatusEnum status; + + if (haReadyToken == Quorum.NO_QUORUM) { + + // Quorum is not met (as percieved by the HAJournal). + status = HAStatusEnum.NotReady; + + } else { + + if (quorumService.isLeader(haReadyToken)) { + + // Service is leader. + status = HAStatusEnum.Leader; + + } else if (quorumService.isFollower(haReadyToken)) { + + // Service is follower. + status = HAStatusEnum.Follower; + + } else { + + /* + * awaitHAReady() should only return successfully (and hence + * haReadyToken should only be a valid token) if the service was + * elected as either a leader or a follower. However, it is + * possible for the service to have concurrently left the met + * quorum, in which case the if/then/else pattern will fall + * through to this code path. + */ + + // Quorum is not met (as percieved by the HAJournal). + status = HAStatusEnum.NotReady; + + } + + } + + return status; + + } + + /** * Special reporting request for HA status. * * @param req @@ -402,49 +464,13 @@ final HAJournal journal = (HAJournal) indexManager; - final ZKQuorumImpl<HAGlue, QuorumService<HAGlue>> quorum = (ZKQuorumImpl<HAGlue, QuorumService<HAGlue>>) journal - .getQuorum(); - -// // The current token. -// final long quorumToken = quorum.token(); -// -// // The last valid token. -// final long lastValidToken = quorum.lastValidToken(); -// -// final int njoined = quorum.getJoined().length; - - final QuorumService<HAGlue> quorumService = quorum.getClient(); - - // check, but do not wait. - long haReadyToken = Quorum.NO_QUORUM; - try { - haReadyToken = journal.awaitHAReady(0/* timeout */, - TimeUnit.NANOSECONDS/* units */); - } catch (TimeoutException ex) { - // ignore. - } catch (AsynchronousQuorumCloseException e) { - // ignore. - } catch (InterruptedException e) { - // propagate the interrupt. - Thread.currentThread().interrupt(); - } + final HAStatusEnum status = getHAStatus(journal); - final String content; - if (haReadyToken != Quorum.NO_QUORUM) { - if (quorumService.isLeader(haReadyToken)) { - content = "Leader"; - } else if (quorumService.isFollower(haReadyToken)) { - content = "Follower"; - } else { - content = "NotReady"; - } - } else { - content = "NotReady"; - } // TODO Alternatively "max-age=1" for max-age in seconds. resp.addHeader("Cache-Control", "no-cache"); + BigdataRDFServlet.buildResponse(resp, BigdataRDFServlet.HTTP_OK, - BigdataRDFServlet.MIME_TEXT_PLAIN, content); + BigdataRDFServlet.MIME_TEXT_PLAIN, status.name()); return; Added: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java (rev 0) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java 2013-03-20 13:48:48 UTC (rev 7020) @@ -0,0 +1,45 @@ +/** +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.rdf.sail.webapp.client; + +/** + * An enumeration of HA status values available from the REST API. The set of + * enumerated states available here is intentionally a simplification of the + * internal states of the HA service. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public enum HAStatusEnum { + /** + * Ready to handle requests as the quorum leader. + */ + Leader, + /** + * Ready to handle requests as a quorum follower. + */ + Follower, + /** + * Not ready to handle HA read/write requests. + */ + NotReady; +} This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-03-20 19:10:08
|
Revision: 7023 http://bigdata.svn.sourceforge.net/bigdata/?rev=7023&view=rev Author: thompsonbry Date: 2013-03-20 19:09:58 +0000 (Wed, 20 Mar 2013) Log Message: ----------- Changed the default #of write cache buffers for HAJournal.config from 12 to 500. This provides a substantial reduction in the IO Wait associated with the leader and allows the use with traditional disk (SATA) as well as SAS and SSD. Added a means to list out the existing snapshots and exposed that list on the "GET .../status" page. Added HAGlue method to compute the digest of a snapshot. Bug fix to computeDigest() methods. The implementations were using digest.digest() rather than digest.update(). This means that the digest was only valid for the last buffer worth of data! Bug fix to RWStore.computeDigest(). This method was computing the digest starting after the root blocks. The digest methods should compute the digest of the entire file. I have also updated the documentation to clarify this. Added ?digests option to the "GET .../status" page. This option will print the digests for the Journal, HALogs, and snapshot files. This is only a debugging tool. Using the ?digests option, I was able to determine that the RWStore and HALog digests were (still) not being computed correctly. The snapshot MD5 digests agree with those computed by the md5 utility in the OS. The RWStore and WORMStrategy compute digest methods were broken. They were failing to update the offset after each block read. These methods were also modified to use the digest.update(ByteBuffer) variant for better efficiency and the temporary byte[] was removed from the code. HALog md5 was also broken with the same root cause. In addition, the sequence counter was not being udpated and the ByteBuffer needed to be flipped after the FileChannelUtility.readAll() call. I fixed both implementations. They now provide the same results as the OS md5 utility. Fixed a bug in RESTORE where an endless loop could occur if a service had an empty HALog file lying around. During RESTORE, the HALog files are applied one by one for each commit point that is +1 over the current committed data. Since the opening and closing root blocks are the same for an empty HALog file, this was failing to advance the commit point on the journal and thus attempting to re-apply the same file again and again. No change in HA test suite failures. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/althalog/HALogFile.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogReader.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java 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/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java branches/READ_CACHE/src/resources/HAJournal/HAJournal.config Added Paths: ----------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestResponse.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestResponse.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -39,6 +39,8 @@ import com.bigdata.ha.msg.IHALogDigestResponse; import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; +import com.bigdata.ha.msg.IHASnapshotDigestRequest; +import com.bigdata.ha.msg.IHASnapshotDigestResponse; import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.journal.AbstractJournal; @@ -152,8 +154,8 @@ RunState getRunState() throws IOException; /** - * Compute the digest of the backing store - <strong>THIS METHOD IS ONLY FOR - * DIAGNOSTIC PURPOSES.</strong> + * Compute the digest of the entire backing store - <strong>THIS METHOD IS + * ONLY FOR DIAGNOSTIC PURPOSES.</strong> * <p> * The digest is useless if there are concurrent writes since it can not be * meaningfully compared with the digest of another store unless both stores @@ -163,8 +165,8 @@ NoSuchAlgorithmException, DigestException; /** - * Compute the digest of a HALog file - <strong>THIS METHOD IS ONLY FOR - * DIAGNOSTIC PURPOSES.</strong> + * Compute the digest of the entire HALog file - <strong>THIS METHOD IS ONLY + * FOR DIAGNOSTIC PURPOSES.</strong> * <p> * The digest is useless if there are concurrent writes since it can not be * meaningfully compared with the digest of another store unless both stores @@ -177,6 +179,18 @@ NoSuchAlgorithmException, DigestException; /** + * Compute the digest of the entire snapshot file - <strong>THIS METHOD IS + * ONLY FOR DIAGNOSTIC PURPOSES.</strong> This digest is computed for the + * compressed data so it may be compared directly with the digest of the + * backing store from which the snapshot was obtained. + * + * @throws FileNotFoundException + * if no snapshot exists for that commit point. + */ + IHASnapshotDigestResponse computeHASnapshotDigest(IHASnapshotDigestRequest req) + throws IOException, NoSuchAlgorithmException, DigestException; + + /** * Obtain a global write lock on the leader. The lock only blocks writers. * Readers may continue to execute without delay. * <p> Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -50,6 +50,8 @@ import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotDigestRequest; +import com.bigdata.ha.msg.IHASnapshotDigestResponse; import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; @@ -239,6 +241,13 @@ } @Override + public IHASnapshotDigestResponse computeHASnapshotDigest( + final IHASnapshotDigestRequest req) throws IOException, + NoSuchAlgorithmException, DigestException { + return delegate.computeHASnapshotDigest(req); + } + + @Override public Future<Void> globalWriteLock(final IHAGlobalWriteLockRequest req) throws IOException, TimeoutException, InterruptedException { return delegate.globalWriteLock(req); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/althalog/HALogFile.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/althalog/HALogFile.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/althalog/HALogFile.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -853,8 +853,8 @@ // The backing ByteBuffer. final ByteBuffer b = buf.buffer(); - // A byte[] with the same capacity as that ByteBuffer. - final byte[] a = new byte[b.capacity()]; +// // A byte[] with the same capacity as that ByteBuffer. +// final byte[] a = new byte[b.capacity()]; // The capacity of that buffer (typically 1MB). final int bufferCapacity = b.capacity(); @@ -893,14 +893,19 @@ // read block FileChannelUtility.readAll(reopener, b, offset); - // Copy data into our byte[]. - final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); +// // Copy data into our byte[]. +// final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); // update digest - digest.digest(c, 0/* off */, nbytes/* len */); +// digest.update(c, 0/* off */, nbytes/* len */); + digest.update(b); + offset += nbytes; + remaining -= nbytes; + sequence++; + } if (log.isInfoEnabled()) Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogReader.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogReader.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogReader.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -513,8 +513,8 @@ // The backing ByteBuffer. final ByteBuffer b = buf.buffer(); - // A byte[] with the same capacity as that ByteBuffer. - final byte[] a = new byte[b.capacity()]; +// // A byte[] with the same capacity as that ByteBuffer. +// final byte[] a = new byte[b.capacity()]; // The capacity of that buffer (typically 1MB). final int bufferCapacity = b.capacity(); @@ -553,14 +553,20 @@ // read block FileChannelUtility.readAll(reopener, b, offset); - // Copy data into our byte[]. - final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); +// // Copy data into our byte[]. +// final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); // update digest - digest.digest(c, 0/* off */, nbytes/* len */); - +// digest.update(c, 0/* off */, nbytes/* len */); + b.flip(); + digest.update(b); + + offset += nbytes; + remaining -= nbytes; - + + sequence++; + } if (log.isInfoEnabled()) Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestRequest.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -0,0 +1,47 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import java.io.Serializable; + +public class HASnapshotDigestRequest implements IHASnapshotDigestRequest, Serializable { + + private static final long serialVersionUID = 1L; + + private final long commitCounter; + + public HASnapshotDigestRequest(final long commitCounter) { + + this.commitCounter = commitCounter; + + } + + @Override + public long getCommitCounter() { + + return commitCounter; + + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HASnapshotDigestResponse.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -0,0 +1,57 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import java.io.Serializable; + +public class HASnapshotDigestResponse implements IHASnapshotDigestResponse, Serializable { + + private static final long serialVersionUID = 1L; + + private final long commitCounter; + private final byte[] digest; + + public HASnapshotDigestResponse(final long commitCounter, final byte[] digest) { + + this.commitCounter = commitCounter; + + this.digest = digest; + + } + + @Override + public long getCommitCounter() { + + return commitCounter; + + } + + @Override + public byte[] getDigest() { + + return digest; + + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestRequest.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -0,0 +1,40 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + + +/** + * Message used to request the digest of the snapshot file associated with + * a specified commit point. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHASnapshotDigestRequest extends IHAMessage { + + /** + * The commit counter snapshot. + */ + long getCommitCounter(); + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHASnapshotDigestResponse.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -0,0 +1,44 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +/** + * Message used to communicate the digest of an snapshot file associated with a + * specific commit point. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHASnapshotDigestResponse extends IHAMessage { + + /** + * The commit counter for snapshot + */ + long getCommitCounter(); + + /** + * The computed disgest. + */ + byte[] getDigest(); + +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -118,6 +118,8 @@ import com.bigdata.ha.msg.IHARootBlockRequest; import com.bigdata.ha.msg.IHARootBlockResponse; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotDigestRequest; +import com.bigdata.ha.msg.IHASnapshotDigestResponse; import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; @@ -5523,6 +5525,15 @@ } @Override + public IHASnapshotDigestResponse computeHASnapshotDigest( + final IHASnapshotDigestRequest req) throws IOException, + NoSuchAlgorithmException, DigestException { + + throw new UnsupportedOperationException(); + + } + + @Override public Future<Void> globalWriteLock(final IHAGlobalWriteLockRequest req) throws IOException, TimeoutException, InterruptedException { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IHABufferStrategy.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -179,7 +179,8 @@ Object snapshotAllocators(); /** - * Compute the digest. + * Compute the digest of the entire backing store (including the magic, file + * version, root blocks, etc). * <p> * Note: The digest is not reliable unless you either use a snapshot or * suspend writes (on the quorum) while it is computed. Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -958,12 +958,36 @@ } private static final String toString(final DateFormat df, final long t) { - + return Long.toString(t) + (t != 0L ? " [" + df.format(new Date(t)) + "]" : ""); - + } - + + private static DateFormat getDateFormat() { + + final DateFormat df = DateFormat.getDateTimeInstance( + DateFormat.FULL/* dateStyle */, DateFormat.FULL/* timeStyle */); + + return df; + + } + + /** + * Format a commit time as the raw milliseconds since the epoch value plus a + * fully expressed date and time. + * + * @param t + * The commit time. + * + * @return The date and time strong. + */ + public static String toString(final long t) { + + return toString(getDateFormat(), t); + + } + public long getMetaBitsAddr() { if (getVersion() < VERSION1) { // Always WORM store before VERSION1 Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/WORMStrategy.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -2740,8 +2740,8 @@ // The backing ByteBuffer. final ByteBuffer b = buf.buffer(); - // A byte[] with the same capacity as that ByteBuffer. - final byte[] a = new byte[b.capacity()]; +// // A byte[] with the same capacity as that ByteBuffer. +// final byte[] a = new byte[b.capacity()]; // The capacity of that buffer (typically 1MB). final int bufferCapacity = b.capacity(); @@ -2780,14 +2780,17 @@ // read block readRaw(/*nbytes,*/ offset, b); - // Copy data into our byte[]. - final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); +// // Copy data into our byte[]. +// final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); // update digest - digest.digest(c, 0/* off */, nbytes/* len */); +// digest.update(c, 0/* off */, nbytes/* len */); + digest.update(b); remaining -= nbytes; - + + offset += nbytes; + sequence++; } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -6045,8 +6045,8 @@ // The backing ByteBuffer. final ByteBuffer b = buf.buffer(); - // A byte[] with the same capacity as that ByteBuffer. - final byte[] a = new byte[b.capacity()]; +// // A byte[] with the same capacity as that ByteBuffer. +// final byte[] a = new byte[b.capacity()]; // The capacity of that buffer (typically 1MB). final int bufferCapacity = b.capacity(); @@ -6061,7 +6061,7 @@ long remaining = totalBytes; // The offset of the current block. - long offset = FileMetadata.headerSize0; // 0L; + long offset = 0L; // The block sequence. long sequence = 0L; @@ -6085,14 +6085,17 @@ // read block readRaw(/*nbytes,*/ offset, b); - // Copy data into our byte[]. - final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); +// // Copy data into our byte[]. +// final byte[] c = BytesUtil.toArray(b, false/* forceCopy */, a); // update digest - digest.digest(c, 0/* off */, nbytes/* len */); + //digest.update(c, 0/* off */, nbytes/* len */); + digest.update(b); remaining -= nbytes; + offset += nbytes; + sequence++; } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -301,6 +301,7 @@ final long snapshotCommitCounter; synchronized (this) { + @SuppressWarnings("unchecked") final ITupleIterator<IRootBlockView> itr = rangeIterator(null/* fromKey */, null/* toKey */, 1/* capacity */, IRangeQuery.DEFAULT Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -75,8 +75,7 @@ final long then = now - millis; - final IRootBlockView rootBlock = jnl.getSnapshotManager() - .getSnapshotIndex().find(then); + final IRootBlockView rootBlock = jnl.getSnapshotManager().find(then); if (rootBlock == null) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -69,6 +69,7 @@ import com.bigdata.ha.msg.HALogDigestResponse; import com.bigdata.ha.msg.HALogRootBlocksResponse; import com.bigdata.ha.msg.HASendStoreResponse; +import com.bigdata.ha.msg.HASnapshotDigestResponse; import com.bigdata.ha.msg.IHADigestRequest; import com.bigdata.ha.msg.IHADigestResponse; import com.bigdata.ha.msg.IHAGlobalWriteLockRequest; @@ -79,6 +80,8 @@ import com.bigdata.ha.msg.IHALogRootBlocksResponse; import com.bigdata.ha.msg.IHARebuildRequest; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotDigestRequest; +import com.bigdata.ha.msg.IHASnapshotDigestResponse; import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; @@ -992,6 +995,32 @@ } + @Override + public IHASnapshotDigestResponse computeHASnapshotDigest( + final IHASnapshotDigestRequest req) throws IOException, + NoSuchAlgorithmException, DigestException { + + if (haLog.isDebugEnabled()) + haLog.debug("req=" + req); + + // The commit counter of the desired closing root block. + final long commitCounter = req.getCommitCounter(); + + final MessageDigest digest = MessageDigest.getInstance("MD5"); + + /* + * Compute digest for snapshot for that commit point. + * + * Note: throws FileNotFoundException if no snapshot for that commit + * point. + */ + getSnapshotManager().getDigest(commitCounter, digest); + + return new HASnapshotDigestResponse(req.getCommitCounter(), + digest.digest()); + + } + /** * {@inheritDoc} * Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -1507,7 +1507,7 @@ */ while (true) { - long commitCounter = journal.getRootBlockView() + final long commitCounter = journal.getRootBlockView() .getCommitCounter(); try { @@ -1515,6 +1515,28 @@ final IHALogReader r = journal.getHALogWriter() .getReader(commitCounter + 1); + if (r.isEmpty()) { + + /* + * There is an empty HALog file. We can not apply it + * since it has no data. This ends our restore + * procedure. + */ + + break; + + } + + if (r.getOpeningRootBlock().getCommitCounter() != commitCounter) { + // Sanity check + throw new AssertionError(); + } + + if (r.getClosingRootBlock().getCommitCounter() != commitCounter + 1) { + // Sanity check + throw new AssertionError(); + } + applyHALog(r); doLocalCommit(r.getClosingRootBlock()); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -50,8 +50,6 @@ * it forward to a specific commit point. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * - * FIXME HARestore : write test suite. */ public class HARestore { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -27,11 +27,17 @@ import java.io.DataOutputStream; import java.io.File; import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.FilenameFilter; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; +import java.security.DigestException; +import java.security.MessageDigest; import java.util.Formatter; +import java.util.LinkedList; +import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -47,6 +53,8 @@ import org.apache.log4j.Logger; import com.bigdata.btree.BytesUtil; +import com.bigdata.btree.ITuple; +import com.bigdata.btree.ITupleIterator; import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.halog.IHALogReader; import com.bigdata.ha.msg.HASnapshotResponse; @@ -59,6 +67,7 @@ import com.bigdata.journal.RootBlockUtility; import com.bigdata.journal.RootBlockView; import com.bigdata.quorum.QuorumException; +import com.bigdata.rawstore.Bytes; import com.bigdata.util.ChecksumUtility; /** @@ -78,7 +87,7 @@ /** * The file extension for journal snapshots. */ - public final static String SNAPSHOT_EXT = ".jnl.zip"; + public final static String SNAPSHOT_EXT = ".jnl.gz"; /** * The prefix for the temporary files used to generate snapshots. @@ -112,6 +121,12 @@ * populated when the {@link HAJournal} starts from the file system and * maintained as snapshots are taken or destroyed. All operations on this * index MUST be synchronized on its object monitor. + * <p> + * Note: This index is not strictly necessary. We can also visit the files + * in the file system. However, the index makes it much faster to locate a + * specific snapshot based on a commit time and provides low latency access + * to the {@link IRootBlockView} for that snapshot (faster than opening the + * snapshot file on the disk). */ private final CommitTimeIndex snapshotIndex; @@ -155,18 +170,18 @@ } - /** - * An in memory index over the last commit time of each snapshot. This is - * populated when the {@link HAJournal} starts from the file system and - * maintained as snapshots are taken or destroyed. All operations on this - * index MUST be synchronized on its object monitor. - */ - CommitTimeIndex getSnapshotIndex() { +// /** +// * An in memory index over the last commit time of each snapshot. This is +// * populated when the {@link HAJournal} starts from the file system and +// * maintained as snapshots are taken or destroyed. All operations on this +// * index MUST be synchronized on its object monitor. +// */ +// private CommitTimeIndex getSnapshotIndex() { +// +// return snapshotIndex; +// +// } - return snapshotIndex; - - } - public SnapshotManager(final HAJournalServer server, final HAJournal journal, final Configuration config) throws IOException, ConfigurationException { @@ -444,6 +459,69 @@ } /** + * Find the commit counter for the most recent snapshot (if any). + * + * @return That commit counter -or- ZERO (0L) if there are no snapshots. + */ + public long getMostRecentSnapshotCommitCounter() { + + return snapshotIndex.getMostRecentSnapshotCommitCounter(); + + } + + /** + * Return the {@link IRootBlock} identifying the snapshot having the largest + * commitTime that is less than or equal to the given value. + * + * @param timestamp + * The given timestamp. + * + * @return The {@link IRootBlockView} of the identified snapshot -or- + * <code>null</code> iff there are no snapshots in the index that + * satisify the probe. + * + * @throws IllegalArgumentException + * if <i>timestamp</i> is less than or equals to ZERO (0L). + */ + public IRootBlockView find(final long timestamp) { + + return snapshotIndex.find(timestamp); + + } + + /** + * Return a list of all known snapshots. The list consists of the + * {@link IRootBlockView} for each snapshot. The list will be in order of + * increasing <code>commitTime</code>. This should also correspond to + * increasing <code>commitCounter</code>. + * + * @return A list of the {@link IRootBlockView} for the known snapshots. + */ + public List<IRootBlockView> getSnapshots() { + + final List<IRootBlockView> l = new LinkedList<IRootBlockView>(); + + synchronized (snapshotIndex) { + + @SuppressWarnings("unchecked") + final ITupleIterator<IRootBlockView> itr = snapshotIndex.rangeIterator(); + + while(itr.hasNext()) { + + final ITuple<IRootBlockView> t = itr.next(); + + final IRootBlockView rootBlock = t.getObject(); + + l.add(rootBlock); + + } + + } + + return l; + } + + /** * Return the {@link Future} of the current snapshot operation (if any). * * @return The {@link Future} of the current snapshot operation -or- @@ -950,4 +1028,73 @@ } // class SendStoreTask + + /** + * Compute the digest of a snapshot file. + * <p> + * Note: The digest is only computed for the data beyond the file header. + * This is for consistency with + * {@link IHABufferStrategy#computeDigest(Object, MessageDigest)} + * + * @param commitCounter + * @param digest + * @throws IOException + * @throws FileNotFoundException + * @throws DigestException + * + * TODO We should pin the snapshot if we are reading it to + * compute its digest. + */ + public void getDigest(final long commitCounter, final MessageDigest digest) + throws FileNotFoundException, IOException, DigestException { + + final File file = getSnapshotFile(commitCounter); + + // Note: Throws FileNotFoundException. + final GZIPInputStream is = new GZIPInputStream( + new FileInputStream(file)); + + try { + + if (log.isInfoEnabled()) + log.info("Computing digest: " + file); + + computeDigest(is, digest); + + } finally { + + is.close(); + + } + + } + + private static void computeDigest(final InputStream is, + final MessageDigest digest) throws DigestException, IOException { + + // The capacity of that buffer. + final int bufferCapacity = Bytes.kilobyte32 * 4; + + // A byte[] with the same capacity as that ByteBuffer. + final byte[] a = new byte[bufferCapacity]; + + while (true) { + + // Read as much as we can. + final int nread = is.read(a, 0/* off */, a.length); + + if (nread == -1) { + + // End of stream. + return; + + } + + // update digest + digest.update(a, 0/* off */, nread/* len */); + + } + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -29,8 +29,10 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.math.BigInteger; import java.security.DigestException; import java.security.NoSuchAlgorithmException; +import java.util.Arrays; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -52,6 +54,8 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.msg.HADigestRequest; import com.bigdata.ha.msg.HALogDigestRequest; +import com.bigdata.ha.msg.HARootBlockRequest; +import com.bigdata.ha.msg.HASnapshotDigestRequest; import com.bigdata.io.TestCase3; import com.bigdata.rdf.sail.TestConcurrentKBCreate; import com.bigdata.rdf.sail.webapp.NanoSparqlServer; @@ -670,6 +674,69 @@ } /** + * Verify the the digest of the journal is equal to the digest of the + * indicated snapshot on the specified service. + * <p> + * Note: This can only succeed if the journal is at the specififed commit + * point. If there are concurrent writes on the journal, then it's digest + * will no longer be consistent with the snapshot. + * + * @param service + * The service. + * @param commitCounter + * The commit counter for the snapshot. + * + * @throws NoSuchAlgorithmException + * @throws DigestException + * @throws IOException + */ + protected void assertSnapshotDigestEquals(final HAGlue service, + final long commitCounter) throws NoSuchAlgorithmException, + DigestException, IOException { + + final long commitCounterBefore = service + .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) + .getRootBlock().getCommitCounter(); + + // Verify the journal is at the desired commit point. + assertEquals(commitCounter, commitCounterBefore); + + final byte[] journalDigest = service.computeDigest( + new HADigestRequest(null/* storeUUID */)).getDigest(); + + final long commitCounterAfter = service + .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) + .getRootBlock().getCommitCounter(); + + // Verify the journal is still at the desired commit point. + assertEquals(commitCounter, commitCounterAfter); + + final byte[] snapshotDigest = service.computeHASnapshotDigest( + new HASnapshotDigestRequest(commitCounter)).getDigest(); + + if (!BytesUtil.bytesEqual(journalDigest, snapshotDigest)) { + + /* + * Note: Provides base 16 rendering as per normal md5 runs. + */ + + final String journalStr = new BigInteger(1, journalDigest) + .toString(16); + + final String snapshotStr = new BigInteger(1, snapshotDigest) + .toString(16); + + fail("journal=" + journalStr + ", snapshot=" + snapshotStr); + +// fail("journal=" + Arrays.toString(journalDigest) + ", snapshot=" +// + Arrays.toString(snapshotDigest) + " for commitCounter=" +// + commitCounter + " on service=" + service); + + } + + } + + /** * Return the name of the foaf data set. * * @param string Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -32,10 +32,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.zip.GZIPInputStream; -import net.jini.config.Configuration; - import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.impl.client.DefaultHttpClient; @@ -49,22 +46,14 @@ import com.bigdata.journal.IRootBlockView; import com.bigdata.quorum.Quorum; import com.bigdata.rdf.sail.webapp.client.ConnectOptions; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** * Test suites for an {@link HAJournalServer} quorum with a replication factor * of THREE (3) and a fully met {@link Quorum}. * - * TODO Verify that the snapshot is consistent in the data with the Journal. - * This could be done using a digest comparison of the journal and a digest - * computed from a {@link GZIPInputStream} reading from the snapshot. [The same - * approach could also work for snapshots under sustained writes if we modify - * the journal digest logic to only pay attention to the committed allocations.] - * - * TODO Verify snapshot is refused for service that is not joined with the met - * quorum. - * - * TODO HARestore test suite: Verify that the snapshot may be unziped and halogs + * FIXME HARestore test suite: Verify that the snapshot may be unziped and halogs * applied by the {@link HARestore} utility in order to obtain a journal * corresponding to a specific commit point. * <p> @@ -74,7 +63,7 @@ * TODO Verify will not take snapshot if size on disk of HALog files since the * last snapshot is LT some percentage. * - * TODO Verify release of old snapshot(s) and HALog(s) when a new snapshot is + * FIXME Verify release of old snapshot(s) and HALog(s) when a new snapshot is * taken in accordence with the {@link IRestorePolicy}. * <p> * Make sure that we never release the most current snapshot or HALogs required @@ -97,26 +86,57 @@ super(name); } +// /** +// * {@inheritDoc} +// * <p> +// * Note: This overrides some {@link Configuration} values for the +// * {@link HAJournalServer} in order to establish conditions suitable for +// * testing the {@link ISnapshotPolicy} and {@link IRestorePolicy}. +// */ +// @Override +// protected String[] getOverrides() { +// +// return new String[]{ +// "com.bigdata.journal.jini.ha.HAJournalServer.snapshotPolicy=new com.bigdata.journal.jini.ha.DefaultSnapshotPolicy()" +// }; +// +// } + /** - * {@inheritDoc} - * <p> - * Note: This overrides some {@link Configuration} values for the - * {@link HAJournalServer} in order to establish conditions suitable for - * testing the {@link ISnapshotPolicy} and {@link IRestorePolicy}. + * Start A. Verify that we can not take a snapshot since it is not joined + * with the met quorum. */ - @Override - protected String[] getOverrides() { + public void testA_snapshot_refused_since_not_met() throws Exception { + + // Start A. + final HAGlue serverA = startA(); + + // Verify the REST API is up and service is not ready. + // TODO Might have to retry this if 404 observed. + assertEquals(HAStatusEnum.NotReady, getNSSHAStatus(serverA)); - return new String[]{ - "com.bigdata.journal.jini.ha.HAJournalServer.snapshotPolicy=new com.bigdata.journal.jini.ha.DefaultSnapshotPolicy()" - }; + // Request a snapshot. + final Future<IHASnapshotResponse> ft = serverA + .takeSnapshot(new HASnapshotRequest(0/* percentLogSize */)); + + if(ft == null) { + // Ok. No snapshot will be taken. + return; + + } + + ft.cancel(true/* mayInterruptIfRunning */); + + fail("Not expecting a future since service is not joined with a met quorum."); + } /** * Start two services. The quorum meets. Take a snapshot. Verify that the * snapshot appears within a resonable period of time and that it is for - * <code>commitCounter:=1</code> (just the KB create). + * <code>commitCounter:=1</code> (just the KB create). Verify that the + * digest of the snapshot agrees with the digest of the journal. */ public void testAB_snapshot() throws Exception { @@ -132,7 +152,7 @@ awaitKBExists(serverA); final HAGlue leader = quorum.getClient().getLeader(token); - + assertEquals(serverA, leader); // A is the leader. { // Verify quorum is still valid. @@ -161,13 +181,18 @@ final IRootBlockView snapshotRB = ft.get().getRootBlock(); + final long commitCounter = 1L; + // Verify snapshot is for the expected commit point. - assertEquals(1L, snapshotRB.getCommitCounter()); + assertEquals(commitCounter, snapshotRB.getCommitCounter()); // Snapshot directory contains the desired filename. assertEquals(new String[] { "00000000000000000001" + SnapshotManager.SNAPSHOT_EXT }, getSnapshotDirA().list()); + // Verify digest of snapshot agrees with digest of journal. + assertSnapshotDigestEquals(leader, commitCounter); + } } @@ -176,7 +201,8 @@ * Start two services. The quorum meets. Take a snapshot using B (NOT the * leader). Verify that the snapshot appears within a resonable period of * time and that it is for <code>commitCounter:=1</code> (just the KB - * create). + * create). Verify that the digest of the snapshot agrees with the digest of + * the journal. */ public void testAB_snapshotB() throws Exception { @@ -225,8 +251,10 @@ final IRootBlockView snapshotRB = ft.get().getRootBlock(); + final long commitCounter = 1L; + // Verify snapshot is for the expected commit point. - assertEquals(1L, snapshotRB.getCommitCounter()); + assertEquals(commitCounter, snapshotRB.getCommitCounter()); // Snapshot directory remains empty on A. assertEquals(0, getSnapshotDirA().list().length); @@ -235,6 +263,9 @@ assertEquals(new String[] { "00000000000000000001" + SnapshotManager.SNAPSHOT_EXT }, getSnapshotDirB().list()); + // Verify digest of snapshot agrees with digest of journal. + assertSnapshotDigestEquals(serverB, commitCounter); + } } @@ -289,13 +320,18 @@ final IRootBlockView snapshotRB = ft.get().getRootBlock(); + final long commitCounter = 1L; + // Verify snapshot is for the expected commit point. - assertEquals(1L, snapshotRB.getCommitCounter()); + assertEquals(commitCounter, snapshotRB.getCommitCounter()); // Snapshot directory contains the desired filename. assertEquals(new String[] { "00000000000000000001" + SnapshotManager.SNAPSHOT_EXT }, getSnapshotDirA().list()); + // Verify digest of snapshot agrees with digest of journal. + assertSnapshotDigestEquals(leader, commitCounter); + } /* @@ -367,7 +403,10 @@ doSnapshotRequest(leader); - // Get the Future. Should still be there, but if not then will be null. + /* + * Get the Future. Should still be there, but if not then will be + * null (it which case the snapshot is already done). + */ final Future<IHASnapshotResponse> ft = leader .takeSnapshot(new HASnapshotRequest(1000/* percentLogSize */)); @@ -395,6 +434,8 @@ assertEquals(new String[] { "00000000000000000001" + SnapshotManager.SNAPSHOT_EXT }, getSnapshotDirA().list()); + assertSnapshotDigestEquals(leader, 1L/* commitCounter */); + } } @@ -528,8 +569,10 @@ final IRootBlockView snapshotRB = ft.get().getRootBlock(); + final long commitCounter = 2L; + // Verify snapshot is for the expected commit point. - assertEquals(2L, snapshotRB.getCommitCounter()); + assertEquals(commitCounter, snapshotRB.getCommitCounter()); // Snapshot directory contains the desired filename. assertEquals( @@ -537,6 +580,9 @@ getSnapshotDirA(), 2L).getName() }, getSnapshotDirA().list()); + // Verify digest of snapshot agrees with digest of journal. + assertSnapshotDigestEquals(leader, commitCounter); + } } Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-20 19:04:55 UTC (rev 7022) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-03-20 19:09:58 UTC (rev 7023) @@ -26,6 +26,11 @@ import java.io.FilenameFilter; import java.io.IOException; import java.io.PrintWriter; +import java.math.BigInteger; +import java.security.DigestException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; import java.util.UUID; import java.util.concurrent.TimeoutException; @@ -36,9 +41,12 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumService; +import com.bigdata.ha.halog.HALogReader; import com.bigdata.ha.halog.IHALogReader; import com.bigdata.ha.msg.HASnapshotRequest; import com.bigdata.journal.IIndexManager; +import com.bigdata.journal.IRootBlockView; +import com.bigdata.journal.RootBlockView; import com.bigdata.journal.jini.ha.HAJournal; import com.bigdata.journal.jini.ha.SnapshotManager; import com.bigdata.quorum.AsynchronousQuorumCloseException; @@ -104,6 +112,8 @@ final QuorumService<HAGlue> quorumService = quorum.getClient(); + final boolean digests = req.getParameter(StatusServlet.DIGESTS) != null; + current.node("h1", "High Availability"); // The quorum state. @@ -144,8 +154,25 @@ { final File file = journal.getFile(); if (file != null) { + String digestStr = null; + if (digests) { + try { + final MessageDigest digest = MessageDigest + .getInstance("MD5"); + journal.getBufferStrategy().computeDigest( + null/* snapshot */, digest); + digestStr = new BigInteger(1, digest.digest()) + .toString(16); + } catch (NoSuchAlgorithmException ex) { + // ignore + } catch (DigestException ex) { + // ignore + } + } p.text("HAJournal: file=" + file + ", nbytes=" - + journal.size()).node("br").close(); + + journal.size() + + (digestStr == null ? "" : ", md5=" + digestStr)) + .node("br").close(); } } @@ -168,16 +195,54 @@ nbytes += file.length(); nfiles++; } - p.text("HALogDir: nfiles=" + nfiles + ", nbytes=" - + nbytes + ", path=" + haLogDir).node("br") - .close(); + p.text("HALogDir: nfiles=" + nfiles + ", nbytes=" + nbytes + + ", path=" + haLogDir).node("br").close(); + if (digests) { + /* + * List each HALog file together with its digest. + * + * FIXME We need to request the live log differently and use + * the lock for it. That makes printing the HALog digests + * here potentially probemantic if there are outstanding + * writes. + */ + for (File file : a) { + String digestStr = null; + final IHALogReader r = new HALogReader(file); + try { + if (digests && !r.isEmpty()) { + try { + final MessageDigest digest = MessageDigest + .getInstance("MD5"); + r.computeDigest(digest); + digestStr = new BigInteger(1, + digest.digest()).toString(16); + } catch (NoSuchAlgorithmException ex) { + // ignore + } catch (DigestException ex) { + // ignore + } + } + } finally { + r.close(); + } + p.text("HALogFile: closingCommitCounter=" + + r.getClosingRootBlock().getCommitCounter() + + ", file=" + + file + + ", nbytes=" + + nbytes + + (digestStr == null ? "" : ", md5=" + + digestStr)).node("br").close(); + } + } } /* * Report #of files and bytes in the snapshot directory. */ { - final File snapshotDir = ((HAJournal) journal) + final File snapshotDir = journal .getSnapshotManager().getSnapshotDir(); final File[] a = snapshotDir.listFiles(new FilenameFilter() { @Override @@ -193,6 +258,44 @@ } p.text("SnapshotDir: nfiles=" + nfiles + ", nbytes=" + nbytes + ", path=" + snapshotDir).node("br").close(); + + if (true) { + + /* + * List the available snapshots. + */ + final List<IRootBlockView> snapshots = journal + .getSnapshotManager().getSnapshots(); + + for (IRootBlockView rb : snapshots) { + + String digestStr = null; + if (digests) { + try { + final MessageDigest digest = MessageDigest + .getInstance("MD5"); + journal.getSnapshotManager().getDigest( + rb.getCommitCounter(), digest); + digestStr = new BigInteger(1, digest.digest()) + .toString(16); + } catch (NoSuchAlgorithmException ex) { + // ignore + } catch (DigestException ex) { + // ignore + } + } + + p.text("SnapshotFile: commitTime=" + + RootBlockView.toString(rb.getLastCommitTime()) + + ", commitCounter=" + + rb.getCommitCounter() + + (digestStr == null ? "" : ", md5=" + + digestStr)).node("br").close(); + + } + + } + } /* Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/Stat... [truncated message content] |
From: <tho...@us...> - 2013-03-21 17:03:40
|
Revision: 7028 http://bigdata.svn.sourceforge.net/bigdata/?rev=7028&view=rev Author: thompsonbry Date: 2013-03-21 17:03:29 +0000 (Thu, 21 Mar 2013) Log Message: ----------- - The write replication logic does not consistently impose asserts that will cause replication to fail fast if the quorum token changes from the token used by the leader when it initiated the replication of some write. QuorumPipelineImpl: assertLeader() needs to be on the then current quorum token specified by WriteTask, NOT on the historical token associated with a replayed HALog message, and it needs to be present for all messages (both live and historical). Also, quorum.assertLeader(token) and/or quorum.assertQuorum(token) must terminate the retrySend() pattern immediately. If we are no longer the leader (or if the quorum token has changed) then we SHOULD NOT attempt to resend the message. The solution for this would appear to be using the [req] field for the live writes as well as the replicated writes. We have a lot of code that tests on [req==null] which will need to be modified to test on req.isLive(). The code will also need to be audited for tests against the IHAWriteMessage's token rather than the req's token. For the moment, I have tightened up the asserts in replicate() and retrySend() to cause a faster fail if the quorum token becomes invalid. - Fixed bug where the 1st follower would attempt to log a historical write during replay of HALogs for resynchronization of the 2nd follower. This would cause an IllegalStateException thrown out of the HALogWriter. For example, the exception could indicat sequence=0 when expecting some non-zero sequence if there were already writes against the live HALog. If there were no writes, then the exception might indicate that the message was for the wrong commit point. Caused by: java.lang.IllegalStateException: nextSequence=27, but msg=com.bigdata.ha.msg.HAWriteMessage{size=1048394,chksum=2145255012,uuid=c6118aac-6613-4ef1-8db7-c6ffb2fdeb1e,commitCounter=6,commitTime=1363869801921,sequence=0,storeType=RW,quorumToken=75,fileExtent=274202624,firstOffset=210010112} - Observed problem where A+B are running, we load data. C is started and fails to met during the load. The load completes. C is still not met. On restart (of all services at any rate), C joins with the met quorum. This corresponds to TestHA3JournalServer.testStartAB_C_LiveResync(). That test is known to fail. I am now seeing it pass when run by itself. However, the problem can still be observed when doing this by hand with A+B and C started once a long LOAD is running. The conditionalJoinWithMetQuorum() fails but we never see another write cache block and thus it hangs rather than attempting to join and even if we have another commit point come through, it remains hung. The problem was a failure to check that the most recent live write message was for the a commit point GTE the current write set that we had identified on the leader. Thus, if any live writes had been received since the last abort or since the resync started, it would conditionJoinWithMetQuorum would always fail. With this change, testStartAB_C_MultiTransactionResync is now GREEN. - The liveLoadRemainsMet tests were failing to check the LOAD thread for an error. Refactored tests to use a Future for a task so we can fail the test if the LOAD fails. I did quite a bit of cleanup of TestHA3JournalServer in this refactor and created several new tests that go after different variantions on liveLoadRemainsMet. The new tests allow us to better characterize the ways in which the replication cluster is and is not robust, especially with respect to the ability of a 3rd service to resync and join with a met quorum during a live load versus to resync and join only after the live load is complete. - Refactored the multi-transaction HA3 tests into distinct parameterized test names so we can observe the different conditions under which the test can reliably pass. This information was previously hidden since many versions were run from within a single test. - Added additional run state values to the HAStatus page. You can now observe the server run state and the inner HAQuorumStatus run state, including what commit counter it is replaying during RESYNC. New new HA test failures (worksheet has been updated). There are some new tests that fail, but these are simply broken out of existing tests that were failing. Multi-transition and live load while met now pass. Live load while met does not pass if you insist that the service sync and join during the live load. This appears to be the only remaining source of test failures. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAWriteSetStateResponse.java branches/READ_CACHE/bigdata/src/java/com/bigdata/io/writecache/WriteCacheService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestRawTransfers.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -154,6 +154,11 @@ RunState getRunState() throws IOException; /** + * The extended run state of the service. + */ + String getExtendedRunState() throws IOException; + + /** * Compute the digest of the entire backing store - <strong>THIS METHOD IS * ONLY FOR DIAGNOSTIC PURPOSES.</strong> * <p> Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -223,6 +223,11 @@ } @Override + public String getExtendedRunState() throws IOException { + return delegate.getExtendedRunState(); + } + + @Override public Future<IHASendStoreResponse> sendHAStore(IHARebuildRequest msg) throws IOException { return delegate.sendHAStore(msg); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -54,6 +54,7 @@ import com.bigdata.io.DirectBufferPool; import com.bigdata.io.IBufferAccess; import com.bigdata.io.writecache.WriteCache; +import com.bigdata.quorum.Quorum; import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.QuorumMember; import com.bigdata.quorum.QuorumStateChangeListener; @@ -786,7 +787,14 @@ private final ByteBuffer b; /** - * The token for the leader. + * The token for the leader. The service that initiates the replication + * of a message MUST be the leader for this token. + * <p> + * The token is either taken from the {@link IHAWriteMessage} (if this + * is a live write) or from the current {@link Quorum#token()}. + * <p> + * Either way, we verify that this service is (and remains) the leader + * for that token throughout the {@link RobustReplicateTask}. */ private final long quorumToken; @@ -829,6 +837,9 @@ quorumToken = msg.getQuorumToken(); + // Must be the leader for that token. + member.assertLeader(quorumToken); + } else { /* @@ -838,12 +849,57 @@ // Use the current quorum token. quorumToken = member.getQuorum().token(); + // Must be the leader for that token. + member.assertLeader(quorumToken); + } } + + /** + * This collects the assertion(s) that we make for the service that is + * attempting to robustly replicate a write into a single method. This + * was done in order to concentrate any conditional logic and design + * rationale into a single method. + * + * Note: IFF we allow non-leaders to replicate HALog messages then this + * assert MUST be changed to verify that the quorum token remains valid + * and that this service remains joined with the met quorum, i.e., + * + * <pre> + * if (!quorum.isJoined(token)) + * throw new QuorumException(); + * </pre> + */ + private void assertQuorumState() throws QuorumException { + + // Must be the leader for that token. + member.assertLeader(quorumToken); + +// if (req == null) { +// +// /* +// * This service must be the leader since this is a LIVE +// * write cache message (versus a historical message that is +// * being replayed). +// * +// * Note: The [quorumToken] is from the message IFF this is a +// * live message and is otherwise the current quorum token. +// */ +// member.assertLeader(quorumToken); +// +// } + + } public Void call() throws Exception { + /* + * Note: This is tested outside of the try/catch. Do NOT retry if + * the quorum state has become invalid. + */ + assertQuorumState(); + try { innerReplicate(0/* retryCount */); @@ -897,35 +953,17 @@ + " bytes, retryCount=" + retryCount + ", req=" + req + ", msg=" + msg); - /* - * Note: disable assert if we allow non-leaders to replicate - * HALog messages (or just verify service joined with the - * quorum). - */ + // retest while holding lock before sending the message. + assertQuorumState(); - if (req == null) { - - // // Note: Do not test quorum token for historical writes. - // member.assertLeader(msg.getQuorumToken()); - - /* - * This service must be the leader. - * - * Note: The [quorumToken] is from the message IFF this is a - * live message and is otherwise the current quorum token. - */ - member.assertLeader(quorumToken); - - } - final PipelineState<S> downstream = pipelineStateRef.get(); final HASendService sendService = getHASendService(); final ByteBuffer b = this.b.duplicate(); - new SendBufferTask<S>(req, msg, b, downstream, sendService, - sendLock).call(); + new SendBufferTask<S>(member, quorumToken, req, msg, b, + downstream, sendService, sendLock).call(); return; @@ -966,6 +1004,12 @@ // Sleep before each retry (including the first). Thread.sleep(RETRY_SLEEP[tryCount-1]/* ms */); + /* + * Note: Tested OUTSIDE of the try/catch so a quorum break will + * immediately stop the retry behavior. + */ + assertQuorumState(); + try { // send to 1st follower. @@ -974,10 +1018,16 @@ // Success. return true; - } catch (Exception ex) { + } catch (Throwable t) { + if (InnerCause.isInnerCause(t, InterruptedException.class)) { + + throw (InterruptedException) t; + + } + // log and retry. - log.error("retry=" + tryCount + " : " + ex, ex); + log.error("retry=" + tryCount + " : " + t, t); continue; @@ -1015,6 +1065,8 @@ static private class SendBufferTask<S extends HAPipelineGlue> implements Callable<Void> { + private final QuorumMember<S> member; + private final long token; // member MUST remain leader for token. private final IHASyncRequest req; private final IHAWriteMessage msg; private final ByteBuffer b; @@ -1022,11 +1074,13 @@ private final HASendService sendService; private final Lock sendLock; - public SendBufferTask(final IHASyncRequest req, - final IHAWriteMessage msg, final ByteBuffer b, - final PipelineState<S> downstream, + public SendBufferTask(final QuorumMember<S> member, final long token, + final IHASyncRequest req, final IHAWriteMessage msg, + final ByteBuffer b, final PipelineState<S> downstream, final HASendService sendService, final Lock sendLock) { + this.member = member; + this.token = token; this.req = req; // Note: MAY be null. this.msg = msg; this.b = b; @@ -1044,6 +1098,7 @@ */ sendLock.lock(); + try { doRunWithLock(); @@ -1078,6 +1133,11 @@ * second. Timeouts are ignored during this loop. */ while (!futSnd.isDone() && !futRec.isDone()) { + /* + * Make sure leader's quorum token remains valid for ALL + * writes. + */ + member.assertLeader(token); try { futSnd.get(1L, TimeUnit.SECONDS); } catch (TimeoutException ignore) { @@ -1116,27 +1176,37 @@ public Future<Void> receiveAndReplicate(final IHASyncRequest req, final IHAWriteMessage msg) throws IOException { + /* + * FIXME We should probably pass the quorum token through from the + * leader for ALL replicated writes. This uses the leader's quorum token + * when it is available (for a live write) and otherwise uses the + * current quorum token (for historical writes, since we are not + * providing the leader's token in this case). + */ + final long token = req == null ? msg.getQuorumToken() : member + .getQuorum().token(); + final RunnableFuture<Void> ft; lock.lock(); try { + // Must be valid quorum. + member.getQuorum().assertQuorum(token); + if (receiveBuffer == null) { - + /* * The quorum broke and the receive buffer was cleared or - * possibly we have become a leader. - * - * TODO We should probably pass in the Quorum and then just - * assert that the msg.getQuorumToken() is valid for the quorum - * (but we can't do that for historical messages). + * possibly we have become a leader (a distinct test since + * otherwise we can hit an NPE on the receiveBuffer). */ throw new QuorumException(); + + } - } - final PipelineState<S> downstream = pipelineStateRef.get(); if (log.isTraceEnabled()) @@ -1180,8 +1250,8 @@ * not the last). */ - ft = new FutureTask<Void>(new ReceiveAndReplicateTask<S>(req, msg, - b, downstream, receiveService)); + ft = new FutureTask<Void>(new ReceiveAndReplicateTask<S>(member, + token, req, msg, b, downstream, receiveService)); } finally { @@ -1203,17 +1273,22 @@ private static class ReceiveAndReplicateTask<S extends HAPipelineGlue> implements Callable<Void> { + private final QuorumMember<S> member; + private final long token; private final IHASyncRequest req; private final IHAWriteMessage msg; private final ByteBuffer b; private final PipelineState<S> downstream; private final HAReceiveService<HAMessageWrapper> receiveService; - public ReceiveAndReplicateTask(final IHASyncRequest req, + public ReceiveAndReplicateTask(final QuorumMember<S> member, + final long token, final IHASyncRequest req, final IHAWriteMessage msg, final ByteBuffer b, final PipelineState<S> downstream, final HAReceiveService<HAMessageWrapper> receiveService) { + this.member = member; + this.token = token; this.req = req; // Note: MAY be null. this.msg = msg; this.b = b; @@ -1247,6 +1322,14 @@ * ignored during this loop. */ while (!futSnd.isDone() && !futRec.isDone()) { + /* + * The token must remain valid, even if this service is + * not joined with the met quorum. If fact, services + * MUST replicate writes regardless of whether or not + * they are joined with the met quorum, but only while + * there is a met quorum. + */ + member.getQuorum().assertQuorum(token); try { futSnd.get(1L, TimeUnit.SECONDS); } catch (TimeoutException ignore) { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -438,17 +438,17 @@ * @param data */ public void write(final IHAWriteMessage msg, final ByteBuffer data) - throws IOException { + throws IOException, IllegalStateException { final Lock lock = m_stateLock.readLock(); lock.lock(); try { assertOpen(); - /* - * Check if this really is a valid message for this file. If it is - * not, then close the file and return immediately - */ + /* + * Check if this really is a valid message for this file. If it is + * not, then throw out an exception. + */ if (m_rootBlock.getCommitCounter() != msg.getCommitCounter()) throw new IllegalStateException("commitCounter=" + m_rootBlock.getCommitCounter() + ", but msg=" + msg); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAWriteSetStateResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAWriteSetStateResponse.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAWriteSetStateResponse.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -56,4 +56,13 @@ return sequence; } + @Override + public String toString() { + + return getClass().getSimpleName() + "{commitCounter=" + commitCounter + + ", sequence=" + sequence + ", lastCommitTime=" + + lastCommitTime + "}"; + + } + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/io/writecache/WriteCacheService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/io/writecache/WriteCacheService.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/io/writecache/WriteCacheService.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -1415,6 +1415,7 @@ */ quorumMember.logWriteCacheBlock(msg, b.duplicate()); + // ASYNC MSG RMI + NIO XFER. remoteWriteFuture = quorumMember.replicate(null/* req */, msg, b.duplicate()); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -5499,6 +5499,13 @@ } @Override + public String getExtendedRunState() { + + throw new UnsupportedOperationException(); + + } + + @Override public long awaitHAReady(final long timeout, final TimeUnit units) throws AsynchronousQuorumCloseException, InterruptedException, TimeoutException { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/AbstractQuorum.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -1054,11 +1054,17 @@ } final public void assertQuorum(final long token) { + + if (this.token == NO_QUORUM) + throw new QuorumException("Quorum not met"); + if (token != NO_QUORUM && this.token == token) { return; } - throw new QuorumException("Expected " + token + ", but is now " - + this.token); + + throw new QuorumException("Quorum not met on token: expected " + token + + ", actual=" + this.token); + } final public void assertLeader(final long token) { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/quorum/Quorum.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -353,8 +353,10 @@ * The token for the quorum. * * @throws QuorumException - * if the quorum is invalid. + * if the quorum is not met. * @throws QuorumException + * if the quorum is not met on a the caller's token. + * @throws QuorumException * if the <i>token</i> is {@link #NO_QUORUM}. */ void assertQuorum(long token); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -358,6 +358,20 @@ private volatile Server jettyServer; /** + * Enum of the run states. The states are labeled by the goal of the run + * state. + */ + private enum RunStateEnum { + Restore, // apply local HALog files GT current commit point. + SeekConsensus, // seek consensus. + RunMet, // run while joined with met quorum. + Resync, // only resynchronization + Rebuild, // online disaster recovery + Error, // error state. + Shutdown; // TODO SHUTDOWN: We are not using this systematically (no ShutdownTask for this run state). + } + + /** * Caching discovery client for the {@link HAGlue} services. */ public HAJournalDiscoveryClient getDiscoveryClient() { @@ -632,8 +646,7 @@ * we would not have to do this since it will already be in the * Shutdown runstate. */ - quorumService.runStateRef - .set(HAQuorumService.RunStateEnum.Shutdown); + quorumService.runStateRef.set(RunStateEnum.Shutdown); /* * Terminate any running task. @@ -759,20 +772,6 @@ */ private final AtomicReference<FutureTask<Void>> runStateFutureRef = new AtomicReference<FutureTask<Void>>(/*null*/); - /** - * Enum of the run states. The states are labeled by the goal of the run - * state. - */ - private enum RunStateEnum { - Restore, // apply local HALog files GT current commit point. - SeekConsensus, // seek consensus. - RunMet, // run while joined with met quorum. - Resync, // only resynchronization - Rebuild, // online disaster recovery - Error, // error state. - Shutdown; // TODO SHUTDOWN: We are not using this systematically (no ShutdownTask for this run state). - } - private final AtomicReference<RunStateEnum> runStateRef = new AtomicReference<RunStateEnum>( null/* none */); @@ -2230,10 +2229,13 @@ final IHAWriteMessage lastLiveMsg = journal.lastLiveHAWriteMessage; - if (lastLiveMsg != null) { + if (lastLiveMsg != null + && lastLiveMsg.getCommitCounter() >= currentWriteSetStateOnLeader + .getCommitCounter()) { /* - * Can not join. Some write has been received. Leader has + * Can not join. Some write has been received for this + * commit point (or a greater commit point). Leader has * moved on. * * Note: [lastLiveMsg] was cleared to [null] when we did a @@ -2493,7 +2495,9 @@ handleResyncMessage((IHALogRequest) req, msg, data); - } else if (journal.getRootBlockView().getCommitCounter() == msg.getCommitCounter() + } else if (req == null // Note: MUST be a live message! + && journal.getRootBlockView().getCommitCounter() == msg + .getCommitCounter() && isJoinedMember(msg.getQuorumToken())) { /* @@ -2780,8 +2784,30 @@ try { + /* + * Throws IllegalStateException if the message is not + * appropriate for the state of the log. + * + * Throws IOException if we can not write on the log. + * + * We catch, log, and rethrow these messages to help diagnose + * problems where the message state is not consistent with the + * log state. + */ journal.getHALogWriter().write(msg, data); + } catch(RuntimeException ex) { + + haLog.error(ex, ex); + + throw ex; + + } catch (IOException ex) { + + haLog.error(ex, ex); + + throw ex; + } finally { logLock.unlock(); @@ -3560,6 +3586,25 @@ } + @Override + public String getExtendedRunState() { + + final HAQuorumService<HAGlue, HAJournal> quorumService = server.quorumService; + + final RunStateEnum innerRunState = (quorumService == null ? null + : quorumService.runStateRef.get()); + + final HAJournal journal = server.journal; + + final String innerRunStateStr = (innerRunState == null ? "N/A" + : (innerRunState.name() + ((innerRunState == RunStateEnum.Resync && journal != null) ? (" @ " + journal + .getRootBlockView().getCommitCounter()) : ""))); + + return "{server=" + server.getRunState() + ", quorumService=" + + innerRunStateStr + "}"; + + } + } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -104,8 +104,15 @@ * The timeout used to await quorum meet or break. */ protected final static long awaitQuorumTimeout = 5000; - + /** + * The timeout for long running LOAD operations. This is used to prevent + * hangs in CI in case the LOAD does not complete for some reason. + */ + protected final static long loadLoadTimeoutMillis = TimeUnit.MINUTES + .toMillis(4); + + /** * A service used to run tasks for some tests. */ protected ExecutorService executorService = null; Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-03-20 23:39:39 UTC (rev 7027) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-03-21 17:03:29 UTC (rev 7028) @@ -31,10 +31,13 @@ import java.io.FileOutputStream; import java.io.IOException; import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.TimeoutException; -import com.bigdata.concurrent.TimeoutException; import com.bigdata.ha.HAGlue; import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; @@ -45,17 +48,16 @@ * Test suites for an {@link HAJournalServer} quorum with a replication factor * of THREE (3) and a fully met {@link Quorum}. * - * TODO In order to guarantee that the LOAD operation does not terminate before - * we are resynchronized, we either need to LOAD a sufficiently large file or we - * need to stream data to the leader. I think we should go with a large file. We - * should also verify that the LOAD is still running when the 3rd service joins - * with the met quorum. - * * TODO Do we have any guards against rolling back a service in RESYNC if the * other services are more than 2 commit points before it? We probably should * not automatically roll it back to the other services in this case, but that * could also reduce the ergonomics of the HA3 configuration. * + * TODO All of these live load remains met tests could also be done with BOUNCE + * rather than SHUTDOWN/RESTART. BOUNCE exercises different code paths and + * corresponds to a zookeeper timeout, e.g., as might occur during a full GC + * pause. + * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> */ public class TestHA3JournalServer extends AbstractHA3JournalServerTestCase { @@ -66,7 +68,7 @@ public TestHA3JournalServer(String name) { super(name); } - + /** * Start 2 services and wait for a quorum meet. Verify that the services * have the same data and that the HALog files exist. Start a 3rd service. @@ -140,27 +142,27 @@ * Now go through a commit point with a fully met quorum. The HALog * files should be purged at that commit point. */ - { + simpleTransaction(); +// { +// +// final StringBuilder sb = new StringBuilder(); +// sb.append("DROP ALL;\n"); +// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); +// sb.append("INSERT DATA {\n"); +// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); +// sb.append(" dc:creator \"A.N.Other\" .\n"); +// sb.append("}\n"); +// +// final String updateStr = sb.toString(); +// +// final HAGlue leader = quorum.getClient().getLeader(token); +// +// // Verify quorum is still valid. +// quorum.assertQuorum(token); +// +// getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); +// } - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - // Current commit point. final long lastCommitCounter2 = serverA .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) @@ -295,30 +297,15 @@ /* * LOAD data on leader. */ - { + final FutureTask<Void> ft = new FutureTask<Void>(new LargeLoadTask( + token, true/* reallyLargeLoad */)); - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); - sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - // Verify quorum is still valid. - quorum.assertQuorum(token); + // Start LOAD. + executorService.submit(ft); + + // Await LOAD, but with a timeout. + ft.get(loadLoadTimeoutMillis, TimeUnit.MILLISECONDS); - repos[0].prepareUpdate(updateStr).evaluate(); - - } - /* * Verify that query on all nodes is allowed and now provides a * non-empty result. @@ -442,27 +429,27 @@ * Now go through a commit point with a met quorum. The HALog * files should be retained at that commit point. */ - { + simpleTransaction(); +// { +// final StringBuilder sb = new StringBuilder(); +// sb.append("DROP ALL;\n"); +// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); +// sb.append("INSERT DATA {\n"); +// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); +// sb.append(" dc:creator \"A.N.Other\" .\n"); +// sb.append("}\n"); +// +// final String updateStr = sb.toString(); +// +// final HAGlue leader = quorum.getClient().getLeader(token); +// +// // Verify quorum is still valid. +// quorum.assertQuorum(token); +// +// getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); +// +// } - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - // Current commit point. final long lastCommitCounter2 = serverA .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) @@ -485,26 +472,8 @@ assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); // Now force further commit when fully met to remove log files - { + simpleTransaction(); - final StringBuilder sb = new StringBuilder(); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book2> dc:title \"Another book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - // And again verify binary equality of ALL journals. assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); @@ -518,30 +487,86 @@ } /** - * Similar to standard resync except that the resync occurs while a sequence of live transactions are in progress. - * @throws Exception + * TWO (2) committed transactions then at 3000ms delay between each + * subsequent transaction. + * <P> + * Note: C should easily catch up with plenty of quiescence + * + * @throws Exception */ - public void testStartAB_C_MultiTransactionResync() throws Exception { - // 2 committed transactions then at 3000ms delay between each subsequent - // C should easily catch up with plenty of quiescence - doStartAB_C_MultiTransactionResync(3000, 2); - - // 5 committed transactions then at 1500ms delay between each subsequent - doStartAB_C_MultiTransactionResync(1500, 5); - - // 5 committed transactions then at 500ms delay between each subsequent - // C may never catch up - doStartAB_C_MultiTransactionResync(500, 5); - - // 0 committed transactions then at 500ms delay between each subsequent - // EARLY STARTING C - doStartAB_C_MultiTransactionResync(500, 0); - } - - public void doStartAB_C_MultiTransactionResync(final long transactionDelay, - final int initialTransactions) throws Exception { - // fail("TEST FAILS"); - try { + public void testStartAB_C_MultiTransactionResync_2tx_then_3000ms_delay() + throws Exception { + + doStartAB_C_MultiTransactionResync(3000, 2); + + } + + /** + * FIVE (5) committed transactions then at 1500ms delay between each + * subsequent transaction. + */ + public void testStartAB_C_MultiTransactionResync_5tx_then_1500ms_delay() + throws Exception { + + doStartAB_C_MultiTransactionResync(1500, 5); + + } + + /** + * FIVE (5) committed transactions then at 500ms delay between each + * subsequent transaction. + * <p> + * Note: C might not catch up since the transactions are spaced more closely + * together. + */ + public void testStartAB_C_MultiTransactionResync_5tx_then_500ms_delay() + throws Exception { + + doStartAB_C_MultiTransactionResync(500, 5); + + } + + /** + * FIVE (5) committed transactions then at 200ms delay between each + * subsequent transaction. + * <p> + * Note: C might not catch up since the transactions are spaced more closely + * together. + */ + public void testStartAB_C_MultiTransactionResync_5tx_then_200ms_delay() + throws Exception { + + doStartAB_C_MultiTransactionResync(200, 5); + + } + + /** + * ZERO (0) committed transactions then at 500ms delay between each + * subsequent transaction. + * <P> + * Note: EARLY STARTING C + */ + public void testStartAB_C_MultiTransactionResync_0tx_then_500ms_delay() + throws Exception { + + doStartAB_C_MultiTransactionResync(500, 0); + } + + /** + * Test where C starts after <i>initialTransactions</i> on A+B. A series of + * transactions are issued with the specified delay. + * + * @param transactionDelay + * The delay between the transactions. + * @param initialTransactions + * The #of initial transactions before starting C. + */ + private void doStartAB_C_MultiTransactionResync( + final long transactionDelay, final int initialTransactions) + throws Exception { + + try { + // Start 2 services. final HAGlue serverA = startA(); final HAGlue serverB = startB(); @@ -552,26 +577,35 @@ // Verify KB exists. awaitKBExists(serverA); + + final HAGlue leader = quorum.getClient().getLeader(token); - /* - * Note: The quorum was not fully met at the last 2-phase commit. - * Instead, 2 services participated in the 2-phase commit and the third - * service resynchronized when it came up and then went through a local - * commit. Therefore, the HALog files should exist on all nodes. - */ + // Verify assumption in this test. + assertEquals(leader, serverA); + // Wait until leader is ready. + leader.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + + /* + * Note: The quorum was not fully met at the last 2-phase commit. + * Instead, 2 services participated in the 2-phase commit and the + * third service resynchronized when it came up and then went + * through a local commit. Therefore, the HALog files should exist + * on all nodes. + */ + // Current commit point. - final long lastCommitCounter = serverA + final long lastCommitCounter = leader .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) .getRootBlock().getCommitCounter(); // There is ONE commit point. assertEquals(1L, lastCommitCounter); - /* - * Verify that HALog files were generated and are available for commit - * point ONE (1) on the services joined with the met quorum. - */ + /* + * Verify that HALog files were generated and are available for + * commit point ONE (1) on the services joined with the met quorum. + */ assertHALogDigestsEquals(1L/* firstCommitCounter */, lastCommitCounter, new HAGlue[] { serverA, serverB }); @@ -582,29 +616,10 @@ * Now go through a commit point with a met quorum. The HALog files * should be retained at that commit point. */ - { + simpleTransaction(); - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - // Current commit point. - final long lastCommitCounter2 = serverA + final long lastCommitCounter2 = leader .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) .getRootBlock().getCommitCounter(); @@ -612,11 +627,9 @@ assertEquals(2L, lastCommitCounter2); // start concurrent task loads that continue until fully met - final AtomicBoolean spin = new AtomicBoolean(false); - final Thread loadThread = new Thread() { - public void run() { + final Callable<Void> task = new Callable<Void>() { + public Void call() throws Exception { int count = 0; - try { while (!quorum.isQuorumFullyMet(token)) { final StringBuilder sb = new StringBuilder(); @@ -635,25 +648,30 @@ // Verify quorum is still valid. quorum.assertQuorum(token); - try { getRemoteRepository(leader).prepareUpdate( updateStr).evaluate(); log.warn("COMPLETED TRANSACTION " + count); Thread.sleep(transactionDelay); - } catch (Exception e) { - fail("Probably unexpected on run " + count, e); - } } - } finally { - spin.set(true); - } + // done. + return null; } }; - loadThread.start(); + final FutureTask<Void> ft = new FutureTask<Void>(task); - // allow number of transactions through first - Thread.sleep(initialTransactions * transactionDelay); + executorService.submit(ft); + + try { + // Allow number of transactions through first + ft.get(initialTransactions * transactionDelay, + TimeUnit.MILLISECONDS); + if (ft.isDone()) { + fail("Not expecting task to be finished."); + } + } catch (TimeoutException ex) { + // Ignore expected exception. + } // Now Start 3rd service. final HAGlue serverC = startC(); @@ -663,9 +681,8 @@ log.info("FULLY MET"); - while (!spin.get()) { - Thread.sleep(50); - } + // Wait for task to end. Check Future. + ft.get(); log.info("Should be safe to test digests now"); @@ -678,31 +695,13 @@ assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); // Now force further commit when fully met to remove log files - { + simpleTransaction(); - final StringBuilder sb = new StringBuilder(); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/bookFinal> dc:title \"Another book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - // And again verify binary equality of ALL journals. // assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); // Now verify no HALog files since fully met quorum @ commit. - final long lastCommitCounter3 = serverA + final long lastCommitCounter3 = leader .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) .getRootBlock().getCommitCounter(); assertHALogNotFound(0L/* firstCommitCounter */, lastCommitCounter3, @@ -713,9 +712,11 @@ } } - /** - * Similar to standard resync except that the resync occurs while a single long transaction is in progress. - * @throws Exception + /** + * Similar to standard resync except that the resync occurs while a single + * long transaction is in progress. + * + * @throws Exception */ public void testStartAB_C_LiveResync() throws Exception { @@ -771,48 +772,57 @@ // There are TWO (2) commit points. assertEquals(2L, lastCommitCounter2); - // start concurrent task loads that continue until fully met - final AtomicBoolean spin = new AtomicBoolean(false); - final Thread loadThread = new Thread() { - public void run() { - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); - sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA\n"); - sb.append("{\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); + /* + * LOAD data on leader. + */ + final FutureTask<Void> ft = new FutureTask<Void>(new LargeLoadTask( + token, true/* reallyLargeLoad */)); - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - try { - getRemoteRepository(leader).prepareUpdate(updateStr) - .evaluate(); - log.info("Updated"); - } catch (Exception e) { - e.printStackTrace(); - - fail("Probably unexpected on run ", e); - } finally { - spin.set(true); - } - } - }; - loadThread.start(); + // Start LOAD. + executorService.submit(ft); + +// // start concurrent task loads that continue until fully met +// final AtomicBoolean spin = new AtomicBoolean(false); +// final Thread loadThread = new Thread() { +// public void run() { +// final StringBuilder sb = new StringBuilder(); +// sb.append("DROP ALL;\n"); +// sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); +// sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); +// sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); +// sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); +// sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); +// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); +// sb.append("INSERT DATA\n"); +// sb.append("{\n"); +// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); +// sb.append(" dc:creator \"A.N.Other\" .\n"); +// sb.append("}\n"); +// +// final String updateStr = sb.toString(); +// +// final HAGlue leader = quorum.getClient().getLeader(token); +// +// // Verify quorum is still valid. +// quorum.assertQuorum(token); +// +// try { +// getRemoteRepository(leader).prepareUpdate(updateStr) +// .evaluate(); +// log.info("Updated"); +// } catch (Exception e) { +// e.printStackTrace(); +// +// fail("Probably unexpected on run ", e); +// } finally { +// spin.set(true); +// } +// } +// }; +// loadThread.start(); // allow load head start - Thread.sleep(300); + Thread.sleep(300/*ms*/); // Now Start 3rd service. final HAGlue serverC = startC(); @@ -822,13 +832,16 @@ log.info("FULLY MET"); - // Need to check if load is active, if not then test has not confirmed active load - assertFalse(spin.get()); - - while (!spin.get()) { - Thread.sleep(50); - } + // Await LOAD, but with a timeout. + ft.get(loadLoadTimeoutMillis, TimeUnit.MILLISECONDS); +// // Need to check if load is active, if not then test has not confirmed active load +// assertFalse(spin.get()); +// +// while (!spin.get()) { +// Thread.sleep(50/*ms*/); +// } + log.info("Should be safe to test digests now"); // Cannot predict last commit counter or whether even logs will remain @@ -840,36 +853,19 @@ assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); // Now force further commit when fully met to remove log files - { + simpleTransaction(); - final StringBuilder sb = new StringBuilder(); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/bookFinal> dc:title \"Another book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - } - - // And again verify binary equality of ALL journals. - // assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); - // Now verify no HALog files since fully met quorum @ commit. final long lastCommitCounter3 = serverA .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) .getRootBlock().getCommitCounter(); + assertHALogNotFound(0L/* firstCommitCounter */, lastCommitCounter3, new HAGlue[] { serverA, serverB, serverC }); + // And again verify binary equality of ALL journals. + assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); + log.info("ALL GOOD!"); } @@ -1277,21 +1273,23 @@ // Verify KB exists. awaitKBExists(startup.serverA); + // Verify A is the leader. assertEquals(startup.serverA, quorum.getClient().getLeader(token)); + // Verify A is fully up. + awaitNSSAndHAReady(startup.serverA); + // Now fail leader! shutdownA(); - // Now check that quorum meets + // Now check that quorum meets around the remaining 2 services. + final long token2 = awaitNextQuorumMeet(token); - final long token2 = awaitMetQuorum(); + // Verify that we have a new leader for the quorum. + final HAGlue leader = quorum.getClient().getLeader(token2); - // with new token - assertTrue(token != token2); - - // and new leader - final HAGlue leader = quorum.getClient().getLeader(token2); - assertTrue(leader.equals(startup.serverB) || leader.equals(startup.serverC)); + assertTrue(leader.equals(startup.serverB) + || leader.equals(startup.serverC)); } /** @@ -1318,17 +1316,6 @@ assertEquals(logsA.listFiles().length, 2); assertEquals(logsB.listFiles().length, 2); -// // Test ends here!! -// -// // but we'll restart to help teardown -// try { -// startA(); -// startB(); -// -// awaitMetQuorum(); -// } catch (Throwable t) { -// log.error("Problem on tidy up", t); -// } } /** @@ -1370,7 +1357,7 @@ assertEquals(logsA.listFiles().length, 1); assertEquals(logsB.listFiles().length, 1); assertEquals(logsC.listFiles().length, 1); -} + } /** * Sandbox stress test, must be disabled before commit for CI runs @@ -1388,7 +1375,6 @@ } } - /** * Tests that halog files are generated and removed after each commit * once fully met. @@ -1399,6 +1385,7 @@ * @throws Exception */ public void testStartABC_halog() throws Exception { + // Start 3 services startA(); startB(); @@ -1558,12 +1545,12 @@ assertLogCount(logsB, 1); assertLogCount(logsC, 1); } - + private void assertLogCount(final File logdir, final int count) { - final int actual = logdir.listFiles().length; - if (actual != count) { - fail("Actual log files: " + actual + ", expected: " + count); - } + final int actual = logdir.listFiles().length; + if (actual != count) { + fail("Actual log files: " + actual + ", expected: " + count); + } } /** @@ -1780,83 +1767,148 @@ awaitPipeline(new HAGlue[] {startup.serverA, serverC2, serverB2}); // and return to quorum assertEquals(token, awaitFullyMetQuorum()); - } + + } /** - * Similar to RemainsMet but with long liveLoad + * Task loads a large data set. + */ + private class LargeLoadTask implements Callable<Void> { + + private final long token; + private final boolean reallyLargeLoad; + + /** + * Large load. + * + * @param token + * The token that must remain valid during the operation. + */ + public LargeLoadTask(final long token) { + + this(token, false/*reallyLargeLoad*/); + + } + + /** + * Either large or really large load. + * + * @param token + * The token that must remain valid during the operation. + * @param reallyLargeLoad + * if we will also load the 3 degrees of freedom file. + */ + public LargeLoadTask(final long token, final boolean reallyLargeLoad) { + + this.token = token; + + this.reallyLargeLoad = reallyLargeLoad; + + } + + public Void call() throws Exception { + + final StringBuilder sb = new StringBuilder(); + sb.append("DROP ALL;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); + if (reallyLargeLoad) + sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); + sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); + sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); + sb.append("INSERT DATA\n"); + sb.append("{\n"); + sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); + sb.append(" dc:creator \"A.N.Other\" .\n"); + sb.append("}\n"); + + final String updateStr = sb.toString(); + + final HAGlue leader = quorum.getClient().getLeader(token); + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + getRemoteRepository(leader).prepareUpdate(updateStr).evaluate... [truncated message content] |
From: <tho...@us...> - 2013-03-21 18:50:18
|
Revision: 7029 http://bigdata.svn.sourceforge.net/bigdata/?rev=7029&view=rev Author: thompsonbry Date: 2013-03-21 18:50:05 +0000 (Thu, 21 Mar 2013) Log Message: ----------- HARestore - added option to transparently decompress a snapshot (the original file is not modified). Tested HARestore on snapshot with 18 HALogs. Rolled forward and was able to use DumpJournal to validate the resulting file. Removed unused pipelineUUIDs field from HAJournal configuration files. Further cleanup on HA3 tests. Modified Paths: -------------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/src/resources/HAJournal/HAJournal.config Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-03-21 18:50:05 UTC (rev 7029) @@ -330,7 +330,7 @@ /** * Apply HALog file(s) to the journal. Each HALog file represents a single * native transaction on the database and will advance the journal by one - * commit point. The journal will go through local commit protocol as each + * commit point. The journal will go through a local commit protocol as each * HALog is applied. HALogs will be applied starting with the first commit * point GT the current commit point on the journal. You may optionally * specify a stopping criteria, e.g., the last commit point that you wish to @@ -357,6 +357,7 @@ * </dl> * * @return <code>0</code> iff the operation was fully successful. + * @throws IOException * * @throws Exception * if the {@link UUID}s or other critical metadata of the @@ -365,7 +366,7 @@ * if an error occcur when reading an HALog or writing on the * journal. */ - public static void main(final String[] args) { + public static void main(final String[] args) throws IOException { if (args.length == 0) { @@ -410,7 +411,7 @@ } - if (i != args.length - 1) { + if (i != args.length - 2) { usage(args); @@ -419,11 +420,40 @@ } // Journal file. - final File journalFile = new File(args[i++]); + File journalFile = new File(args[i++]); // HALogDir. final File haLogDir = new File(args[i++]); + /* + * Decompress the snapshot onto a temporary file in the current working + * directory. + */ + + if (journalFile.getName().endsWith(SnapshotManager.SNAPSHOT_EXT)) { + + // source is the snapshot. + final File in = journalFile; + + final String basename = journalFile.getName().substring( + 0, + journalFile.getName().length() + - SnapshotManager.SNAPSHOT_EXT.length()); + + // temporary file in the same directory as the snapshot. + final File out = File.createTempFile(basename + "-", + Journal.Options.JNL, journalFile.getAbsoluteFile() + .getParentFile()); + + System.out.println("Decompressing " + in + " to " + out); + + // Decompress the snapshot. + SnapshotManager.decompress(in, out); + + journalFile = out; + + } + // Validate journal file. { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-03-21 18:50:05 UTC (rev 7029) @@ -32,6 +32,7 @@ import java.io.FilenameFilter; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.security.DigestException; import java.security.MessageDigest; @@ -1037,7 +1038,10 @@ * {@link IHABufferStrategy#computeDigest(Object, MessageDigest)} * * @param commitCounter + * The commit counter that identifies the snapshot. * @param digest + * The digest. + * * @throws IOException * @throws FileNotFoundException * @throws DigestException @@ -1050,6 +1054,33 @@ final File file = getSnapshotFile(commitCounter); + getSnapshotDigest(file, digest); + + } + + /** + * Compute the digest of a snapshot file. + * <p> + * Note: The digest is only computed for the data beyond the file header. + * This is for consistency with + * {@link IHABufferStrategy#computeDigest(Object, MessageDigest)} + * + * @param commitCounter + * The commit counter that identifies the snapshot. + * @param digest + * The digest. + * + * @throws IOException + * @throws FileNotFoundException + * @throws DigestException + * + * TODO We should pin the snapshot if we are reading it to + * compute its digest. + */ + static public void getSnapshotDigest(final File file, + final MessageDigest digest) throws FileNotFoundException, + IOException, DigestException { + // Note: Throws FileNotFoundException. final GZIPInputStream is = new GZIPInputStream( new FileInputStream(file)); @@ -1097,4 +1128,87 @@ } + /** + * Copy the input stream to the output stream. + * + * @param content + * The input stream. + * @param outstr + * The output stream. + * + * @throws IOException + */ + static private void copyStream(final InputStream content, + final OutputStream outstr) throws IOException { + + final byte[] buf = new byte[1024]; + + while (true) { + + final int rdlen = content.read(buf); + + if (rdlen <= 0) { + + break; + + } + + outstr.write(buf, 0, rdlen); + + } + + } + + /** + * Decompress a snapshot onto the specified file. The original file is not + * modified. + * + * @param src + * The snapshot. + * @param dst + * The file onto which the decompressed snapshot will be written. + * + * @throws IOException + * if the source file does not exist. + * @throws IOException + * if the destination file exists and is not empty. + * @throws IOException + * if there is a problem decompressing the source file onto the + * destination file. + */ + public static void decompress(final File src, final File dst) + throws IOException { + + if (!src.exists()) + throw new FileNotFoundException(src.getAbsolutePath()); + + if (!dst.exists() && dst.length() == 0) + throw new IOException("Output file exists and is not empty: " + + dst.getAbsolutePath()); + + if (log.isInfoEnabled()) + log.info("src=" + src + ", dst=" + dst); + + InputStream is = null; + OutputStream os = null; + try { + is = new GZIPInputStream(new FileInputStream(src)); + os = new FileOutputStream(dst); + copyStream(is, os); + os.flush(); + } finally { + if (is != null) + try { + is.close(); + } catch (IOException ex) { + } + if (os != null) + try { + os.close(); + } catch (IOException ex) { + } + } + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-03-21 18:50:05 UTC (rev 7029) @@ -2073,4 +2073,118 @@ } + /** + * Task loads a large data set. + */ + protected class LargeLoadTask implements Callable<Void> { + + private final long token; + private final boolean reallyLargeLoad; + + /** + * Large load. + * + * @param token + * The token that must remain valid during the operation. + */ + public LargeLoadTask(final long token) { + + this(token, false/*reallyLargeLoad*/); + + } + + /** + * Either large or really large load. + * + * @param token + * The token that must remain valid during the operation. + * @param reallyLargeLoad + * if we will also load the 3 degrees of freedom file. + */ + public LargeLoadTask(final long token, final boolean reallyLargeLoad) { + + this.token = token; + + this.reallyLargeLoad = reallyLargeLoad; + + } + + public Void call() throws Exception { + + final StringBuilder sb = new StringBuilder(); + sb.append("DROP ALL;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); + sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); + if (reallyLargeLoad) + sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); + sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); + sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); + sb.append("INSERT DATA\n"); + sb.append("{\n"); + sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); + sb.append(" dc:creator \"A.N.Other\" .\n"); + sb.append("}\n"); + + final String updateStr = sb.toString(); + + final HAGlue leader = quorum.getClient().getLeader(token); + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + // Done. + return null; + + } + + } + + /** + * Spin, looking for the quorum to fully meet *before* the LOAD is finished. + * + * @return <code>true</code> iff the LOAD finished before the {@link Future} + * was done. + */ + protected boolean awaitFullyMetDuringLOAD(final long token, + final Future<Void> ft) throws InterruptedException, + ExecutionException, TimeoutException { + + final long begin = System.currentTimeMillis(); + boolean fullyMetBeforeLoadDone = false; + while (!fullyMetBeforeLoadDone) { + final long elapsed = System.currentTimeMillis() - begin; + if (elapsed > loadLoadTimeoutMillis) { + /** + * This timeout is a fail safe for LOAD operations that get HUNG + * on the server and prevents CI hangs. + */ + throw new TimeoutException( + "LOAD did not complete in a timely fashion."); + } + try { + if (quorum.isQuorumFullyMet(token) && !ft.isDone()) { + // The quorum is fully met before the load is done. + fullyMetBeforeLoadDone = true; + } + // Check LOAD for error. + ft.get(50/* timeout */, TimeUnit.MILLISECONDS); + // LOAD is done (no errors, future is done). + assertTrue(fullyMetBeforeLoadDone); + break; + } catch (TimeoutException ex) { + // LOAD still running. + continue; + } + } + + return fullyMetBeforeLoadDone; + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config 2013-03-21 18:50:05 UTC (rev 7029) @@ -263,8 +263,6 @@ ); */ - pipelineUUIDs = bigdata.pipeline; - replicationFactor = bigdata.replicationFactor; } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config 2013-03-21 18:50:05 UTC (rev 7029) @@ -262,8 +262,6 @@ ); */ - pipelineUUIDs = bigdata.pipeline; - replicationFactor = bigdata.replicationFactor; } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config 2013-03-21 18:50:05 UTC (rev 7029) @@ -262,8 +262,6 @@ ); */ - pipelineUUIDs = bigdata.pipeline; - replicationFactor = bigdata.replicationFactor; } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-03-21 18:50:05 UTC (rev 7029) @@ -32,8 +32,6 @@ import java.io.IOException; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -42,6 +40,7 @@ import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.quorum.Quorum; +import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** @@ -143,25 +142,6 @@ * files should be purged at that commit point. */ simpleTransaction(); -// { -// -// final StringBuilder sb = new StringBuilder(); -// sb.append("DROP ALL;\n"); -// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); -// sb.append("INSERT DATA {\n"); -// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); -// sb.append(" dc:creator \"A.N.Other\" .\n"); -// sb.append("}\n"); -// -// final String updateStr = sb.toString(); -// -// final HAGlue leader = quorum.getClient().getLeader(token); -// -// // Verify quorum is still valid. -// quorum.assertQuorum(token); -// -// getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); -// } // Current commit point. final long lastCommitCounter2 = serverA @@ -430,25 +410,6 @@ * files should be retained at that commit point. */ simpleTransaction(); -// { -// final StringBuilder sb = new StringBuilder(); -// sb.append("DROP ALL;\n"); -// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); -// sb.append("INSERT DATA {\n"); -// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); -// sb.append(" dc:creator \"A.N.Other\" .\n"); -// sb.append("}\n"); -// -// final String updateStr = sb.toString(); -// -// final HAGlue leader = quorum.getClient().getLeader(token); -// -// // Verify quorum is still valid. -// quorum.assertQuorum(token); -// -// getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); -// -// } // Current commit point. final long lastCommitCounter2 = serverA @@ -781,46 +742,6 @@ // Start LOAD. executorService.submit(ft); -// // start concurrent task loads that continue until fully met -// final AtomicBoolean spin = new AtomicBoolean(false); -// final Thread loadThread = new Thread() { -// public void run() { -// final StringBuilder sb = new StringBuilder(); -// sb.append("DROP ALL;\n"); -// sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); -// sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); -// sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); -// sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); -// sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); -// sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); -// sb.append("INSERT DATA\n"); -// sb.append("{\n"); -// sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); -// sb.append(" dc:creator \"A.N.Other\" .\n"); -// sb.append("}\n"); -// -// final String updateStr = sb.toString(); -// -// final HAGlue leader = quorum.getClient().getLeader(token); -// -// // Verify quorum is still valid. -// quorum.assertQuorum(token); -// -// try { -// getRemoteRepository(leader).prepareUpdate(updateStr) -// .evaluate(); -// log.info("Updated"); -// } catch (Exception e) { -// e.printStackTrace(); -// -// fail("Probably unexpected on run ", e); -// } finally { -// spin.set(true); -// } -// } -// }; -// loadThread.start(); - // allow load head start Thread.sleep(300/*ms*/); @@ -835,13 +756,6 @@ // Await LOAD, but with a timeout. ft.get(loadLoadTimeoutMillis, TimeUnit.MILLISECONDS); -// // Need to check if load is active, if not then test has not confirmed active load -// assertFalse(spin.get()); -// -// while (!spin.get()) { -// Thread.sleep(50/*ms*/); -// } - log.info("Should be safe to test digests now"); // Cannot predict last commit counter or whether even logs will remain @@ -1015,44 +929,71 @@ * @throws Exception */ public void testStartABC_RebuildWithPipelineReorganisation() throws Exception { - final ABC startup = new ABC(true/*sequential*/); - - awaitFullyMetQuorum(); + + new ABC(true/* sequential */); + + awaitFullyMetQuorum(); // Now run several transactions for (int i = 0; i < 5; i++) - simpleTransaction(); - + simpleTransaction(); + // shutdown AB and destroy C destroyC(); shutdownA(); shutdownB(); + + /* + * Now restart A, B & C. + * + * Note: We start C first so it will be in the way when A or B attempts + * to become the leader, thus forcing a pipeline reorganization. + */ + final HAGlue serverC = startC(); + awaitPipeline(new HAGlue[] { serverC }); + + // Now start A. + final HAGlue serverA = startA(); + awaitPipeline(new HAGlue[] { serverC, serverA }); - // Now restart A, B & C - final HAGlue serverC = startC(); - awaitPipeline(new HAGlue[] {serverC}); - - final HAGlue serverA = startA(); - awaitPipeline(new HAGlue[] {serverC, serverA}); - final HAGlue serverB = startB(); - - // A & B should meet - awaitMetQuorum(); - - awaitPipeline(new HAGlue[] {serverA, serverB, serverC}); - - // Check HALogs equal - assertHALogDigestsEquals(7L/* firstCommitCounter */, - 7L, new HAGlue[] { serverA, serverB }); - log.warn("CHECK AB LOGS ON MET QUORUM"); - - // C will have go through Rebuild before joining - awaitFullyMetQuorum(); + // And finally start B. + final HAGlue serverB = startB(); + + // A & B should meet + final long token2 = awaitMetQuorum(); + + // The expected pipeline. C was moved to the end. + awaitPipeline(new HAGlue[] { serverA, serverB, serverC }); + + // Wait until A is fully ready. + assertEquals(HAStatusEnum.Leader, awaitNSSAndHAReady(serverA)); + // Verify A is the leader. + assertEquals(serverA, quorum.getClient().getLeader(token2)); + + // Check journals for equality on A, B. + assertDigestsEquals(new HAGlue[] { serverA, serverB }); + + // Check HALogs equal on A, B. + assertHALogDigestsEquals(7L/* firstCommitCounter */, 7L, new HAGlue[] { + serverA, serverB }); + + // C will have go through Rebuild before joining + assertEquals(token2, awaitFullyMetQuorum()); + +// Note: I have seen this timeout. This warrants exploring. BBT. +// // Wait until C is fully ready. +// assertEquals(HAStatusEnum.Follower, awaitNSSAndHAReady(serverC)); + // Verify binary equality of ALL journals. assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); + + // Check HALogs equal on ALL services. + assertHALogDigestsEquals(7L/* firstCommitCounter */, 7L, new HAGlue[] { + serverA, serverB, serverC }); + } - + /** * Test Rebuild of C service where quorum was previously * fully met and where a new quorum is met before C joins for rebuild. @@ -1771,120 +1712,6 @@ } /** - * Task loads a large data set. - */ - private class LargeLoadTask implements Callable<Void> { - - private final long token; - private final boolean reallyLargeLoad; - - /** - * Large load. - * - * @param token - * The token that must remain valid during the operation. - */ - public LargeLoadTask(final long token) { - - this(token, false/*reallyLargeLoad*/); - - } - - /** - * Either large or really large load. - * - * @param token - * The token that must remain valid during the operation. - * @param reallyLargeLoad - * if we will also load the 3 degrees of freedom file. - */ - public LargeLoadTask(final long token, final boolean reallyLargeLoad) { - - this.token = token; - - this.reallyLargeLoad = reallyLargeLoad; - - } - - public Void call() throws Exception { - - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); - if (reallyLargeLoad) - sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); - sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA\n"); - sb.append("{\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - final HAGlue leader = quorum.getClient().getLeader(token); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - // Done. - return null; - - } - - } - - /** - * Spin, looking for the quorum to fully meet *before* the LOAD is finished. - * - * @return <code>true</code> iff the LOAD finished before the {@link Future} - * was done. - */ - private boolean awaitFullyMetDuringLOAD(final long token, - final Future<Void> ft) throws InterruptedException, - ExecutionException, TimeoutException { - - final long begin = System.currentTimeMillis(); - boolean fullyMetBeforeLoadDone = false; - while (!fullyMetBeforeLoadDone) { - final long elapsed = System.currentTimeMillis() - begin; - if (elapsed > loadLoadTimeoutMillis) { - /** - * This timeout is a fail safe for LOAD operations that get HUNG - * on the server and prevents CI hangs. - */ - throw new TimeoutException( - "LOAD did not complete in a timely fashion."); - } - try { - if (quorum.isQuorumFullyMet(token) && !ft.isDone()) { - // The quorum is fully met before the load is done. - fullyMetBeforeLoadDone = true; - } - // Check LOAD for error. - ft.get(50/* timeout */, TimeUnit.MILLISECONDS); - // LOAD is done (no errors, future is done). - assertTrue(fullyMetBeforeLoadDone); - break; - } catch (TimeoutException ex) { - // LOAD still running. - continue; - } - } - - return fullyMetBeforeLoadDone; - - } - - /** * Start A+B+C in strict sequence. Wait until the quorum fully meets. Start * a long running LOAD. While the LOAD is running, fail C (the last * follower). Verify that the LOAD completes successfully with the remaining Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-03-21 18:50:05 UTC (rev 7029) @@ -479,30 +479,8 @@ /* * LOAD data on leader. */ - { + new LargeLoadTask(token, true/* reallyLargeLoad */).call(); - final StringBuilder sb = new StringBuilder(); - sb.append("DROP ALL;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-0.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-1.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-2.nq.gz") + ">;\n"); - sb.append("LOAD <" + getFoafFileUrl("data-3.nq.gz") + ">;\n"); - sb.append("INSERT {?x rdfs:label ?y . } WHERE {?x foaf:name ?y };\n"); - sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); - sb.append("INSERT DATA {\n"); - sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); - sb.append(" dc:creator \"A.N.Other\" .\n"); - sb.append("}\n"); - - final String updateStr = sb.toString(); - - // Verify quorum is still valid. - quorum.assertQuorum(token); - - repos[0].prepareUpdate(updateStr).evaluate(); - - } - /* * Verify that query on all nodes is allowed and now provides a * non-empty result. Modified: branches/READ_CACHE/src/resources/HAJournal/HAJournal.config =================================================================== --- branches/READ_CACHE/src/resources/HAJournal/HAJournal.config 2013-03-21 17:03:29 UTC (rev 7028) +++ branches/READ_CACHE/src/resources/HAJournal/HAJournal.config 2013-03-21 18:50:05 UTC (rev 7029) @@ -51,16 +51,7 @@ /* * This is a sample configuration file for a highly available Journal. A * version of this file must be available to each HAJournalServer in the - * pipeline. The pipeline depends on the stable assignment of ServiceID - * to HAJournalServers. A unique ServiceID must be explicitly assigned to - * each HAJournalServer in its configuration entry. The ordered list of - * those ServiceIDs is shared by all services and defines the write - * replication pipeline. The first entry in the write replication pipeline - * is the leader (aka master). You can use UUID.randomUUID() or GenUUID - * to create UUIDs. - * - * Note: The ServiceUUID Entry MUST be different for each file. It assigns - * a ServiceID to the service! + * pipeline. */ /* @@ -321,8 +312,6 @@ bigdata.haPort ); - pipelineUUIDs = bigdata.pipeline; - replicationFactor = bigdata.replicationFactor; haLogDir = bigdata.haLogDir; This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-04 21:34:17
|
Revision: 7035 http://bigdata.svn.sourceforge.net/bigdata/?rev=7035&view=rev Author: thompsonbry Date: 2013-04-04 21:34:01 +0000 (Thu, 04 Apr 2013) Log Message: ----------- Continued work on HA Backup. Test suite all green except for "join during live load" and the new test for the HA restore policy. In addition, there are some new tests to be written, which I have outlined below. - Write unit test to verify that A+B take a snapshot of the empty journal. The snapshot should not exist on either service until both are running and the quorum meets. - Write unit test to verify that C snapshots the journal when it enters RunMet after resynchronizing from A+B. (This can just be start A+B, await quorum meet, then start C. C will resync from the leader. The snapshot should be taken when resync is done and we enter RunMet.) - Write unit test to verify that C snapshots the journal when it is rebuilding from the A+B. The leader must not have the HALogs on hand to trigger a REBUILD rather than a RESYNC. E.g., delete the log files and snapshots on the leader by hand. C should snapshot the journal when the REBUILD and RESYNC are done and it enters RunMet. - Write test to verify that a rebuild triggered while a snapshot is running will cancel the running snapshot. This is necessary since we must overwrite the root blocks. Start A+B+C. Load a bunch of data. Request a snapshot on C. Write on leader. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/DumpJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java branches/READ_CACHE/bigdata/src/resources/logging/log4j-dev.properties branches/READ_CACHE/bigdata/src/test/com/bigdata/journal/ha/AbstractHAJournalTestCase.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ForeverRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/IRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-A.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-B.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/HAJournal-C.config branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServerGlobalWriteLock.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java branches/READ_CACHE/src/resources/HAJournal/HAJournal.config Added Paths: ----------- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3BackupTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java Removed Paths: ------------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoRestorePolicy.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -180,14 +180,8 @@ * goes through a commit point in which the quorum is fully met. At that * moment, we no longer require these log files to resynchronize any * service. - * - * NOTE: We should never remove the open log file - * - * This parameter has been removed - * - * @param includeCurrent - * When <code>true</code>, the current HA Log file will also be - * purged. + * <p> + * Note: The open HALog file is NOT removed by this method. */ void purgeHALogs(); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumService.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumService.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -123,4 +123,22 @@ void installRootBlocks(final IRootBlockView rootBlock0, final IRootBlockView rootBlock1); + /** + * Callback method. + * + * @param token + * The token on which the service joined a met quorum. + * @param commitCounter + * The commitCounter for the local service. + * @param isLeader + * <code>true</code> iff the local service is the quorum leader. + */ + /* + * I added this in but wound up not needed it. Do not use without good + * justification. + */ + @Deprecated + void didMeet(final long token, final long commitCounter, + final boolean isLeader); + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -4905,9 +4905,16 @@ } + // This quorum member. + final QuorumService<HAGlue> localService = quorum.getClient(); + /* * Both a meet and a break require an exclusive write lock. */ + final boolean isLeader; + final boolean isFollower; + final long localCommitCounter; + final WriteLock lock = _fieldReadWriteLock.writeLock(); lock.lock(); @@ -4936,6 +4943,9 @@ * widthdrawn. That is currently done by QuorumWatcherBase. So, * we have to wait until we observe that to cast a new vote. */ + + localCommitCounter = -1; + isLeader = isFollower = false; haReadyToken = Quorum.NO_QUORUM; // volatile write. @@ -4947,14 +4957,16 @@ quorumToken = newValue; - // This quorum member. - final QuorumService<HAGlue> localService = quorum.getClient(); - boolean installedRBs = false; + localCommitCounter = _rootBlock.getCommitCounter(); + if (localService.isFollower(quorumToken)) { - if (_rootBlock.getCommitCounter() == 0L) { + isLeader = false; + isFollower = true; + + if (localCommitCounter == 0L) { /* * Take the root blocks from the quorum leader and use @@ -5003,11 +5015,17 @@ } else if (localService.isLeader(quorumToken)) { + isLeader = true; + isFollower = false; + // ready as leader. tmp = newValue; } else { + isLeader = false; + isFollower = false; + // Not ready. tmp = Quorum.NO_QUORUM; @@ -5027,7 +5045,7 @@ doLocalAbort(); } - + this.haReadyToken = tmp; // volatile write. haReadyCondition.signalAll(); // signal ALL. @@ -5043,7 +5061,13 @@ lock.unlock(); } + + if (isLeader || isFollower) { + localService.didMeet(newValue, localCommitCounter, isLeader); + + } + } private final Condition haReadyCondition = _fieldReadWriteLock.writeLock().newCondition(); private volatile long haReadyToken = Quorum.NO_QUORUM; @@ -5192,6 +5216,14 @@ throw new IllegalArgumentException(); if (rootBlock1.isRootBlock0()) throw new IllegalArgumentException(); +// if (rootBlock0.getCommitCounter() != 0L) +// throw new IllegalArgumentException(); +// if (rootBlock1.getCommitCounter() != 0L) +// throw new IllegalArgumentException(); + if (!rootBlock0.getStoreType().equals(rootBlock1.getStoreType())) + throw new IllegalArgumentException(); + if (!rootBlock0.getUUID().equals(rootBlock1.getUUID())) + throw new IllegalArgumentException(); // if (_rootBlock.getCommitCounter() != 0) { // Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/DumpJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/DumpJournal.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/DumpJournal.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -155,6 +155,8 @@ boolean showTuples = false; + boolean alternateRootBlock = false; + final List<Long> addrs = new LinkedList<Long>(); for(; i<args.length; i++) { @@ -197,7 +199,13 @@ showTuples = true; } - + + else if(arg.equals("-alternateRootBlock")) { + + alternateRootBlock = true; + + } + else if(arg.equals("-addr")) { addrs.add(Long.valueOf(args[i + 1])); @@ -254,6 +262,10 @@ properties.setProperty(Options.READ_ONLY, "" + true); + if (alternateRootBlock) + properties.setProperty(Options.ALTERNATE_ROOT_BLOCK, + "" + true); + properties.setProperty(Options.BUFFER_MODE, BufferMode.Disk.toString()); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/RootBlockView.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -627,6 +627,46 @@ return new RootBlockView(rootBlock0, asReadOnlyBuffer(), checker); } + + /** + * {@inheritDoc} + * <p> + * Overridden to implement the hash code as the hash code of the data in the + * backing {@link ByteBuffer}. The {@link IRootBlockView} is immutable. The + * hash code is computed lazily. Once computed, the hash code is NOT + * recomputed for the same {@link IRootBlockView}. + */ + @Override + public int hashCode() { + if (hash == 0) { + /* + * Computed once (unless there is a data race, in which case it + * might be computed more than once but it will be consistent). + */ + hash = buf.asReadOnlyBuffer().hashCode(); + } + return hash; + } + + private volatile int hash = 0; + + /** + * {@inheritDoc} + * + * Overriden to implement equality based on the data in the + * {@link IRootBlockView}. + */ + @Override + public boolean equals(final Object o) { + if (this == o) + return true; + if (!(o instanceof IRootBlockView)) + return false; + final IRootBlockView o2 = (IRootBlockView) o; + if (!buf.asReadOnlyBuffer().equals(o2.asReadOnlyBuffer())) + return false; + return true; + } /** * Create a new read-only view from the supplied buffer. Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -2616,6 +2616,17 @@ */ m_writeCacheService.close(); m_writeCacheService = newWriteCache(); + } else if (m_writeCacheService != null) { + /* + * Note: We DO NOT need to reset() the WriteCacheService. If a + * record was already flushed to the disk, then it is on the + * disk and clearing the record from the cache will not change + * that. If the record has not yet been flushed to the disk, + * then we already cleared it from the WCS when we reset the + * FixedAllocators (above). + */ +// m_writeCacheService.reset(); +// m_writeCacheService.setExtent(convertAddr(m_fileSize)); } /* * Discard any writes on the delete blocks. Those deletes MUST NOT Modified: branches/READ_CACHE/bigdata/src/resources/logging/log4j-dev.properties =================================================================== --- branches/READ_CACHE/bigdata/src/resources/logging/log4j-dev.properties 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/resources/logging/log4j-dev.properties 2013-04-04 21:34:01 UTC (rev 7035) @@ -269,13 +269,13 @@ # Normal data loader (single threaded). log4j.logger.com.bigdata.rdf.store.DataLoader=INFO -#log4j.logger.com.bigdata.ha=ALL -#log4j.logger.com.bigdata.txLog=ALL -#log4j.logger.com.bigdata.haLog=ALL +log4j.logger.com.bigdata.ha=ALL +log4j.logger.com.bigdata.txLog=ALL +log4j.logger.com.bigdata.haLog=ALL #log4j.logger.com.bigdata.rwstore=ALL -#log4j.logger.com.bigdata.journal=ALL +log4j.logger.com.bigdata.journal=INFO #log4j.logger.com.bigdata.journal.AbstractBufferStrategy=ALL -#log4j.logger.com.bigdata.journal.jini.ha=ALL +log4j.logger.com.bigdata.journal.jini.ha=ALL #log4j.logger.com.bigdata.service.jini.lookup=ALL #log4j.logger.com.bigdata.quorum=ALL #log4j.logger.com.bigdata.quorum.zk=ALL @@ -285,7 +285,7 @@ log4j.logger.benchmark.bigdata.TestBSBM=INFO # Test suite logger. -#log4j.logger.junit=INFO +log4j.logger.junit=INFO #log4j.logger.junit=DEBUG log4j.logger.com.bigdata.btree.AbstractBTreeTestCase=INFO Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/journal/ha/AbstractHAJournalTestCase.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/journal/ha/AbstractHAJournalTestCase.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/journal/ha/AbstractHAJournalTestCase.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -472,6 +472,12 @@ } @Override + public void didMeet(final long token, final long commitCounter, + final boolean isLeader) { + throw new UnsupportedOperationException(); + } + + @Override public File getServiceDir() { throw new UnsupportedOperationException(); } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -143,8 +143,6 @@ if(index == -1) { // No match. - log.warn("Not found: " + timestamp); - return null; } @@ -229,13 +227,13 @@ pos = -(pos+1); - if(pos == 0) { + if (pos == 0) { // No entry is less than or equal to this timestamp. return -1; - + } - + pos--; return pos; @@ -290,41 +288,164 @@ super.insert(key, BytesUtil.getBytes(rootBlock.asReadOnlyBuffer())); } + + /** + * Find and return the {@link IRootBlockView} for the oldest snapshot (if + * any). + * + * @return That {@link IRootBlockView} -or- <code>null</code> if there are + * no snapshots. + */ + public IRootBlockView getOldestSnapshot() { + + synchronized (this) { + + if (getEntryCount() == 0L) { + + // Empty index. + return null; + + } + + // Lookup first tuple in index. + @SuppressWarnings("unchecked") + final ITuple<IRootBlockView> t = valueAt(0L, getLookupTuple()); + + final IRootBlockView rb = t.getObject(); + + return rb; + + } + + } /** - * Find the commit counter for the most recent snapshot (if any). + * Find the {@link IRootBlockView} for the most recent snapshot (if any). * - * @return That commit counter -or- ZERO (0L) if there are no snapshots. + * @return That {@link IRootBlockView} -or- <code>null</code> if there are + * no snapshots. */ - public long getMostRecentSnapshotCommitCounter() { + public IRootBlockView getNewestSnapshot() { - final long snapshotCommitCounter; + /* + * Note: This could also be written using valueAt(nentries). + */ synchronized (this) { @SuppressWarnings("unchecked") - final ITupleIterator<IRootBlockView> itr = - rangeIterator(null/* fromKey */, null/* toKey */, - 1/* capacity */, IRangeQuery.DEFAULT - | IRangeQuery.REVERSE/* flags */, null/* filter */); + final ITupleIterator<IRootBlockView> itr = rangeIterator( + null/* fromKey */, null/* toKey */, 1/* capacity */, + IRangeQuery.DEFAULT | IRangeQuery.REVERSE/* flags */, null/* filter */); - if (itr.hasNext()) { + if (!itr.hasNext()) { + return null; + + } + + final ITuple<IRootBlockView> t = itr.next(); + + final IRootBlockView rb = t.getObject(); + + return rb; + + } + + } + + /** + * Find the oldest snapshot whose commit counter is LTE the specified commit + * counter. + * + * @return The {@link IRootBlockView} for that snapshot -or- + * <code>null</code> if there is no such snapshot. + * + * @throws IllegalArgumentException + * if <code>commitCounter LT ZERO (0)</code> + */ + public IRootBlockView findByCommitCounter(final long commitCounter) { + + if (commitCounter < 0L) + throw new IllegalArgumentException(); + + synchronized (this) { + + // Reverse scan. + @SuppressWarnings("unchecked") + final ITupleIterator<IRootBlockView> itr = rangeIterator( + null/* fromKey */, null/* toKey */, 0/* capacity */, + IRangeQuery.DEFAULT | IRangeQuery.REVERSE/* flags */, null/* filter */); + + while (itr.hasNext()) { + final ITuple<IRootBlockView> t = itr.next(); - - final IRootBlockView rootBlock = t.getObject(); - snapshotCommitCounter = rootBlock.getCommitCounter(); + final IRootBlockView rb = t.getObject(); - } else { + if (rb.getCommitCounter() <= commitCounter) { - snapshotCommitCounter = 0L; + // First snapshot LTE that commit counter. + return rb; + } + } + return null; + } + + } + + /** + * Return the snapshot that is associated with the specified ordinal index + * (origin ZERO) counting backwards from the most recent snapshot (0) + * towards the earliest snapshot (nsnapshots-1). + * <p> + * Note: The effective index is given by <code>(entryCount-1)-index</code>. + * If the effective index is LT ZERO (0) then there is no such snapshot and + * this method will return <code>null</code>. + * + * @param index + * The index. + * + * @return The {@link IRootBlockView} for that snapshot -or- + * <code>null</code> if there is no such snapshot. + * + * @throws IllegalArgumentException + * if <code>index LT ZERO (0)</code> + */ + public IRootBlockView getSnapshotByReverseIndex(final int index) { - return snapshotCommitCounter; + if (index < 0) + throw new IllegalArgumentException(); + synchronized (this) { + + final long entryCount = getEntryCount(); + + if (entryCount > Integer.MAX_VALUE) + throw new AssertionError(); + + final int effectiveIndex = ((int) entryCount - 1) - index; + + if (effectiveIndex < 0) { + + // No such snapshot. + return null; + + } + + @SuppressWarnings("unchecked") + final ITuple<IRootBlockView> t = valueAt(effectiveIndex, + getLookupTuple()); + + final IRootBlockView rb = t.getObject(); + + return rb; + + } + } /** Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -25,61 +25,193 @@ import java.util.concurrent.TimeUnit; +import org.apache.log4j.Logger; + import com.bigdata.journal.IRootBlockView; /** - * The default restore policy. + * The default restore policy. This policy supports three different criteria for + * deciding when snapshots (aka full backups) and HALogs (aka write ahead log + * files for each commit point aka incremental backups) may be purged. + * <dl> + * <dt>minSnapshotAgeMillis</dt> + * <dd>The minimum age of a snapshot before it may be deleted.</dd> + * <dt>minSnapshots</dt> + * <dd>The minimum number of snapshot files (aka full backups) that must be + * retained.</dd> + * <dt>minRestorePoints</dt> + * <dd>The minimum number of commit points that must be restorable from backup. + * This explicitly controls the number of HALog files that will be retained. It + * also implicitly controls the number of snapshot files that will be retained + * since an HALog file will pin the newest snapshot whose commit counter is LTE + * to the the closing commit counter on that HALog file.</dd> + * </dl> * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> */ public class DefaultRestorePolicy implements IRestorePolicy { - private final long millis; + private static final Logger log = Logger + .getLogger(DefaultRestorePolicy.class); + + /** + * The default minimum age of a snapshot before it may be purged (7 days). + */ + public static final long DEFAULT_MIN_SNAPSHOT_AGE_MILLIS = TimeUnit.DAYS + .toMillis(7); /** + * The default minimum #of snapshots which must be retained ( + * {@value #DEFAULT_MIN_SNAPSHOTS}). + */ + public static final int DEFAULT_MIN_SNAPSHOTS = 1; + + /** + * The default minimum #of commit points that must be retained ( + * {@value #DEFAULT_MIN_RESTORE_POINTS}). + */ + public static final int DEFAULT_MIN_RESTORE_POINTS = 0; + + /** + * The minimum age of a snapshot before it may be purged. + */ + private final long minSnapshotAgeMillis; + + /** + * The minimum #of snapshots that must be retained. + */ + private final int minSnapshots; + + /** + * The minimum #of restore points (HALogs) that must be retained. + */ + private final int minRestorePoints; + + @Override + public String toString() { + + return DefaultRestorePolicy.class.getSimpleName()// + + "{minSnapshotAge=" + minSnapshotAgeMillis + "ms"// + + ",minSnapshots=" + minSnapshots // + + ",minRestorePoints=" + minRestorePoints // + + "}"; + + } + + /** * The default is to keep local backups on hand for 7 days. */ public DefaultRestorePolicy() { - this(TimeUnit.DAYS.toMillis(7)); - + this(DEFAULT_MIN_SNAPSHOT_AGE_MILLIS, DEFAULT_MIN_SNAPSHOTS, + DEFAULT_MIN_RESTORE_POINTS); + } /** - * Create a policy that will keep local backups on hand for the specified - * number of milliseconds. + * Create a policy that determines when local backups may be purged. The + * policy will retain local backups unless all of the criteria are + * satisified. * - * @param millis - * The #of milliseconds of state that can be restored from local - * backups. + * @param minSnapshotAgeMillis + * The minimum age of a snapshot (in milliseconds) before it may + * be purged. */ - public DefaultRestorePolicy(final long millis) { - - if (millis < 0) - throw new IllegalArgumentException(); + public DefaultRestorePolicy(final long minSnapshotAgeMillis) { - this.millis = millis; - + this(minSnapshotAgeMillis, DEFAULT_MIN_SNAPSHOTS, + DEFAULT_MIN_RESTORE_POINTS); + } /** + * Create a policy that determines when local backups may be purged. The + * policy will retain local backups unless all of the criteria are + * satisified. + * + * @param minSnapshotAgeMillis + * The minimum age of a snapshot (in milliseconds) before it may + * be purged. + * @param minSnapshots + * The minimum number of snapshots (aka full backups) that must + * be retained locally. + * @param minRestorePoints + * The minimum number of restore points (aka HALog files) that + * must be retained locally. If an HALog is pinned by this + * parameter, then the oldest snapshot LTE the commit counter of + * that HALog is also pinned, as are all HALog files GTE the + * snapshot and LT the HALog. + */ + public DefaultRestorePolicy(final long minSnapshotAgeMillis, + final int minSnapshots, final int minRestorePoints) { + + if (minSnapshotAgeMillis < 0) + throw new IllegalArgumentException( + "minSnapshotAgeMillis must be GTE ZERO (0), not " + + minSnapshotAgeMillis); + + if (minSnapshots < 1) + throw new IllegalArgumentException( + "minSnapshots must be GTE ONE (1), not " + minSnapshots); + + if (minRestorePoints < 0) + throw new IllegalArgumentException( + "minRestorePoints must be GTE ZERO (0), not " + + minRestorePoints); + + this.minSnapshotAgeMillis = minSnapshotAgeMillis; + + this.minSnapshots = minSnapshots; + + this.minRestorePoints = minRestorePoints; + + } + + /** * This finds and returns the commit counter for the most recent snapshot * whose commit time is LTE <code>now - millis</code>, where <i>millis</i> - * is the #of milliseconds specified by the constructor for this policy. The - * return value will be ZERO (0) if there are no commit points. + * is the #of milliseconds specified by the constructor for this policy. */ - @Override - public long getEarliestRestorableCommitPoint(final HAJournal jnl) { + private long getEarliestRestorableCommitCounterByAge(final HAJournal jnl, + final long commitCounterOnJournal) { final long now = System.currentTimeMillis(); - - final long then = now - millis; + final long then = now - minSnapshotAgeMillis; + final IRootBlockView rootBlock = jnl.getSnapshotManager().find(then); if (rootBlock == null) { // There are no snapshots. + return commitCounterOnJournal; + + } + + return rootBlock.getCommitCounter(); + + } + + /** + * This finds the snapshot that is <i>minSnapshots</i> back and returns its + * commit counter. If there are fewer than <i>minSnapshots</i>, then this + * returns ZERO (0). + */ + private long getEarliestRestorableCommitCounterBySnapshots( + final HAJournal jnl, final long commitCounterOnJournal) { + + if (minSnapshots == 0) { + + return commitCounterOnJournal; + + } + + final IRootBlockView rootBlock = jnl.getSnapshotManager() + .getSnapshotByReverseIndex(minSnapshots); + + if (rootBlock == null) { + + // There are fewer than minSnapshots snapshots. return 0L; } @@ -88,4 +220,80 @@ } + /** + * Find the oldest snapshot that is at least <i>minRestorePoints</i> old and + * returns its commit counter. If there is no such snapshot, then this + * returns ZERO (0). + */ + private long getEarliestRestorableCommitCounterByHALogs( + final HAJournal jnl, final long commitCounterOnJournal) { + + // The commit point that is [minRestorePoints] old. + final long desiredCommitCounter = commitCounterOnJournal + - minRestorePoints; + + if (desiredCommitCounter <= 0) { + + // There are fewer than this many commit points on the journal. + return 0L; + + } + + // Find the oldest snapshot LTE that commitCounter. + final IRootBlockView rootBlock = jnl.getSnapshotManager() + .findByCommitCounter(desiredCommitCounter); + + if (rootBlock == null) { + + return commitCounterOnJournal; + + } + + return rootBlock.getCommitCounter(); + + } + + /** + * {@inheritDoc} + * <p> + * Note: We use three different criteria here. Of necessity, this decision + * can not be atomic. Therefore, it is possible for new snapshots and HALogs + * to be created while we are concurrently evaluating those criteria. + * However, this concurrency can never result in fewer files being retained + * than are required by this policy. + */ + @Override + public long getEarliestRestorableCommitPoint(final HAJournal jnl) { + + // Current commit point on the journal. + final long commitCounterOnJournal = jnl.getRootBlockView() + .getCommitCounter(); + + final long commitCounterByAge = getEarliestRestorableCommitCounterByAge( + jnl, commitCounterOnJournal); + + final long commitCounterBySnapshots = getEarliestRestorableCommitCounterBySnapshots( + jnl, commitCounterOnJournal); + + final long commitCounterByHALogs = getEarliestRestorableCommitCounterByHALogs( + jnl, commitCounterOnJournal); + + final long ret = Math.min(commitCounterByAge, + Math.min(commitCounterBySnapshots, commitCounterByHALogs)); + + if (log.isInfoEnabled()) { + + log.info("policy=" + this + ", commitCounterOnJournal=" + + commitCounterOnJournal + ", commitCounterByAge=" + + commitCounterByAge + ", commitCounterBySnapshots=" + + commitCounterBySnapshots + ", commitCounterByHALogs=" + + commitCounterByHALogs + + ", effectiveCommitCounterReported=" + ret); + + } + + return ret; + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -25,6 +25,7 @@ import java.lang.ref.WeakReference; import java.util.Calendar; +import java.util.Formatter; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -47,6 +48,24 @@ final private int timeOfDay; final private int percentLogSize; + + @Override + public String toString() { + + final StringBuilder sb = new StringBuilder(); + + final Formatter f = new Formatter(sb); + + f.format("%04d", timeOfDay); + f.flush(); + f.close(); + + final String todStr = sb.toString(); + + return DefaultSnapshotPolicy.class.getSimpleName() + "{timeOfDay=" + + todStr + ", percentLogSize=" + percentLogSize + "%}"; + + } /** * The default policy wakes up at <code>0200</code> and takes a snapshot if Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ForeverRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ForeverRestorePolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ForeverRestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -40,7 +40,7 @@ * This policy always returns ZERO to prevent backups from being released. */ @Override - public long getEarliestRestorableCommitPoint(HAJournal jnl) { + public long getEarliestRestorableCommitPoint(final HAJournal jnl) { return 0; Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -414,6 +414,36 @@ super.setQuorumToken(newValue); + if (newValue == Quorum.NO_QUORUM) { + + /* + * If there is a running snapshot, then cancel it since the quorum + * has broken. + * + * Note: The snapshot task will automatically terminate if it + * observes a quorum break or similar event. This is just being + * proactive. + * + * TODO Lift into HAJournalServer.quorumBreak() handler? + * + * TODO This will not be called if the quorum remains met but the + * local service leaves the quorum. However, we should still cancel + * a running snapshot if that occurs. + */ + + final Future<IHASnapshotResponse> ft = getSnapshotManager() + .getSnapshotFuture(); + + if (ft != null && !ft.isDone()) { + + haLog.info("Canceling snapshot."); + + ft.cancel(true/* mayInterruptIfRunning */); + + } + + } + } /** Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -264,6 +264,26 @@ ISnapshotPolicy DEFAULT_SNAPSHOT_POLICY = new DefaultSnapshotPolicy(); +// /** +// * <strong>TEST SUITE OPTION ONLY!</strong> +// * <p> +// * By default, a snapshot will be taken the first time the quorum meets +// * for each service. This provide the initial restore point, which +// * corresponds to an empty {@link HAJournal} with the correct root +// * blocks for the quorum. +// * <p> +// * This option MAY be used to suppress this behavior. This is used by +// * the test suite to avoid the creation of the initial snapshot. In +// * combination with an {@link DefaultRestorePolicy} which specifies +// * <code>minRestorePoints:=0</code>, this has the effect that we do not +// * hold onto HALog files (other than the current HALog file) until a +// * snapshot has been taken. The test suites control when snapshots are +// * taken and are thus able to test a variety of backup scenarios. +// */ +// String SNAPSHOT_ON_FIRST_MEET = "snapshotOnFirstMeet"; +// +// boolean DEFAULT_SNAPSHOT_ON_FIRST_MEET = true; + /** * The policy identifies the first commit point whose backups MUST NOT * be released. The policy may be based on the age of the commit point, @@ -1227,7 +1247,15 @@ final long token = getQuorum().token(); if (getQuorum().isQuorumFullyMet(token)) { - + /* + * TODO Even though the quorum is fully met, we should wait + * until we have a positive indication from the leader that + * it is "ha ready" before purging the HA logs and aging put + * snapshots. The leader might need to explicitly schedule + * this operation against the joined services and the + * services should then verify that the quorum is fully met + * before they actually age out the HALogs and snapshots. + */ purgeHALogs(); } @@ -1462,6 +1490,16 @@ } // validation of pre-conditions. + /* + * Conditionally take a snapshot of the journal iff there is no + * existing snapshot. The journal may or may not be empty, but + * we do not have any existing snapshots and we need to have one + * to serve as a restore point. The service MUST be joined with + * a met quorum in order to take a snapshot. + */ + + journal.getSnapshotManager().takeInitialSnapshot(); + // Block until this run state gets interrupted. blockInterruptably(); @@ -1687,10 +1725,17 @@ * Note: We MUST NOT install the local root blocks unless both * this service and the leader at at commitCounter ZERO(0L). */ + // Wait for the new root blocks. awaitJournalToken(token); { /* + * Get rid of any existing backups. They will not be + * consistent with the rebuild. + */ + deleteBackups(); + + /* * The current root block on the leader (We want to get some * immutatable metadata from the leader's root block). */ @@ -1723,6 +1768,12 @@ */ installRootBlocks(rbu.rootBlock0, rbu.rootBlock1); + // Note: Snapshot requires joined with met quorum. +// /* +// * Take a snapshot. +// */ +// journal.getSnapshotManager().takeInitialSnapshot(); + } /* @@ -1981,6 +2032,12 @@ openRootBlock.asRootBlock(true/* rootBlock0 */), openRootBlock.asRootBlock(false/* rootBlock0 */)); + // Note: snapshot requires joined with met quorum. +// /* +// * Take a snapshot. +// */ +// journal.getSnapshotManager().takeInitialSnapshot(); + } // Make sure we have the correct HALogWriter open. @@ -2923,20 +2980,20 @@ .getSnapshotManager().getRestorePolicy() .getEarliestRestorableCommitPoint(journal); - if (earliestRestorableCommitPoint == Long.MAX_VALUE) { + /* + * Release snapshots and HALog files no longer required by the + * restore policy. + * + * Note: The current HALog is NOT deleted. + */ - // Do not retain HALogs (other than the current one). - deleteHALogs(Long.MAX_VALUE); - - } else { + // Delete snapshots, returning commit counter of the oldest + // retained snapshot. + final long earliestRetainedSnapshotLastCommitCounter = deleteSnapshots(earliestRestorableCommitPoint); - // Release HALogs no longer required by the restore policy. - final long earliestRetainedSnapshotLastCommitCounter = deleteSnapshots(earliestRestorableCommitPoint); + // Delete HALogs not retained by that snapshot. + deleteHALogs(earliestRetainedSnapshotLastCommitCounter); - deleteHALogs(earliestRetainedSnapshotLastCommitCounter); - - } - } finally { logLock.unlock(); @@ -2946,10 +3003,38 @@ } /** + * We need to destroy the local backups if we do a REBUILD. Those files + * are no longer guaranteed to be consistent with the history of the + * journal. + */ + private void deleteBackups() { + + logLock.lock(); + + try { + + haLog.warn("Destroying local backups."); + + // Delete all snapshots. + deleteSnapshots(Long.MAX_VALUE); + + // Delete all HALogs (except the current one). + deleteHALogs(Long.MAX_VALUE); + + } finally { + + logLock.unlock(); + + } + + } + + /** * Delete snapshots that are no longer required. * <p> * Note: If ZERO (0) is passed into this method, then no snapshots will - * be deleted. + * be deleted. This is because the first possible commit counter is ONE + * (1). * * @param earliestRestorableCommitPoint * The earliest commit point that we need to be able to @@ -2962,6 +3047,8 @@ * List the snapshot files for this service. */ final File[] files; + // #of snapshot files found. Set during scan. + final AtomicLong nfound = new AtomicLong(); // Set to the commit counter of the earliest retained snapshot. final AtomicLong earliestRetainedSnapshotCommitCounter = new AtomicLong(Long.MAX_VALUE); final SnapshotManager snapshotManager = journal @@ -2995,6 +3082,9 @@ // Closing commitCounter for snapshot file. final long commitCounter = Long.parseLong(fileBaseName); + // Count all snapshot files. + nfound.incrementAndGet(); + if (commitCounter >= earliestRestorableCommitPoint) { /* * We need to retain this snapshot. @@ -3019,22 +3109,28 @@ int ndeleted = 0; long totalBytes = 0L; - if (files.length == 0) { + /* + * If people specify NoSnapshotPolicy then backup is in their hands. + * HALogs will not be retained beyond a fully met commit unless + * there is a snapshot against which they can be applied.. + */ + +// if (files.length == 0) { +// +// /* +// * Note: If there are no snapshots then we MUST retain ALL HALog +// * files. +// */ +// earliestRetainedSnapshotCommitCounter.set(0L); +// +// } else { - /* - * Note: If there are no snapshots then we MUST retain ALL HALog - * files. - */ - earliestRetainedSnapshotCommitCounter.set(0L); - - } else { - for (File file : files) { // #of bytes in that file. final long len = file.length(); - if (snapshotManager.removeSnapshot(file)) { + if (!snapshotManager.removeSnapshot(file)) { haLog.warn("COULD NOT DELETE FILE: " + file); @@ -3048,14 +3144,15 @@ } - } +// } - haLog.info("PURGED SNAPSHOTS: ndeleted=" + ndeleted - + ", totalBytes=" + totalBytes - + ", earliestRestorableCommitPoint=" - + earliestRestorableCommitPoint - + ", earliestRetainedSnapshotCommitCounter=" - + earliestRetainedSnapshotCommitCounter.get()); + if (haLog.isInfoEnabled()) + haLog.info("PURGED SNAPSHOTS: nfound=" + nfound + ", ndeleted=" + + ndeleted + ", totalBytes=" + totalBytes + + ", earliestRestorableCommitPoint=" + + earliestRestorableCommitPoint + + ", earliestRetainedSnapshotCommitCounter=" + + earliestRetainedSnapshotCommitCounter.get()); return earliestRetainedSnapshotCommitCounter.get(); @@ -3075,6 +3172,8 @@ * List the HALog files for this service. */ final File[] logFiles; + // #of HALog files found. Set during scan. + final AtomicLong nfound = new AtomicLong(); { final File currentLogFile = journal.getHALogWriter() @@ -3103,12 +3202,14 @@ return false; } + // track #of HALog files in the directory. + nfound.incrementAndGet(); + // filter out the current log file if (currentLogFile != null && name.equals(currentLogFileName)) { /* - * The caller requested that we NOT purge the - * current HALog, and this is it. + * This is the current HALog. We never purge it. */ return false; } @@ -3158,9 +3259,11 @@ } - haLog.info("PURGED LOGS: ndeleted=" + ndeleted + ", totalBytes=" - + totalBytes + ", earliestRetainedSnapshotCommitCounter=" - + earliestRetainedSnapshotCommitCounter); + if (haLog.isInfoEnabled()) + haLog.info("PURGED LOGS: nfound=" + nfound + ", ndeleted=" + + ndeleted + ", totalBytes=" + totalBytes + + ", earliestRetainedSnapshotCommitCounter=" + + earliestRetainedSnapshotCommitCounter); } @@ -3173,8 +3276,16 @@ } @Override + public void didMeet(final long token, final long commitCounter, + final boolean isLeader) { + // NOP + } + + @Override public File getServiceDir() { + return server.getServiceDir(); + } /** Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/IRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/IRestorePolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/IRestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -23,12 +23,21 @@ */ package com.bigdata.journal.jini.ha; - /** * The restore policy identifies the first commit point whose backups MUST NOT - * be released. The policy may be based on the age of the commit point, the - * number of intervening commit points, etc. A policy that always returns ZERO - * (0) will never release any backups. + * be released. Restore begins the most recent snapshot LTE the desired restore + * point and applies zero or more HALog files until the desired commit point has + * been restored. Therefore, the {@link IRestorePolicy} will pin the most recent + * snapshot LTE the computed earliest possible restore point and any HALog files + * for commit points GT that snapshot. A snapshot is taken automatically when + * the service first starts (and joins with a met quorum). This provides the + * initial full backup. Incremental logs are then retained until the next + * snapshot, at which point the {@link IRestorePolicy} may allow the older + * snapshot(s) and HALog files to be aged out. + * <p> + * The policy may be based on the age of the commit point, the number of + * intervening commit points, etc. A policy that always returns ZERO (0) will + * never release any backups. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> */ @@ -37,10 +46,29 @@ /** * Return the earlist commit point whose supporting backup files MUST NOT be * released in order to preseve the ability to restore that commit point - * from local backup files. + * from local backup files. The commit point is a commitCounter value. The + * first valid commit counter is ONE (1). A return value of ZERO (0) is + * therefore less than any possible commit point and will cause all commit + * points to be retained. The special value {@link Long#MAX_VALUE} may be + * used to indicate that NO commit points will be retained (it is greater + * than any possible commit counter). + * <p> + * Note: It is possible that new HALog files and new snapshots may appear + * concurrently during the evaluation of this method. This is unavoidable + * since both the leader and followers must evaluate this method in order to + * decide whether to release existing snapshot and HALog files. However, it + * is guaranteed that only one thread at a time on a given service will make + * the decision to release snapshots or HALog files. Since HALogs and + * snapshots may spring into existence concurrent with that decision, the + * {@link IRestorePolicy} can make decisions that retain slightly more + * information than would be required if the policy could have been + * evaluated atomicically with respect to the existing HALog files and + * snapshot files. This transient boundary condition can not result in fewer + * files being retained than are necessary to satisify the policy. * * @param jnl * The journal. + * * @return The earliest commit point whose supporting backup files must not * be released. */ Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -45,5 +45,11 @@ * Initialize the policy. */ void init(HAJournal jnl); + + /** + * Return a human readable summary of the policy. + */ + @Override + String toString(); } Deleted: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoRestorePolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoRestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -1,46 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2007. 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 -*/ -package com.bigdata.journal.jini.ha; - -/** - * A policy that does not preseve any backups. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - */ -public class NoRestorePolicy implements IRestorePolicy { - - /** - * {@inheritDoc} - * <p> - * This policy always returns {@link Long#MAX_VALUE} to prevent backups from - * being retained. - */ - @Override - public long getEarliestRestorableCommitPoint(HAJournal jnl) { - - return Long.MAX_VALUE; - - } - -} Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -34,5 +34,12 @@ public void init(HAJournal jnl) { // NOP } + + @Override + public String toString() { + return getClass().getSimpleName(); + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-03 13:43:25 UTC (rev 7034) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-04 21:34:01 UTC (rev 7035) @@ -57,6 +57,9 @@ import com.bigdata.btree.ITuple; import com.bigdata.btree.ITupleIterator; import com.bigdata.concurrent.FutureTaskMon; +import com.bigdata.concurrent.NamedLock; +import com.bigdata.ha.HAGlue; +import com.bigdata.ha.QuorumService; import com.bigdata.ha.halog.IHALogReader; import com.bigdata.ha.msg.HASnapshotResponse; import com.bigdata.ha.msg.IHASnapshotRequest; @@ -67,6 +70,7 @@ import com.bigdata.journal.ITx; import com.bigdata.journal.RootBlockUtility; import com.bigdata.journal.RootBlockView; +import com.bigdata.quorum.Quorum; import com.bigdata.quorum.QuorumException; import com.bigdata.rawstore.Bytes; import com.bigdata.util.ChecksumUtility; @@ -112,6 +116,11 @@ */ private final ISnapshotPolicy snapshotPolicy; +// /** +// * @see HAJournalServer.ConfigurationOptions#SNAPSHOT_ON_FIRST_MEET +// */ +// private final boolean snapshotOnFirstMeet; + /** * @see HAJournalServer.ConfigurationOptions#RESTORE_POLICY */ @@ -154,6 +163,18 @@ } +// /** +// * Return <code>true</code> iff the service will take a snapshot of the +// * empty journal when the service first joins with a met quorum. +// * +// * @see HAJournalServer.ConfigurationOptions#SNAPSHOT_ON_FIRST_MEET +// */ +// boolean isSnapshotOnFirstMeet() { +// +// return snapshotOnFirstMeet; +// +// } + /** * Return the {@link IRestorePolicy}. * @@ -216,6 +237,13 @@ ISnapshotPolicy.class,// HAJournalServer.ConfigurationOptions.DEFAULT_SNAPSHOT_POLICY); +// snapshotOnFirstMeet = !(snapshotPolicy instanceof NoSnapshotPolicy); +// snapshotOnFirstMeet = (Boolean) config.getEntry( +// HAJournalServer.ConfigurationOptions.COMPONENT, +// HAJournalServer.ConfigurationOptions.SNAPSHOT_ON_FIRST_MEET, +// Boolean.TYPE, +// HAJournalServer.ConfigurationOptions.DEFAULT_SNAPSHOT_ON_FIRST_MEET); + restorePolicy = (IRestorePolicy) config.getEntry( HAJournalServer.ConfigurationOptions.COMPONENT, HAJournalServer.ConfigurationOptions.RESTORE_POLICY, @@ -379,17 +407,17 @@ /* * Validate the snapshot. * - * TODO If the root blocks are bad, then this will throw an - * IOException and that will prevent the startup of the - * HAJournalServer. However, if we start up the server with a known - * bad snapshot *and* the snapshot is the earliest snapshot, then we - * can not restore commit points which depend on that earliest - * snapshot. + * TODO If the root blocks are bad, then this will throw an IOException + * and that will prevent the startup of the HAJournalServer. However, if + * we start up the ... [truncated message content] |
From: <tho...@us...> - 2013-04-05 18:25:22
|
Revision: 7036 http://bigdata.svn.sourceforge.net/bigdata/?rev=7036&view=rev Author: thompsonbry Date: 2013-04-05 18:25:08 +0000 (Fri, 05 Apr 2013) Log Message: ----------- Fix problems with test suite and code for IRestorePolicy test. @see https://sourceforge.net/apps/trac/bigdata/ticket/645 (HA Backup) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumServiceBase.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipeline.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -182,7 +182,13 @@ * service. * <p> * Note: The open HALog file is NOT removed by this method. + * + * @param token + * The quorum token. The quorum must remain fully met for this + * token across this operation. If that invariant is violated + * then the operation will terminate once this violation is + * noticed. */ - void purgeHALogs(); + void purgeHALogs(final long token); } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumServiceBase.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumServiceBase.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumServiceBase.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -149,9 +149,9 @@ } @Override - public void purgeHALogs() { + public void purgeHALogs(final long token) { - QuorumServiceBase.this.purgeHALogs(); + QuorumServiceBase.this.purgeHALogs(token); } @@ -273,7 +273,7 @@ * Note: The default implementation is a NOP. */ @Override - public void purgeHALogs() { + public void purgeHALogs(final long token) { // NOP Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -5980,18 +5980,14 @@ + rootBlock); } - if (quorum.isQuorumFullyMet(rootBlock.getQuorumToken())) { + /* + * The HA log files are purged on each node any time the + * quorum is fully met and goes through a commit point. + * Leaving only the current open log file. + */ - /* - * The HA log files are purged on each node any time - * the quorum is fully met and goes through a commit - * point. Leaving only the current open log file. - */ + localService.purgeHALogs(rootBlock.getQuorumToken()); - localService.purgeHALogs(); - - } - } catch(Throwable t) { haLog.error("ERROR IN 2-PHASE COMMIT: " + t Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -377,9 +377,9 @@ } @Override - public void purgeHALogs() { + public void purgeHALogs(final long token) { - MyMockQuorumMember.this.purgeHALogs(); + MyMockQuorumMember.this.purgeHALogs(token); } @@ -479,7 +479,7 @@ } @Override - public void purgeHALogs() { + public void purgeHALogs(final long token) { // NOP Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -168,15 +168,16 @@ } /** - * Find the first journal whose <em>createTime</em> is strictly greater - * than the timestamp. + * Return the {@link IRootBlockView} identifying the first snapshot whose + * <em>commitTime</em> is strictly greater than the timestamp. * * @param timestamp * The timestamp. A value of ZERO (0) may be used to find the - * first journal. + * first snapshot. * - * @return The commit record -or- <code>null</code> if there is no commit - * record whose timestamp is strictly greater than <i>timestamp</i>. + * @return The root block of that snapshot -or- <code>null</code> if there + * is no snapshot whose timestamp is strictly greater than + * <i>timestamp</i>. */ synchronized public IRootBlockView findNext(final long timestamp) { @@ -188,8 +189,8 @@ throw new IllegalArgumentException(); // find first strictly greater than. - final long index = findIndexOf(Math.abs(timestamp)) + 1; - + final long index = findIndexOf(timestamp) + 1; + if (index == nentries) { // No match. Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -34,17 +34,33 @@ * deciding when snapshots (aka full backups) and HALogs (aka write ahead log * files for each commit point aka incremental backups) may be purged. * <dl> - * <dt>minSnapshotAgeMillis</dt> - * <dd>The minimum age of a snapshot before it may be deleted.</dd> + * <dt>minRestoreAgeMillis</dt> + * <dd>The minimum restore period (in milliseconds). Snapshots and/or HALog + * files will be retained to ensure the ability to restore a commit point this + * far in the past.</dd> * <dt>minSnapshots</dt> * <dd>The minimum number of snapshot files (aka full backups) that must be - * retained.</dd> + * retained (positive integer). + * <p> + * This must be a positive integer. If the value were ZERO a snapshot would be + * purged as soon as it is taken. That would not provide an opportunity to make + * a copy of the snapshot, rendering the snapshot mechanism useless. + * <p> + * If <code>minSnapshots:=1</code> then a snapshot, once taken, will be retained + * until the next snapshot is taken. Further, the existence of the shapshot will + * cause HALog files for commit points GT that snapshot to accumulate until the + * next snapshot. This will occur regardless of the value of the other + * parameters. Thus, if you occasionally take snapshots and move them offsite, + * you must REMOVE the snapshot by hand in order to allow the retained HALogs to + * be reclaimed as well. + * <p> + * This concern does not arise if you are taking periodic snapshots.</dd> * <dt>minRestorePoints</dt> * <dd>The minimum number of commit points that must be restorable from backup. - * This explicitly controls the number of HALog files that will be retained. It - * also implicitly controls the number of snapshot files that will be retained - * since an HALog file will pin the newest snapshot whose commit counter is LTE - * to the the closing commit counter on that HALog file.</dd> + * This explicitly controls the number of HALog files that will be retained + * (non-negative). It also implicitly controls the number of snapshot files that + * will be retained since an HALog file will pin the newest snapshot whose + * commit counter is LTE to the the closing commit counter on that HALog file.</dd> * </dl> * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> @@ -75,7 +91,7 @@ /** * The minimum age of a snapshot before it may be purged. */ - private final long minSnapshotAgeMillis; + private final long minRestoreAgeMillis; /** * The minimum #of snapshots that must be retained. @@ -91,7 +107,7 @@ public String toString() { return DefaultRestorePolicy.class.getSimpleName()// - + "{minSnapshotAge=" + minSnapshotAgeMillis + "ms"// + + "{minRestoreAge=" + minRestoreAgeMillis + "ms"// + ",minSnapshots=" + minSnapshots // + ",minRestorePoints=" + minRestorePoints // + "}"; @@ -113,13 +129,13 @@ * policy will retain local backups unless all of the criteria are * satisified. * - * @param minSnapshotAgeMillis + * @param minRestoreAgeMillis * The minimum age of a snapshot (in milliseconds) before it may * be purged. */ - public DefaultRestorePolicy(final long minSnapshotAgeMillis) { + public DefaultRestorePolicy(final long minRestoreAgeMillis) { - this(minSnapshotAgeMillis, DEFAULT_MIN_SNAPSHOTS, + this(minRestoreAgeMillis, DEFAULT_MIN_SNAPSHOTS, DEFAULT_MIN_RESTORE_POINTS); } @@ -129,37 +145,72 @@ * policy will retain local backups unless all of the criteria are * satisified. * - * @param minSnapshotAgeMillis + * @param minRestoreAgeMillis * The minimum age of a snapshot (in milliseconds) before it may - * be purged. + * be purged (non-negative integer). * @param minSnapshots * The minimum number of snapshots (aka full backups) that must - * be retained locally. + * be retained locally (positive integer). + * <p> + * This must be a positive integer. If the value were ZERO a + * snapshot would be purged as soon as it is taken. That would + * not provide an opportunity to make a copy of the snapshot, + * rendering the snapshot mechanism useless. + * <p> + * If <code>minSnapshots:=1</code> then a snapshot, once taken, + * will be retained until the next snapshot is taken. Further, + * the existence of the shapshot will cause HALog files for + * commit points GT that snapshot to accumulate until the next + * snapshot. This will occur regardless of the value of the other + * parameters. Thus, if you occasionally take snapshots and move + * them offsite, you must REMOVE the snapshot by hand in order to + * allow the retained HALogs to be reclaimed as well. + * <p> + * This concern does not arise if you are taking periodic + * snapshots. * @param minRestorePoints * The minimum number of restore points (aka HALog files) that - * must be retained locally. If an HALog is pinned by this - * parameter, then the oldest snapshot LTE the commit counter of - * that HALog is also pinned, as are all HALog files GTE the - * snapshot and LT the HALog. + * must be retained locally (non-negative integer). If an HALog + * is pinned by this parameter, then the oldest snapshot LTE the + * commit counter of that HALog is also pinned, as are all HALog + * files GTE the snapshot and LT the HALog. */ - public DefaultRestorePolicy(final long minSnapshotAgeMillis, + public DefaultRestorePolicy(final long minRestoreAgeMillis, final int minSnapshots, final int minRestorePoints) { - if (minSnapshotAgeMillis < 0) + if (minRestoreAgeMillis < 0) throw new IllegalArgumentException( - "minSnapshotAgeMillis must be GTE ZERO (0), not " - + minSnapshotAgeMillis); + "minRestoreAgeMillis must be GTE ZERO (0), not " + + minRestoreAgeMillis); - if (minSnapshots < 1) + if (minSnapshots < 1) { + /* + * This must be a positive integer. If the value were ZERO a + * snapshot would be purged as soon as it is taken. That would not + * provide an opportunity to make a copy of the snapshot, rendering + * the snapshot mechanism useless. + * + * If minSnapshots:=1 then a snapshot, once taken, will be retained + * until the next snapshot is taken. Further, the existence of the + * shapshot will cause HALog files for commit points GT that + * snapshot to accumulate until the next snapshot. This will occur + * regardless of the value of the other parameters. Thus, if you + * occasionally take snapshots and move them offsite, you must + * REMOVE the snapshot by hand in order to allow the retained HALogs + * to be reclaimed as well. + * + * This concern does not arise if you are taking periodic snapshots. + */ throw new IllegalArgumentException( "minSnapshots must be GTE ONE (1), not " + minSnapshots); + } if (minRestorePoints < 0) throw new IllegalArgumentException( "minRestorePoints must be GTE ZERO (0), not " + minRestorePoints); - this.minSnapshotAgeMillis = minSnapshotAgeMillis; + this.minRestoreAgeMillis = minRestoreAgeMillis; this.minSnapshots = minSnapshots; @@ -175,19 +226,26 @@ private long getEarliestRestorableCommitCounterByAge(final HAJournal jnl, final long commitCounterOnJournal) { + // The current time. final long now = System.currentTimeMillis(); - final long then = now - minSnapshotAgeMillis; + // A moment [minRestoreAge] milliseconds ago. + final long then = now - minRestoreAgeMillis; + // The root block for the snapshot with a commitTime LTE [then]. final IRootBlockView rootBlock = jnl.getSnapshotManager().find(then); if (rootBlock == null) { - // There are no snapshots. + // There are no matching snapshots. return commitCounterOnJournal; } + if (log.isInfoEnabled()) + log.info("minRestoreAgeMillis=" + minRestoreAgeMillis + ", now=" + + now + ", then=" + then + ", rootBlock=" + rootBlock); + return rootBlock.getCommitCounter(); } @@ -200,14 +258,8 @@ private long getEarliestRestorableCommitCounterBySnapshots( final HAJournal jnl, final long commitCounterOnJournal) { - if (minSnapshots == 0) { - - return commitCounterOnJournal; - - } - final IRootBlockView rootBlock = jnl.getSnapshotManager() - .getSnapshotByReverseIndex(minSnapshots); + .getSnapshotByReverseIndex(minSnapshots - 1); if (rootBlock == null) { @@ -269,30 +321,45 @@ final long commitCounterOnJournal = jnl.getRootBlockView() .getCommitCounter(); - final long commitCounterByAge = getEarliestRestorableCommitCounterByAge( + final long commitCounterRetainedByAge = getEarliestRestorableCommitCounterByAge( jnl, commitCounterOnJournal); - final long commitCounterBySnapshots = getEarliestRestorableCommitCounterBySnapshots( + final long commitCounterRetainedBySnapshotCount = getEarliestRestorableCommitCounterBySnapshots( jnl, commitCounterOnJournal); - final long commitCounterByHALogs = getEarliestRestorableCommitCounterByHALogs( + final long commitCounterRetainedByHALogCount = getEarliestRestorableCommitCounterByHALogs( jnl, commitCounterOnJournal); - final long ret = Math.min(commitCounterByAge, - Math.min(commitCounterBySnapshots, commitCounterByHALogs)); + /* + * Take the minimum of those values. This is the commit counter that + * will be retained. + * + * Snapshot files and HALogs GTE this commit counter will not be + * released. + */ + final long commitCounterRetained = Math.min(commitCounterRetainedByAge, + Math.min(commitCounterRetainedBySnapshotCount, + commitCounterRetainedByHALogCount)); if (log.isInfoEnabled()) { - log.info("policy=" + this + ", commitCounterOnJournal=" - + commitCounterOnJournal + ", commitCounterByAge=" - + commitCounterByAge + ", commitCounterBySnapshots=" - + commitCounterBySnapshots + ", commitCounterByHALogs=" - + commitCounterByHALogs - + ", effectiveCommitCounterReported=" + ret); + log.info("policy=" + + this + + // + ", commitCounterOnJournal=" + + commitCounterOnJournal // + + ", commitCounterByAge=" + + commitCounterRetainedByAge // + + ", commitCounterBySnapshots=" + + commitCounterRetainedBySnapshotCount // + + ", commitCounterByHALogs=" + + commitCounterRetainedByHALogCount// + + ", effectiveCommitCounterRetained=" + + commitCounterRetained); } - return ret; + return commitCounterRetained; } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -505,7 +505,7 @@ public void deleteResources() { super.deleteResources(); - + recursiveDelete(getHALogDir(), new FileFilter() { @Override Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -77,6 +77,7 @@ import com.bigdata.ha.msg.IHALogRootBlocksResponse; import com.bigdata.ha.msg.IHARebuildRequest; import com.bigdata.ha.msg.IHASendStoreResponse; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateResponse; @@ -1256,7 +1257,7 @@ * services should then verify that the quorum is fully met * before they actually age out the HALogs and snapshots. */ - purgeHALogs(); + purgeHALogs(token); } @@ -1497,9 +1498,31 @@ * to serve as a restore point. The service MUST be joined with * a met quorum in order to take a snapshot. */ + { - journal.getSnapshotManager().takeInitialSnapshot(); + // Conditionally request initial snapshot. + final Future<IHASnapshotResponse> ft = journal + .getSnapshotManager().takeInitialSnapshot(); + if (ft != null) { + + /* + * Wait for outcome. + * + * Note: Even though we are blocking on the Future, the + * service is live and can receive writes. Once the + * Future is done, we are just going to block anyway in + * blockInterruptably(). + * + * Note: An exception thrown here will cause the service + * to transition into the error state. + */ + ft.get(); + + } + + } + // Block until this run state gets interrupted. blockInterruptably(); @@ -1730,12 +1753,6 @@ { /* - * Get rid of any existing backups. They will not be - * consistent with the rebuild. - */ - deleteBackups(); - - /* * The current root block on the leader (We want to get some * immutatable metadata from the leader's root block). */ @@ -1757,17 +1774,34 @@ // Verify that the quorum remains met on this token. getQuorum().assertQuorum(token); - + /* - * Install both root blocks. + * Critical section. * - * Note: This will take us through a local abort. That is - * important. We need to discard any writes that might have - * been buffered before we start the resynchronization of - * the local store. + * Up to now we have not modified anything on the disk. Now + * we are going to destroy the local data (both backups and + * the root blocks of the journal). */ - installRootBlocks(rbu.rootBlock0, rbu.rootBlock1); + { + /* + * Get rid of any existing backups. They will not be + * consistent with the rebuild. + */ + deleteBackups(); + + /* + * Install both root blocks. + * + * Note: This will take us through a local abort. That + * is important. We need to discard any writes that + * might have been buffered before we start the + * resynchronization of the local store. + */ + installRootBlocks(rbu.rootBlock0, rbu.rootBlock1); + + } + // Note: Snapshot requires joined with met quorum. // /* // * Take a snapshot. @@ -2969,12 +3003,22 @@ * Note: The current HALog file is NOT deleted by this method. */ @Override - public void purgeHALogs() { + public void purgeHALogs(final long token) { logLock.lock(); try { + if (!getQuorum().isQuorumFullyMet(token)) { + /* + * Halt operation. + * + * Note: This is not an error, but we can not remove + * snapshots or HALogs if this invariant is violated. + */ + return; + } + // We need to retain the backups for this commit point. final long earliestRestorableCommitPoint = journal .getSnapshotManager().getRestorePolicy() @@ -2989,10 +3033,11 @@ // Delete snapshots, returning commit counter of the oldest // retained snapshot. - final long earliestRetainedSnapshotLastCommitCounter = deleteSnapshots(earliestRestorableCommitPoint); + final long earliestRetainedSnapshotLastCommitCounter = deleteSnapshots( + token, earliestRestorableCommitPoint); // Delete HALogs not retained by that snapshot. - deleteHALogs(earliestRetainedSnapshotLastCommitCounter); + deleteHALogs(token, earliestRetainedSnapshotLastCommitCounter); } finally { @@ -3003,33 +3048,6 @@ } /** - * We need to destroy the local backups if we do a REBUILD. Those files - * are no longer guaranteed to be consistent with the history of the - * journal. - */ - private void deleteBackups() { - - logLock.lock(); - - try { - - haLog.warn("Destroying local backups."); - - // Delete all snapshots. - deleteSnapshots(Long.MAX_VALUE); - - // Delete all HALogs (except the current one). - deleteHALogs(Long.MAX_VALUE); - - } finally { - - logLock.unlock(); - - } - - } - - /** * Delete snapshots that are no longer required. * <p> * Note: If ZERO (0) is passed into this method, then no snapshots will @@ -3042,7 +3060,8 @@ * * @return The commitCounter of the earliest retained snapshot. */ - private long deleteSnapshots(final long earliestRestorableCommitPoint) { + private long deleteSnapshots(final long token, + final long earliestRestorableCommitPoint) { /* * List the snapshot files for this service. */ @@ -3076,30 +3095,43 @@ } // Strip off the filename extension. - final String fileBaseName = name.substring(0, - SnapshotManager.SNAPSHOT_EXT.length()); + final int len = name.length() + - SnapshotManager.SNAPSHOT_EXT.length(); + final String fileBaseName = name.substring(0, len); // Closing commitCounter for snapshot file. final long commitCounter = Long.parseLong(fileBaseName); // Count all snapshot files. nfound.incrementAndGet(); - - if (commitCounter >= earliestRestorableCommitPoint) { + + // true iff we will delete this snapshot. + final boolean deleteFile = commitCounter < earliestRestorableCommitPoint; + + if (haLog.isInfoEnabled()) + log.info("snapshotFile=" + + name// + + ", deleteFile=" + + deleteFile// + + ", commitCounter=" + + commitCounter// + + ", earliestRestoreableCommitPoint=" + + earliestRestorableCommitPoint); + + if (!deleteFile + && commitCounter < earliestRetainedSnapshotCommitCounter + .get()) { + /* - * We need to retain this snapshot. + * Update the earliest retained snapshot. */ - if (commitCounter < earliestRetainedSnapshotCommitCounter - .get()) { - // Update the earliest retained snapshot. - earliestRetainedSnapshotCommitCounter - .set(commitCounter); - } - return false; + + earliestRetainedSnapshotCommitCounter + .set(commitCounter); + } - // This snapshot MAY be deleted. - return true; + return deleteFile; } }); @@ -3130,6 +3162,16 @@ // #of bytes in that file. final long len = file.length(); + if (!getQuorum().isQuorumFullyMet(token)) { + /* + * Halt operation. + * + * Note: This is not an error, but we can not remove + * snapshots or HALogs if this invariant is violated. + */ + break; + } + if (!snapshotManager.removeSnapshot(file)) { haLog.warn("COULD NOT DELETE FILE: " + file); @@ -3167,7 +3209,8 @@ * that are GTE this commit counter since they will be * applied to that snapshot. */ - private void deleteHALogs(final long earliestRetainedSnapshotCommitCounter) { + private void deleteHALogs(final long token, + final long earliestRetainedSnapshotCommitCounter) { /* * List the HALog files for this service. */ @@ -3215,22 +3258,29 @@ } // Strip off the filename extension. - final String logFileBaseName = name.substring(0, - IHALogReader.HA_LOG_EXT.length()); + + final int len = name.length() + - IHALogReader.HA_LOG_EXT.length(); + + final String fileBaseName = name.substring(0, len); // Closing commitCounter for HALog file. final long logCommitCounter = Long - .parseLong(logFileBaseName); + .parseLong(fileBaseName); - if (logCommitCounter >= earliestRetainedSnapshotCommitCounter) { - /* - * We need to retain this log file. - */ - return false; - } + final boolean deleteFile = logCommitCounter < earliestRetainedSnapshotCommitCounter; + + if (haLog.isInfoEnabled()) + haLog.info("logFile=" + + name// + + ", delete=" + + deleteFile// + + ", logCommitCounter=" + + logCommitCounter// + + ", earliestRestoreableCommitPoint=" + + earliestRetainedSnapshotCommitCounter); - // This HALog file MAY be deleted. - return true; + return deleteFile; } }); @@ -3245,6 +3295,16 @@ // #of bytes in that HALog file. final long len = logFile.length(); + if (!getQuorum().isQuorumFullyMet(token)) { + /* + * Halt operation. + * + * Note: This is not an error, but we can not remove + * snapshots or HALogs if this invariant is violated. + */ + break; + } + if (!logFile.delete()) { haLog.warn("COULD NOT DELETE FILE: " + logFile); @@ -3267,6 +3327,91 @@ } + /** + * We need to destroy the local backups if we do a REBUILD. Those files + * are no longer guaranteed to be consistent with the history of the + * journal. + * <p> + * Note: This exists as a distinct code path because we will destroy + * those backups without regard to the quorum token. The normal code + * path requires a fully met journal in order to delete snapshots and + * HALog files. + * + * @throws IOException + * if a file could not be deleted. + */ + private void deleteBackups() throws IOException { + + logLock.lock(); + + try { + + haLog.warn("Destroying local backups."); + + // Delete all snapshots. + { + + final File snapshotDir = journal.getSnapshotManager() + .getSnapshotDir(); + + final File[] files = snapshotDir + .listFiles(new FilenameFilter() { + @Override + public boolean accept(final File dir, + final String name) { + return name + .endsWith(SnapshotManager.SNAPSHOT_EXT); + } + }); + for (File file : files) { + if (!file.delete()) + throw new IOException("COULD NOT DELETE FILE: " + + file); + } + + } + + // Delete all HALogs (except the current one). + { + + final File currentLogFile = journal.getHALogWriter() + .getFile(); + + final String currentLogFileName = currentLogFile == null ? null + : currentLogFile.getName(); + + final File logDir = journal.getHALogDir(); + + final File[] files = logDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(final File dir, final String name) { + // filter out the current log file + if (currentLogFile != null + && name.equals(currentLogFileName)) { + /* + * This is the current HALog. We never purge it. + */ + return false; + } + return name.endsWith(IHALogReader.HA_LOG_EXT); + } + }); + for (File file : files) { + if (!file.delete()) + throw new IOException("COULD NOT DELETE FILE: " + + file); + } + + } + + } finally { + + logLock.unlock(); + + } + + } + @Override public void installRootBlocks(final IRootBlockView rootBlock0, final IRootBlockView rootBlock1) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -512,8 +512,8 @@ } /** - * Return the {@link IRootBlock} identifying the snapshot having the largest - * lastCommitTime that is less than or equal to the given value. + * Return the {@link IRootBlockView} identifying the snapshot having the + * largest lastCommitTime that is less than or equal to the given value. * * @param timestamp * The given timestamp. @@ -532,6 +532,24 @@ } /** + * Return the {@link IRootBlockView} identifying the first snapshot whose + * <em>commitTime</em> is strictly greater than the timestamp. + * + * @param timestamp + * The timestamp. A value of ZERO (0) may be used to find the + * first snapshot. + * + * @return The root block of that snapshot -or- <code>null</code> if there + * is no snapshot whose timestamp is strictly greater than + * <i>timestamp</i>. + */ + public IRootBlockView findNext(final long timestamp) { + + return snapshotIndex.findNext(timestamp); + + } + + /** * Find the oldest snapshot that is at least <i>minRestorePoints</i> old and * returns its commit counter. * @@ -1148,7 +1166,7 @@ if (localService != null) { - localService.purgeHALogs(); + localService.purgeHALogs(token); } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -599,7 +599,11 @@ * @param lastCommitCounter * The last commit point to be verified (inclusive upper bound). * @param services - * The set of services whose HALog files will be tested. + * The set of services whose HALog files will be tested. If there + * is more than one service, then this method will verify that + * the services have the same digests for their HALog files. If + * there is only one service, then this will verify that the + * HALog file exists by computing its digest. * * @throws IOException * @throws DigestException @@ -762,17 +766,20 @@ * @param expected * The expected commit point. * @param haGlue - * The remote server interface. + * The remote server interface(s). * * @throws IOException */ - protected void assertCommitCounter(final long expected, final HAGlue haGlue) + protected void assertCommitCounter(final long expected, final HAGlue... haGlue) throws IOException { - assertEquals( - expected, - haGlue.getRootBlock(new HARootBlockRequest(null/* storeUUID */)) - .getRootBlock().getCommitCounter()); + for (HAGlue server : haGlue) { + assertEquals( + expected, + server.getRootBlock( + new HARootBlockRequest(null/* storeUUID */)) + .getRootBlock().getCommitCounter()); + } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -119,9 +119,15 @@ final int N = 7; // #of transactions to run before the snapshot. final int M = 8; // #of transactions to run after the snapshot. - // Start 2 services. + /* + * Start 3 services. + * + * Note: We need to have three services running in order for the + * snapshots + */ final HAGlue serverA = startA(); final HAGlue serverB = startB(); + final HAGlue serverC = startC(); // Wait for a quorum meet. final long token = quorum.awaitQuorum(awaitQuorumTimeout, @@ -134,9 +140,15 @@ assertEquals(serverA, quorum.getClient().getLeader(token)); // Await initial commit point (KB create). - awaitCommitCounter(1L, serverA, serverB); + awaitCommitCounter(1L, serverA, serverB, serverC); - assertCommitCounter(1L, serverA); + /* + * There should not be any snapshots yet since we are using the + * NoSnapshotPolicy. + */ + assertEquals(0, getSnapshotDirA().list().length); + assertEquals(0, getSnapshotDirB().list().length); + assertEquals(0, getSnapshotDirC().list().length); // Now run N transactions. for (int i = 0; i < N; i++) { @@ -147,11 +159,12 @@ final long commitCounterN = N + 1; - assertCommitCounter(commitCounterN, serverA); + assertCommitCounter(commitCounterN, serverA, serverB, serverC); - // Check HALogs equal on A, B. - assertHALogDigestsEquals(1L/* firstCommitCounter */, commitCounterN, - new HAGlue[] { serverA, serverB }); + // Only the live log is retained on the services. + assertEquals(1, getHALogDirA().list().length); + assertEquals(1, getHALogDirA().list().length); + assertEquals(1, getHALogDirA().list().length); /* * Take a snapshot. @@ -208,8 +221,8 @@ /* * Now run sets of M transactions until we have exceeded releasePolicy's - * minimum age for the existing snapshot. Since there is only one - * snapshot, it SHOULD NOT be removed. + * minimum age for the existing snapshot. However, since there is only + * one snapshot, it SHOULD NOT be removed. */ int nnewtx = 0; { @@ -228,15 +241,26 @@ final long commitCounterM = nnewtx + N + 1; - assertCommitCounter(commitCounterM, serverA); - - // Check HALogs equal on A, B. - assertHALogDigestsEquals(1L/* firstCommitCounter */, commitCounterM, - new HAGlue[] { serverA, serverB }); + assertCommitCounter(commitCounterM, serverA, serverB, serverC); - // Snapshot directory contains just the expected snapshot + /* + * Verify that the snapshot directory contains just the expected + * snapshot. + */ assertEquals(new String[] { snapshotFile0.getName() }, getSnapshotDirA().list()); + + /* + * Check HALogs for existence on A. + * + * Note: We can not compare the digests for equality on the + * difference servers because we only took a snapshot on A and + * therefore we have not pinned the HALogs on B or C. + */ + assertHALogDigestsEquals( + commitCounterN + 1/* firstCommitCounter */, + commitCounterM, new HAGlue[] { serverA }); + } // Verify snapshot still exists. @@ -251,10 +275,15 @@ /* * Request another snapshot. * - * Note: We will now have 2 snapshots. The original snapshot is purged - * since it is older than the minimum retention time for a snapshot. + * Note: We will now have 2 snapshots. The original snapshot is NOT + * purged. While it is older than the minimum retention time for a + * snapshot, we do not yet have another snapshot that will allow us to + * recover the commit points GT our oldest snapshot that are within the + * required recovered period. */ final IRootBlockView snapshotRB1; + final File snapshotFile1; + final long lastCommitCounter; { // Request snapshot on A. @@ -275,34 +304,111 @@ snapshotRB1 = ft.get().getRootBlock(); // The name of the new snapshot file. - final File snapshotFile1 = SnapshotManager.getSnapshotFile( - getSnapshotDirA(), snapshotRB1.getCommitCounter()); + snapshotFile1 = SnapshotManager.getSnapshotFile(getSnapshotDirA(), + snapshotRB1.getCommitCounter()); // Verify new snapshot exists. assertTrue(snapshotFile1.exists()); - // Verify old snapshot is gone. - assertFalse(snapshotFile0.exists()); + // Verify old snapshot exists. + assertTrue(snapshotFile0.exists()); - // Verify snapshot directory contains the only the one file. - assertEquals(new String[] { snapshotFile1.getName() }, - getSnapshotDirA().list()); + // Verify snapshot directory contains the necessary files. + assertEquals( + new String[] { snapshotFile0.getName(), + snapshotFile1.getName() }, getSnapshotDirA().list()); - /* - * Verify only the expected HALog files are retained. - */ - // The current commit counter on A. - final long lastCommitCounter = serverA + lastCommitCounter = serverA .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) .getRootBlock().getCommitCounter(); - // Check HALogs equal on A, B. + // Check HALogs found on A. assertHALogDigestsEquals( + commitCounterN + 1/* firstCommitCounter */, + lastCommitCounter/* lastCommitCounter */, + new HAGlue[] { serverA }); + + /* + * Verify only the expected HALog files are retained on A (in fact, + * all HALogs will still be retained on B since we are not taking + * any snapshots there). + */ + assertHALogNotFound(1L/* firstCommitCounter */, + commitCounterN - 1/* lastCommitCounter */, + new HAGlue[] { serverA }); + + } + + /* + * Sleep until the most recent snapshot is old enough to satisify our + * recovery period. + */ + Thread.sleep(restorePolicyMinSnapshotAgeMillis); + + /* + * The older snapshot should still exist since we have not gone through + * a commit. + */ + { + + // Verify new snapshot exists. + assertTrue(snapshotFile1.exists()); + + // Verify old snapshot exists. + assertTrue(snapshotFile0.exists()); + + // Verify snapshot directory contains the necessary files. + assertEquals( + new String[] { snapshotFile0.getName(), + snapshotFile1.getName() }, getSnapshotDirA().list()); + + // Check HALogs found on A. + assertHALogDigestsEquals( + commitCounterN + 1/* firstCommitCounter */, + lastCommitCounter/* lastCommitCounter */, + new HAGlue[] { serverA }); + + } + + // Do a simple transaction. + simpleTransaction(); + + final long lastCommitCounter2 = lastCommitCounter + 1; + + // Verify the current commit counter on A, B. + assertCommitCounter(lastCommitCounter2, new HAGlue[] { serverA, + serverB, serverC }); + + /* + * Verify older snapshot and logs LT the newer snapshot are gone. + */ + { + + // Verify new snapshot exists. + assertTrue(snapshotFile1.exists()); + + // Verify old snapshot is done. + assertFalse(snapshotFile0.exists()); + + // Verify snapshot directory contains the necessary files. + assertEquals(new String[] { snapshotFile1.getName() }, + getSnapshotDirA().list()); + + // Check HALogs found on A. + assertHALogDigestsEquals( snapshotRB1.getCommitCounter()/* firstCommitCounter */, - lastCommitCounter/* lastCommitCounter */, new HAGlue[] { - serverA, serverB }); + lastCommitCounter2/* lastCommitCounter */, + new HAGlue[] { serverA }); + /* + * Verify HALogs were removed from A (again, all HALogs will still + * be on B since we have not taken a snapshot there). + */ + assertHALogNotFound(1L/* firstCommitCounter */, + snapshotRB1.getCommitCounter() - 1/* lastCommitCounter */, + new HAGlue[] { serverA }); + } } Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-04 21:34:01 UTC (rev 7035) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-05 18:25:08 UTC (rev 7036) @@ -51,8 +51,6 @@ import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.RootBlockView; import com.bigdata.journal.jini.ha.HAJournal; -import com.bigdata.journal.jini.ha.IRestorePolicy; -import com.bigdata.journal.jini.ha.ISnapshotPolicy; import com.bigdata.journal.jini.ha.SnapshotManager; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; @@ -172,15 +170,15 @@ + journal.getSnapshotManager().getRestorePolicy()) .node("br").close(); - if(true) { - /* - * FIXME HABackup: disable this code block. It is for - * debug purposes only. - */ - p.text("Service: getEarliestRestorableCommitPoint()=" - + journal.getSnapshotManager().getRestorePolicy().getEarliestRestorableCommitPoint(journal)) - .node("br").close(); - } +// if(true) { +// /* +// * HABackup: disable this code block. It is for +// * debug purposes only. +// */ +// p.text("Service: getEarliestRestorableCommitPoint()=" +// + journal.getSnapshotManager().getRestorePolicy().getEarliestRestorableCommitPoint(journal)) +// .node("br").close(); +// } } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-05 21:16:09
|
Revision: 7037 http://bigdata.svn.sourceforge.net/bigdata/?rev=7037&view=rev Author: thompsonbry Date: 2013-04-05 21:15:56 +0000 (Fri, 05 Apr 2013) Log Message: ----------- Modified AbstractJournal.setQuorumToken() to handle the case where a service joins a met quorum. Modified HAJournalServer to call setQuorumToken() when joining a met quorum (after it calls serviceJoin()). Modified HAJournalServer to take a snapshot on entering RunMet if there are no snapshots and the ISnapshotPolicy != NoSnapshotPolicy. Implemented the tests described above. All HA CI tests are green except for the "join during live load". MC is looking at those tests now. @see https://sourceforge.net/apps/trac/bigdata/ticket/645 (HA Backup) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Added Paths: ----------- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -230,7 +230,10 @@ * in progress, then the {@link Future} for that request will be returned. * * @param req - * The request. + * The request (optional). When <code>null</code>, the + * {@link Future} for any existing snapshot operation will be + * returned but the request WILL NOT schedule a snapshot if none + * is running. * * @return A {@link Future} for the snapshot -or- <code>null</code> if no * snapshot is running and none will be taken for that request. Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -4858,15 +4858,19 @@ if (haLog.isInfoEnabled()) haLog.info("oldValue=" + oldValue + ", newToken=" + newValue); - if (oldValue == newValue) { + if (oldValue == newValue && oldValue == haReadyToken) { // No change. return; } - + + // This quorum member. + final QuorumService<HAGlue> localService = quorum.getClient(); + final boolean didBreak; final boolean didMeet; + final boolean didJoinMetQuorum; if (newValue == Quorum.NO_QUORUM && oldValue != Quorum.NO_QUORUM) { @@ -4880,6 +4884,7 @@ didBreak = true; didMeet = false; + didJoinMetQuorum = false; } else if (newValue != Quorum.NO_QUORUM && oldValue == Quorum.NO_QUORUM) { @@ -4891,23 +4896,28 @@ didBreak = false; didMeet = true; + didJoinMetQuorum = false; - } else { + } else if (newValue != Quorum.NO_QUORUM + && haReadyToken == Quorum.NO_QUORUM && localService != null + && localService.isJoinedMember(newValue)) { /* - * Excluded middle. If there was no change, then we returned - * immediately up above. If there is a change, then it must be - * either a quorum break or a quorum meet, which were identified in - * the if-then-else above. + * This service is joining a quorum that is already met. */ + + didBreak = false; + didMeet = false; + didJoinMetQuorum = true; + + } else { - throw new AssertionError(); + didBreak = false; + didMeet = false; + didJoinMetQuorum = false; } - // This quorum member. - final QuorumService<HAGlue> localService = quorum.getClient(); - /* * Both a meet and a break require an exclusive write lock. */ @@ -4951,7 +4961,7 @@ haReadyCondition.signalAll(); // signal ALL. - } else if (didMeet) { + } else if (didMeet || didJoinMetQuorum) { final long tmp; Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -94,8 +94,8 @@ if (timeOfDay < 0) throw new IllegalArgumentException(); - if (percentLogSize < 10 || percentLogSize > 400) - throw new IllegalArgumentException("percentage must be in [10:400]"); + if (percentLogSize < 0 || percentLogSize > 400) + throw new IllegalArgumentException("percentage must be in [0:400]"); this.timeOfDay = timeOfDay; Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -1488,7 +1488,7 @@ */ throw new InterruptedException(); } - + } // validation of pre-conditions. /* @@ -1499,7 +1499,14 @@ * a met quorum in order to take a snapshot. */ { + + // Await the initial KB create commit point. + while (journal.getRootBlockView().getCommitCounter() < 1) { + Thread.sleep(100/* ms */); + + } + // Conditionally request initial snapshot. final Future<IHASnapshotResponse> ft = journal .getSnapshotManager().takeInitialSnapshot(); @@ -2388,6 +2395,9 @@ // Verify that the quorum is valid. getQuorum().assertQuorum(token); + // Set the token on the journal. + journal.setQuorumToken(token); + haLog.warn("Joined met quorum: runState=" + runStateRef + ", commitCounter=" + openingCommitCounter + ", lastCommitTimeOfLeader=" @@ -2818,6 +2828,9 @@ */ getActor().serviceJoin(); + // Set the token on the journal. + journal.setQuorumToken(token); + // Transition to RunMet. enterRunState(new RunMetTask(token, leaderId)); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -709,15 +709,11 @@ * @return The {@link Future} if a snapshot is already being made -or- if a * snapshot was started by the request and <code>null</code> if no * snapshot will be taken in response to this request. - * - * @throws Exception - * @throws ExecutionException - * @throws InterruptedException */ public Future<IHASnapshotResponse> takeSnapshot(final IHASnapshotRequest req) { - if (req == null) - throw new IllegalArgumentException(); +// if (req == null) +// throw new IllegalArgumentException(); lock.lock(); @@ -736,6 +732,13 @@ } + if (req == null) { + + /* Not running. No snapshot was scheduled (null request). */ + return null; + + } + if (!isReadyToSnapshot(req)) { // Pre-conditions are not met. Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -68,6 +68,8 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.RunState; import com.bigdata.ha.msg.HARootBlockRequest; +import com.bigdata.ha.msg.HASnapshotDigestRequest; +import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.jini.start.IServiceListener; import com.bigdata.jini.start.config.JavaServiceConfiguration; import com.bigdata.jini.start.config.ServiceConfiguration; @@ -76,6 +78,7 @@ import com.bigdata.jini.start.process.ProcessHelper; import com.bigdata.jini.util.ConfigMath; import com.bigdata.jini.util.JiniUtil; +import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.jini.ha.HAJournalServer.ConfigurationOptions; import com.bigdata.quorum.AbstractQuorumClient; import com.bigdata.quorum.AsynchronousQuorumCloseException; @@ -2198,4 +2201,154 @@ } + /** + * Remove files in the directory, except the "open" log file. + * + * @param dir + * The HALog directory. + * @param openFile + * The name of the open log file. + */ + protected void removeFiles(final File dir, final String openFile) { + final File[] files = dir.listFiles(); + if (files != null) + for (File file: files) { + if (!file.getName().equals(openFile)) { + log.warn("removing file " + file.getName()); + + file.delete(); + } + } + } + + protected void copyFiles(File src, File dst) throws IOException { + final File[] files = src.listFiles(); + log.warn("Copying " + src.getAbsolutePath() + " to " + dst.getAbsolutePath() + ", files: " + files.length); + if (files != null) { + for (File srcFile: files) { + final File dstFile = new File(dst, srcFile.getName()); + log.info("Copying " + srcFile.getAbsolutePath() + " to " + dstFile.getAbsolutePath()); + final FileInputStream instr = new FileInputStream(srcFile); + final FileOutputStream outstr = new FileOutputStream(dstFile); + + final byte[] buf = new byte[8192]; + while (true) { + final int len = instr.read(buf); + if (len == -1) + break; + + outstr.write(buf, 0, len); + } + + outstr.close(); + instr.close(); + } + } + } + + + /** + * Wait the service self-reports "RunMet". + */ + protected void awaitRunMet(final HAGlue haGlue) throws Exception { + // Wait until HA and NSS are ready. + awaitNSSAndHAReady(haGlue); + // Wait until self-reports RunMet. + assertCondition(new Runnable() { + public void run() { + try { + final String extendedRunState = haGlue.getExtendedRunState(); + if (!extendedRunState.contains("RunMet")) { + fail("Expecting RunMet, not " + extendedRunState); + } + } catch (Exception e) { + fail(); + } + } + + }, 5, TimeUnit.SECONDS); + } + + /** + * Await the specified snapshot. + * + * @param server + * The service. + * @param commitCounter + * The commitCounter for the snapshot. + */ + protected void awaitSnapshotExists(final HAGlue server, + final long commitCounter) throws Exception { + + awaitRunMet(server); + + // Wait until self-reports RunMet. + assertCondition(new Runnable() { + public void run() { + try { + + /* + * Check the server for an active Future for a snapshot. If + * we find one, then just wait for that Future. + * + * Note: This WILL NOT schedule a snapshot. It just gets the + * current Future (if any). + * + * Note: If the snapshot is already done, then this will + * return [null]. So we can not tell the difference using + * this method between a snapshot that has not been + * requested yet and a snapshot that is already done. If the + * Future is null we will request the snapshot digest + * (below) to figure out if the snapshot already exists. + */ + { + final Future<IHASnapshotResponse> ftA = server.takeSnapshot(null/* GET */); + + if (ftA != null) { + + final IRootBlockView snapshotRB = ftA.get() + .getRootBlock(); + + if (snapshotRB.getCommitCounter() == commitCounter) { + + // This is the snapshot that we were waiting + // for. + return; + + } + + } + + } + + try { + + // If we can get the digest then the snapshot exists. + server.computeHASnapshotDigest(new HASnapshotDigestRequest( + commitCounter)); + + // Found it. + return; + + } catch (Exception ex) { + + if (!InnerCause.isInnerCause(ex, + FileNotFoundException.class)) { + + log.error("Not expecting: " + ex, ex); + + fail("Not expecting: " + ex.getMessage(), ex); + + } + } + + } catch (Exception e) { + fail(); + } + } + + }, 10, TimeUnit.SECONDS); + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -762,6 +762,9 @@ /** * Assert that the remote server is at the specified commit point. + * <p> + * This method DOES NOT WAIT. It will fail unless the service is already at + * the specified commit point. * * @param expected * The expected commit point. @@ -769,7 +772,10 @@ * The remote server interface(s). * * @throws IOException + * + * @deprecated by {@link #awaitCommitCounter(long, HAGlue...)}. */ + @Deprecated protected void assertCommitCounter(final long expected, final HAGlue... haGlue) throws IOException { Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -92,6 +92,7 @@ // HA3 snapshot policy test suite. suite.addTestSuite(TestHA3SnapshotPolicy.class); + suite.addTestSuite(TestHA3SnapshotPolicy2.class); // HA3 restore policy test suite. suite.addTestSuite(TestHA3RestorePolicy.class); Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -27,9 +27,6 @@ package com.bigdata.journal.jini.ha; import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; @@ -41,6 +38,7 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; +import com.bigdata.journal.AbstractJournal; import com.bigdata.quorum.Quorum; import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; @@ -59,6 +57,15 @@ * corresponds to a zookeeper timeout, e.g., as might occur during a full GC * pause. * + * TODO Update the existing tests to verify that the quorum token is properly + * set on C when C resyncs with A+B and that + * {@link AbstractJournal#getHAReady()} reports the correct token. This tests + * for a problem where we did not call setQuorumToken() again when we resync and + * transition into the met quorum. This meant that the HAReady token is not set + * for a service unless it is part of the initial quorum meet. One of the HA3 + * backup tests covers this, but we should be checking the HAReadyToken in this + * test suite as well. + * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> */ public class TestHA3JournalServer extends AbstractHA3JournalServerTestCase { @@ -802,16 +809,6 @@ final long token = quorum.awaitQuorum(awaitQuorumTimeout, TimeUnit.MILLISECONDS); -// // Verify KB exists. -// awaitKBExists(serverA); - - /* - * Note: The quorum was not fully met at the last 2-phase commit. - * Instead, 2 services participated in the 2-phase commit and the third - * service resynchronized when it came up and then went through a local - * commit. Therefore, the HALog files should exist on all nodes. - */ - // Current commit point. final long lastCommitCounter = 1; @@ -849,9 +846,8 @@ // now remove the halog files ensuring a RESYNC is not possible // but MUST leave currently open file!! final String openLog = HALogWriter.getHALogFileName(lastCommitCounter2 + 1); - final File serviceDir = new File("benchmark/CI-HAJournal-1"); - removeFiles(new File(serviceDir, "A/HALog"), openLog); - removeFiles(new File(serviceDir, "B/HALog"), openLog); + removeFiles(getHALogDirA(), openLog); + removeFiles(getHALogDirB(), openLog); // Now Start 3rd service. final HAGlue serverC = startC(); @@ -880,45 +876,8 @@ assertHALogNotFound(0L/* firstCommitCounter */, lastCommitCounter3, new HAGlue[] { serverA, serverB, serverC }); - } + } - private void removeFiles(final File dir, final String openFile) { - final File[] files = dir.listFiles(); - if (files != null) - for (File file: files) { - if (!file.getName().equals(openFile)) { - log.warn("removing file " + file.getName()); - - file.delete(); - } - } - } - - private void copyFiles(File src, File dst) throws IOException { - final File[] files = src.listFiles(); - log.warn("Copying " + src.getAbsolutePath() + " to " + dst.getAbsolutePath() + ", files: " + files.length); - if (files != null) { - for (File srcFile: files) { - final File dstFile = new File(dst, srcFile.getName()); - log.info("Copying " + srcFile.getAbsolutePath() + " to " + dstFile.getAbsolutePath()); - final FileInputStream instr = new FileInputStream(srcFile); - final FileOutputStream outstr = new FileOutputStream(dstFile); - - final byte[] buf = new byte[8192]; - while (true) { - final int len = instr.read(buf); - if (len == -1) - break; - - outstr.write(buf, 0, len); - } - - outstr.close(); - instr.close(); - } - } - } - /** * Test Rebuild of early starting C service where quorum was previously * fully met. This forces a pipeline re-organisation Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -159,7 +159,7 @@ final long commitCounterN = N + 1; - assertCommitCounter(commitCounterN, serverA, serverB, serverC); + awaitCommitCounter(commitCounterN, serverA, serverB, serverC); // Only the live log is retained on the services. assertEquals(1, getHALogDirA().list().length); @@ -241,7 +241,7 @@ final long commitCounterM = nnewtx + N + 1; - assertCommitCounter(commitCounterM, serverA, serverB, serverC); + awaitCommitCounter(commitCounterM, serverA, serverB, serverC); /* * Verify that the snapshot directory contains just the expected @@ -377,7 +377,7 @@ final long lastCommitCounter2 = lastCommitCounter + 1; // Verify the current commit counter on A, B. - assertCommitCounter(lastCommitCounter2, new HAGlue[] { serverA, + awaitCommitCounter(lastCommitCounter2, new HAGlue[] { serverA, serverB, serverC }); /* Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -584,8 +584,6 @@ // Await initial commit point (KB create). awaitCommitCounter(1L, serverA, serverB); - assertCommitCounter(1L, serverA); - // Now run N transactions. for (int i = 0; i < N; i++) { @@ -595,7 +593,7 @@ final long commitCounterN = N + 1; - assertCommitCounter(commitCounterN, serverA); + awaitCommitCounter(commitCounterN, serverA); // Check HALogs equal on A, B. assertHALogDigestsEquals(1L/* firstCommitCounter */, commitCounterN, @@ -649,7 +647,7 @@ final long commitCounterM = M + N + 1; - assertCommitCounter(commitCounterM, serverA); + awaitCommitCounter(commitCounterM, serverA); // Check HALogs equal on A, B. assertHALogDigestsEquals(1L/* firstCommitCounter */, commitCounterM, Added: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java (rev 0) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -0,0 +1,342 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 Oct 31, 2012 + */ +package com.bigdata.journal.jini.ha; + +import java.util.Calendar; +import java.util.concurrent.TimeUnit; + +import net.jini.config.Configuration; + +import com.bigdata.ha.HAGlue; +import com.bigdata.ha.halog.HALogWriter; +import com.bigdata.ha.msg.HARootBlockRequest; + +/** + * Test suites for the {@link DefaultSnapshotPolicy}. + * <p> + * Note: Since the {@link DefaultSnapshotPolicy} runs once a day, we have to + * configure the time at which it will run so that it will not run during the + * unit test. If it were to run during the test, then that would throw off the + * conditions that we are trying to control from the test suite. + * <p> + * Note: The {@link NoSnapshotPolicy} is tested for explictly in the code in + * order to avoid taking the initial snapshop and thereby pinning the HALogs. + * This test suite uses the {@link DefaultSnapshotPolicy} and verifies that the + * initial snapshot IS taken when the {@link DefaultSnapshotPolicy} is used. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * + * @see TestHA3SnapshotPolicy + * + * TODO Write test to verify that a rebuild triggered while a snapshot is + * running will cancel the running snapshot. This is necessary since we + * must overwrite the root blocks. + */ +public class TestHA3SnapshotPolicy2 extends AbstractHA3BackupTestCase { + + public TestHA3SnapshotPolicy2() { + } + + public TestHA3SnapshotPolicy2(String name) { + super(name); + } + + /** How long to wait for snapshots to appear. */ + private final long awaitSnapshotMillis = 5000; + + /** + * {@inheritDoc} + * <p> + * Note: This overrides some {@link Configuration} values for the + * {@link HAJournalServer} in order to establish conditions suitable for + * testing the {@link ISnapshotPolicy} and {@link IRestorePolicy}. + */ + @Override + protected String[] getOverrides() { + + /* + * We need to set the time at which the DefaultSnapshotPolicy runs to + * some point in the Future in order to avoid test failures due to + * violated assumptions when the policy runs up self-triggering (based + * on the specified run time) during a CI run. + * + * We do this by adding one hour to [now] and then converting it into + * the 'hhmm' format as an integer. + */ + + // Right now. + final Calendar c = Calendar.getInstance(); + + // Plus an hour. + c.add(Calendar.HOUR_OF_DAY, 1); + + // Get the hour. + final int hh = c.get(Calendar.HOUR_OF_DAY); + + // And the minutes. + final int mm = c.get(Calendar.MINUTE); + + // Format as hhmm. + final String neverRun = "" + hh + (mm < 10 ? "0" : "") + mm; + +// assert neverRun.length() < 4 && neverRun.length() >= 3 : neverRun; + + return new String[]{ + "com.bigdata.journal.jini.ha.HAJournalServer.restorePolicy=new com.bigdata.journal.jini.ha.DefaultRestorePolicy(0L,1,0)", + "com.bigdata.journal.jini.ha.HAJournalServer.snapshotPolicy=new com.bigdata.journal.jini.ha.DefaultSnapshotPolicy("+neverRun+",0)" + }; + + } + + /** + * Verify that A+B take a snapshot of the empty journal once the quorum + * meets (but not before). + */ + public void test_AB_snapshotOnQuorumMeet() throws Exception { + + // Start A. Service should NOT take a snapshot. + final HAGlue serverA = startA(); + + // Snapshot directory is empty. + assertEquals(0, getSnapshotDirA().list().length); + + // Wait a bit. + Thread.sleep(awaitSnapshotMillis/*ms*/); + + // Snapshot directory is still empty. + assertEquals(0, getSnapshotDirA().list().length); + + // Start B. Both A and B should take a snapshot when quorum meets. + final HAGlue serverB = startB(); + + // Await quorum meet. + final long token = awaitMetQuorum(); + + // Wait until both services are ready. + awaitNSSAndHAReady(serverA); + awaitNSSAndHAReady(serverB); + + // Expected commit point on A, B. + final long commitCounter = 1; + + // Verify/await expected commit point. + awaitCommitCounter(commitCounter, serverA, serverB); + + // Verify/await snapshot on A. + awaitSnapshotExists(serverA, commitCounter); + + // Verify/await snapshot on B. + awaitSnapshotExists(serverB, commitCounter); + + } + + /** + * Verify that C snapshots the journal when it enters RunMet after + * resynchronizing from A+B. (This can just be start A+B, await quorum meet, + * then start C. C will resync from the leader. The snapshot should be taken + * when resync is done and we enter RunMet.) + */ + public void test_AB_snapshotOnQuorumMeet_C_snapshotOnResync() + throws Exception { + + // Start A, B. + final HAGlue serverA = startA(); + final HAGlue serverB = startB(); + + // Await quorum meet. + final long token = awaitMetQuorum(); + + // Wait until both services are ready. + awaitNSSAndHAReady(serverA); + awaitNSSAndHAReady(serverB); + + // Expected commit point on A, B. + final long commitCounter = 1; + + // Verify/await expected commit point. + awaitCommitCounter(commitCounter, serverA, serverB); + + // Verify/await snapshot on A. + awaitSnapshotExists(serverA, commitCounter); + + // Verify/await snapshot on B. + awaitSnapshotExists(serverB, commitCounter); + + // Start C. It will resync and join. + final HAGlue serverC = startC(); + + // Await fully quorum meet. + final long token2 = awaitFullyMetQuorum(); + + // Token is unchanged. + assertEquals(token, token2); + + // Wait until service is ready. + awaitNSSAndHAReady(serverC); + + // Verify that C took a snapshot. + awaitSnapshotExists(serverC, commitCounter); + + } + + /** + * Test is a variant on the above that verifies that the snapshot taken by C + * is once it enters RunMet. It verifies this by issuing a update request + * against (A+B) before starting C. This means that the snapshot on C will + * be at commitCounter:=2 rather than commitCounter:=1. + */ + public void test_AB_snapshotOnQuorumMeet_C_snapshotOnResync2() + throws Exception { + + // Start A, B. + final HAGlue serverA = startA(); + final HAGlue serverB = startB(); + + // Await quorum meet. + final long token = awaitMetQuorum(); + + // Wait until both services are ready. + awaitNSSAndHAReady(serverA); + awaitNSSAndHAReady(serverB); + + // Expected commit point on A, B. + final long commitCounter = 1; + + // Verify/await expected commit point. + awaitCommitCounter(commitCounter, serverA, serverB); + + // Verify/await snapshot on A. + awaitSnapshotExists(serverA, commitCounter); + + // Verify/await snapshot on B. + awaitSnapshotExists(serverB, commitCounter); + + // Take A+B through one more commit point. + simpleTransaction(); + + // Verify/await expected commit point. + awaitCommitCounter(commitCounter + 1, serverA, serverB); + + // Start C. It will resync and join. + final HAGlue serverC = startC(); + + // Await fully quorum meet. + final long token2 = awaitFullyMetQuorum(); + + // Token is unchanged. + assertEquals(token, token2); + + // Wait until service is ready. + awaitNSSAndHAReady(serverC); + + // Verify/await expected commit point. + awaitCommitCounter(commitCounter + 1, serverA, serverB, serverC); + + // Verify that C took a snapshot @ commitCounter:=2. + awaitSnapshotExists(serverC, commitCounter + 1); + + } + + /** + * Unit test verifies that C snapshots the journal when it is rebuilding + * from the A+B. The leader must not have the HALogs on hand to trigger a + * REBUILD rather than a RESYNC. E.g., delete the log files and snapshots on + * the leader by hand. C should snapshot the journal when the REBUILD and + * RESYNC are done and it enters RunMet. + */ + public void test_AB_snapshotOnQuorumMeet_C_snapshotOnRebuild() + throws Exception { + + + // Start 2 services. + final HAGlue serverA = startA(); + final HAGlue serverB = startB(); + + // Wait for a quorum meet. + final long token = quorum.awaitQuorum(awaitQuorumTimeout, + TimeUnit.MILLISECONDS); + + // Current commit point. + final long lastCommitCounter = 1; + + // Await initial commit point (KB create) on A + B. + awaitCommitCounter(lastCommitCounter, serverA, serverB); + + /* + * Verify that HALog files were generated and are available for commit + * point ONE (1) on the services joined with the met quorum. + */ + assertHALogDigestsEquals(1L/* firstCommitCounter */, lastCommitCounter, + new HAGlue[] { serverA, serverB }); + + // Verify binary equality of (A,B) journals. + assertDigestsEquals(new HAGlue[] { serverA, serverB }); + + /* + * Now go through a commit point with a met quorum. The HALog files + * should be retained at that commit point. + */ + simpleTransaction(); + + // Current commit point. + final long lastCommitCounter2 = serverA + .getRootBlock(new HARootBlockRequest(null/* storeUUID */)) + .getRootBlock().getCommitCounter(); + + // There are TWO (2) commit points. + assertEquals(2L, lastCommitCounter2); + + // HALog files now exist for A & B, and original commitCounter! + assertHALogDigestsEquals(lastCommitCounter, lastCommitCounter2, + new HAGlue[] { serverA, serverB }); + + // now remove the halog files ensuring a RESYNC is not possible + // but MUST leave currently open file!! + final String openLog = HALogWriter.getHALogFileName(lastCommitCounter2 + 1); + removeFiles(getHALogDirA(), openLog); + removeFiles(getHALogDirB(), openLog); + + // Now Start 3rd service. + final HAGlue serverC = startC(); + + // Wait until the quorum is fully met. After REBUILD + assertEquals(token, awaitFullyMetQuorum()); + + // HALog files now exist on ALL services @ current commit counter + final long currentCommitCounter = lastCommitCounter2 + 1; + assertHALogDigestsEquals(currentCommitCounter, currentCommitCounter, + new HAGlue[] { serverA, serverB, serverC }); + + // Verify binary equality of ALL journals. + assertDigestsEquals(new HAGlue[] { serverA, serverB, serverC }); + + // Verify that C took a snapshot *after* the REBUILD+RESYNC. + awaitSnapshotExists(serverC, currentCommitCounter); + + } + +} Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-05 18:25:08 UTC (rev 7036) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-05 21:15:56 UTC (rev 7037) @@ -111,6 +111,10 @@ // The last valid token. final long lastValidToken = quorum.lastValidToken(); + // This token is a bit different. It is senstive to the journal role in + // the quorum (joined or not). + final long haReadyToken = journal.getHAReady(); + final int njoined = quorum.getJoined().length; final QuorumService<HAGlue> quorumService = quorum.getClient(); @@ -136,9 +140,12 @@ p.text("logicalServiceId=" + quorumService.getLogicalServiceId()) .node("br").close(); - // Note: This is the *local* value of getHAStatus(). - p.text("HAStatus: " + getHAStatus(journal)).node("br").close(); - + // Note: This is the *local* value of getHAStatus(). + // Note: The HAReady token reflects whether or not the service is + // joined. + p.text("HAStatus: " + getHAStatus(journal) + ", HAReadyToken=" + + haReadyToken).node("br").close(); + /* * Report on the Service. */ This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-06 18:20:20
|
Revision: 7038 http://bigdata.svn.sourceforge.net/bigdata/?rev=7038&view=rev Author: thompsonbry Date: 2013-04-06 18:20:04 +0000 (Sat, 06 Apr 2013) Log Message: ----------- Merging changes from BIGDATA_RELEASE_1_2_2 into READ_CACHE. See https://sourceforge.net/apps/trac/bigdata/ticket/530 (HA Journal) for detailed status on this merge. update /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD -r HEAD --force At revision 7037. merge https://bigdata.svn.sourceforge.net/svnroot/bigdata/branches/BIGDATA_RELEASE_1_2_0 /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD --- Merging r6889 through r7037 into /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/test/com/bigdata/ha/TestAll.java C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/test/com/bigdata/io/writecache/TestWriteCache.java C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/io/writecache/WriteCacheService.java C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java Actually, the only conflict in StatusServlet was: {{{ new HAStatusServletUtil(getIndexManager()) .doGet(req, resp, current); }}} versus {{{ new HAStatusServletUtil(getIndexManager()).showQuorum(req, resp, current); }}} This is a simple method rename in HAStatusServlet. The READ_CACHE has the right version. Also, fixed a problem in HAStatusServlet where it was flushing out before closing the pre element. The correct order for those actions is shown below. {{{ // close section. out.print("\n</pre>"); // flush PrintWriter before resuming writes on Writer. out.flush(); }}} HA CI test outcomes are unchanged by this merge. Revision Links: -------------- http://bigdata.svn.sourceforge.net/bigdata/?rev=7037&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=6889&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7037&view=rev Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/DumpIndexSegment.java branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/IndexSegmentBuilder.java branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/view/FusedTupleCursor.java branches/READ_CACHE/bigdata/src/java/com/bigdata/relation/AbstractResource.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hiterator.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractFederation.java branches/READ_CACHE/bigdata/src/test/com/bigdata/search/TestPrefixSearch.java branches/READ_CACHE/bigdata/src/test/com/bigdata/search/TestSearch.java branches/READ_CACHE/bigdata/src/test/com/bigdata/search/TestSearchRestartSafe.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/service/jini/LoadBalancerServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/service/jini/util/DumpFederation.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/service/jini/TestBigdataClient.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/inf/FullyBufferedJustificationIterator.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/internal/constraints/RegexBOp.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/internal/impl/AbstractIV.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/lexicon/ITextIndexer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/ArbitraryLengthPathNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/FunctionRegistry.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/GroupMemberNodeBase.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/IQueryNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/JoinGroupNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/NamedSubqueryInclude.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/NamedSubqueryRoot.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/QueryNodeBase.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/StatementPatternNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/SubqueryRoot.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/ASTSearchOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/SearchInSearchServiceFactory.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/SearchServiceFactory.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTJoinOrderByTypeOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTPropertyPathOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTRangeOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTServiceNodeOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTSetValueExpressionsOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTSimpleOptionalOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTStaticJoinOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/AbstractJoinGroupOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/DefaultOptimizerList.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/service/ServiceRegistry.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/spo/SPOIndexWriter.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/spo/SPORelation.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/store/BD.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/store/DataLoader.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/lexicon/TestFullTextIndex.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/lexicon/TestSubjectCentricFullTextIndex.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/optimizers/TestASTEmptyGroupOptimizer.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/optimizers/TestASTQueryHintOptimizer.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/optimizers/TestASTSearchOptimizer.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/optimizers/TestASTServiceNodeOptimizer.java branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/sparql/ast/optimizers/TestASTStaticJoinOptimizer.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/Bigdata2Sesame2BindingSetIterator.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/BigdataSailTupleQuery.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFContext.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFServletContextListener.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ConfigParams.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/MultiTenancyServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/SparqlEndpointConfig.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/RemoteRepositoryManager.java branches/READ_CACHE/bigdata-sails/src/samples/com/bigdata/samples/SampleCode.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/TestSearchQuery.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/TestTicket581.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/tck/BigdataSPARQLUpdateTest.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/AbstractTestNanoSparqlClient.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/TestBigdataSailRemoteRepository.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/TestMultiTenancyAPI.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/TestNanoSparqlClient.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/TestSparqlUpdate.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/client/TestEncodeDecodeValue.java branches/READ_CACHE/bigdata-sails/src/test/org/openrdf/query/parser/sparql/SPARQLUpdateTest.java branches/READ_CACHE/bigdata-war/src/html/index.html branches/READ_CACHE/bigdata-war/src/resources/WEB-INF/web.xml branches/READ_CACHE/pom.xml Added Paths: ----------- branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/IReorderableNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/OldBackupPathNode.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/eval/ASTSearchInSearchOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTCardinalityOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/sparql/ast/optimizers/ASTRangeCountOptimizer.java branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/store/BDS.java branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/TestHelper.java Property Changed: ---------------- branches/READ_CACHE/ branches/READ_CACHE/bigdata/lib/jetty/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba/ branches/READ_CACHE/bigdata/src/java/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/test/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/util/httpd/ branches/READ_CACHE/bigdata-compatibility/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/attr/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/disco/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/util/config/ branches/READ_CACHE/bigdata-perf/ branches/READ_CACHE/bigdata-perf/btc/ branches/READ_CACHE/bigdata-perf/btc/src/resources/ branches/READ_CACHE/bigdata-perf/lubm/ branches/READ_CACHE/bigdata-perf/uniprot/ branches/READ_CACHE/bigdata-perf/uniprot/src/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/changesets/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/error/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/util/ branches/READ_CACHE/bigdata-rdf/src/samples/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/changesets/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/dsi-utils/ branches/READ_CACHE/dsi-utils/LEGAL/ branches/READ_CACHE/dsi-utils/lib/ branches/READ_CACHE/dsi-utils/src/ branches/READ_CACHE/dsi-utils/src/java/ branches/READ_CACHE/dsi-utils/src/java/it/ branches/READ_CACHE/dsi-utils/src/java/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/dsi/ branches/READ_CACHE/lgpl-utils/src/java/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/lgpl-utils/src/test/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/osgi/ branches/READ_CACHE/src/resources/bin/config/ Property changes on: branches/READ_CACHE ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-6889 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-7037 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata/lib/jetty ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-6889 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-7037 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-6889 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-6889 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-6889 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/DumpIndexSegment.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/DumpIndexSegment.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/DumpIndexSegment.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -65,33 +65,59 @@ /** * Dump one or more {@link IndexSegment}s. + * <p> + * Note: The <code>-nodeState</code> and <code>-leafState</code> options + * also require you to turn up the logging level in order to see the output. + * However, when true they will apply a variety of validation tests to the + * nodes and leaves regardless of whether their state is written onto the + * console. * * @param args - * usage <code>[file|-d level]+</code>, where <i>file</i> is - * the name of a n {@link IndexSegmentStore} file and - * <code>level is the name of the {@link Level} to be used - * for the {@link AbstractBTree#dumpLog}</code> - * - * @throws IOException + * usage <code>[file|-d level|-nodeState|-leafState]+</code>, + * where + * <dl> + * <dt>file</dt> + * <dl> + * is the name of a n {@link IndexSegmentStore} file + * </dl> + * <dt>level</dt> + * <dl> + * is the name of the {@link Level} to be used for the + * {@link AbstractBTree#dumpLog} + * </dl> + * <dt>-nodeState</dt> + * <dl> + * Enables the dump of the {@link Node} state + * </dl> + * <dt>-leafState</dt> + * <dl> + * Enables the dump of the {@link Node} state + * </dl> + * </dl> + * + * @throws IOException */ - public static void main(String[] args) throws IOException { - - if(args.length==0) { + public static void main(final String[] args) throws IOException { + + if (args.length == 0) { usage(); - + System.exit(1); - + } + boolean dumpNodeState = false; + boolean dumpLeafState = false; + for (int i = 0; i < args.length; i++) { final String arg = args[i]; if (arg.startsWith("-")) { - if( arg.equals("-d")) { - + if (arg.equals("-d")) { + final Level level = Level.toLevel(args[++i]); System.out.println("Setting log level: "+level); @@ -116,6 +142,14 @@ } } + } else if(arg.equals("-nodeState")) { + + dumpNodeState = true; + + } else if(arg.equals("-leafState")) { + + dumpLeafState = true; + } else { System.err.println("Unknown option: "+arg); @@ -136,7 +170,7 @@ } - dumpIndexSegment(file); + dumpIndexSegment(file, dumpNodeState, dumpLeafState); } @@ -144,17 +178,9 @@ } - static void dumpIndexSegment(final File file) throws IOException { + static void dumpIndexSegment(final File file, final boolean dumpNodeState, + final boolean dumpLeafState) throws IOException { - /* - * Note: These options also require you to turn up the logging level in - * order to see the output. However, when true they will apply a variety - * of validation tests to the nodes and leaves regardless of whether - * their state is written onto the console. - */ - boolean dumpNodeState = true; // @todo command line option - boolean dumpLeafState = true;// @todo command line option - final IndexSegmentStore store = new IndexSegmentStore(file); // dump the checkpoint record, index metadata record, etc. @@ -354,8 +380,9 @@ * * @param store */ - static void dumpLeavesReverseScan(IndexSegmentStore store,boolean dumpLeafState) { - + static void dumpLeavesReverseScan(final IndexSegmentStore store, + final boolean dumpLeafState) { + final long begin = System.currentTimeMillis(); final AbstractBTree btree = store.loadIndexSegment(); @@ -458,8 +485,9 @@ * * @param store */ - static void dumpLeavesForwardScan(IndexSegmentStore store,boolean dumpLeafState) { - + static void dumpLeavesForwardScan(final IndexSegmentStore store, + final boolean dumpLeafState) { + final long begin = System.currentTimeMillis(); final AbstractBTree btree = store.loadIndexSegment(); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/IndexSegmentBuilder.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/IndexSegmentBuilder.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/IndexSegmentBuilder.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -1031,6 +1031,240 @@ } /** + * Variant using an array of objects in the desired order. A single root + * leaf is generated from those objects. The root leaf is then fed into the + * algorithm to efficient construct the corresponding read-only + * {@link IndexSegment}. + * + * @param a + * The array of objects to be written onto the index. The index + * must know how to generate tuples from these objects. The + * objects must already be in the natural order of the keys that + * will be generated for those tuples. + * @param alen + * The #of elements in that array. + * @param indexMetadata + * The {@link IndexMetadata} that will serve as the template for + * the generated {@link IndexSegment}. + * @param outFile + * The file on which the {@link IndexSegment} will be written. + * The file MAY exist, but if it exists then it MUST be empty. + * @param tmpDir + * The temporary directory in data are buffered during the build + * (optional - the default temporary directory is used if this is + * <code>null</code>). + * @param m + * The branching factor for the generated {@link IndexSegment}. + * @param compactingMerge + * When <code>true</code> the caller asserts that <i>src</i> is a + * {@link FusedView} and deleted index entries WILL NOT be + * included in the generated {@link IndexSegment}. Otherwise, it + * is assumed that the only select component(s) of the index + * partition view are being exported onto an {@link IndexSegment} + * and deleted index entries will therefore be propagated to the + * new {@link IndexSegment} (aka an incremental build). + * @param createTime + * The commit time associated with the view from which the + * {@link IndexSegment} is being generated. This value is written + * into {@link IndexSegmentCheckpoint#commitTime}. + * @param bufferNodes + * When <code>true</code> the generated nodes will be fully + * buffered in RAM (faster, but imposes a memory constraint). + * Otherwise they will be written onto a temporary file and then + * transferred to the output file en mass. + * @return + * @throws IOException + * + * TODO We could pass a flag indicating whether the leaf needs + * to be sorted after it is generated, but the caller would + * still be responsible for ensuring that there are no + * duplicates in the array. + */ +// * @param fromKey +// * The lowest key that will be included (inclusive). When +// * <code>null</code> there is no lower bound. +// * @param toKey +// * The first key that will be included (exclusive). When +// * <code>null</code> there is no upper bound. + @SuppressWarnings("unchecked") + public static IndexSegmentBuilder newInstance( + final Object[] a, final int alen, + final IndexMetadata indexMetadata, final File outFile, + final File tmpDir, final int m, final boolean compactingMerge, + final long createTime, + //final byte[] fromKey, final byte[] toKey, + final boolean bufferNodes) + throws IOException { + + if (a == null) + throw new IllegalArgumentException(); + + if (alen < 0) + throw new IllegalArgumentException(); + + if (alen > a.length) + throw new IllegalArgumentException(); + + if (indexMetadata == null) + throw new IllegalArgumentException(); + + if (outFile == null) + throw new IllegalArgumentException(); + + if (tmpDir == null) + throw new IllegalArgumentException(); + + if (createTime <= 0L) + throw new IllegalArgumentException(); + + final boolean hasVersionTimestamps = indexMetadata + .getVersionTimestamps(); + + if (hasVersionTimestamps) + throw new IllegalArgumentException( + "versionTimestamps not available in source []."); + + final boolean hasDeleteMarkers = indexMetadata.getDeleteMarkers(); + + if (hasDeleteMarkers && !compactingMerge) + throw new IllegalArgumentException( + "deleteMarkers not available in source []."); + + final boolean hasRawRecords = indexMetadata.getRawRecords(); + + // A temporary leaf used to buffer the data in RAM. + final MutableLeafData tleaf = new MutableLeafData(alen, + hasVersionTimestamps, hasDeleteMarkers, hasRawRecords); + + final int flags; + if (compactingMerge) { + + /* + * For a compacting merge the delete markers are ignored so they + * will NOT be transferred to the new index segment. + */ + + flags = IRangeQuery.DEFAULT; + + } else { + + /* + * For an incremental build the deleted tuples are propagated to the + * new index segment. This is required in order for the fact that + * those tuples were deleted as of the commitTime to be retained by + * the generated index segment. + */ + + flags = IRangeQuery.DEFAULT | IRangeQuery.DELETED; + + } + + /* + * Iterator reading the source tuples to be copied to the index segment. + * + * Note: The DELETED flag was set above unless this is a compacting + * merge. That is necessary to ensure that deleted tuples are preserved + * when the index segment does not reflect the total history of a view. + * + * The tuples are materialized and buffered in a single, and potentially + * very large, leaf. That is Ok since the MutableLeaf is using very + * simple data structures. + * + * @todo The fastRangeCount is a hint that we want to eagerly + * materialize all of the data. This hint should be turned into + * pre-fetch and into a single IO for the index segment leaves if they + * are not in memory. [In fact, the hint is completely ignored at this + * point. If hints get more weight, then review code for their use.] + */ +// final ITupleIterator<?> titr = src.rangeIterator(fromKey, toKey, +// (int) fastRangeCount/* capacity */, flags, null/* filter */); + + // init per API specification. + long minimumVersionTimestamp = Long.MAX_VALUE; + long maximumVersionTimestamp = Long.MIN_VALUE; + @SuppressWarnings("rawtypes") + final ITupleSerializer tupSer = indexMetadata.getTupleSerializer(); + for (int i = 0; i < alen; i++) { + +// final ITuple<?> tuple = titr.next(); + + tleaf.keys.keys[i] = tupSer.serializeKey(a[i]); + + // Note: Version timestamps are not available from a[]. +// if (hasVersionTimestamps) { +// +// final long t = tuple.getVersionTimestamp(); +// +// tleaf.versionTimestamps[i] = t; +// +// if (t < minimumVersionTimestamp) { +// +// minimumVersionTimestamp = t; +// +// } +// +// if (t > maximumVersionTimestamp) { +// +// maximumVersionTimestamp = t; +// +// } +// +// } + + // Note: delete markers are not available from a[]. +// if (hasDeleteMarkers && tuple.isDeletedVersion()) { +// +// /* +// * Note: When delete markers are used, the array will be +// * pre-populated with [false] so we only have to set the flag on +// * the tuples that are actually deleted. +// */ +// tleaf.deleteMarkers[i] = true; +// +// } else { + + /* + * Note: If the source has raw records for some values, then + * this will cause those records to be materialized within the + * single massive root leaf. From there, the data will be + * written onto the index segment file. + */ + tleaf.vals.values[i] = tupSer.serializeVal(a[i]); + +// } + +// i++; + + } + + tleaf.keys.nkeys = alen; // note final #of tuples. + tleaf.vals.nvalues = alen; // note final #of tuples. + tleaf.maximumVersionTimestamp = maximumVersionTimestamp; + tleaf.minimumVersionTimestamp = minimumVersionTimestamp; + + // The exact range count. + final int nentries = alen; + + // The source iterator (reading on the fully buffered tuples). + @SuppressWarnings("rawtypes") + final ITupleIterator<?> itr = new MyTupleIterator(tleaf, flags); + + // Setup the index segment build operation. + return IndexSegmentBuilder.newInstance(// + outFile, // + tmpDir, // + nentries, // exact range count + itr, // source iterator + m, // the output branching factor. + indexMetadata,// + createTime,// + compactingMerge,// + bufferNodes// + ); + + } + + /** * <p> * A more flexible factory for an {@link IndexSegment} build which permits * override of the index segment branching factor, replacement of the Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/view/FusedTupleCursor.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/view/FusedTupleCursor.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/btree/view/FusedTupleCursor.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -83,8 +83,8 @@ * on a fused view require that the tuple is overwritten by a * delete marker in the 1st index of the view. */ - public FusedTupleCursor(int flags, boolean deleted, ITupleCursor<E>[] srcs, - IIndex ndx) { + public FusedTupleCursor(final int flags, final boolean deleted, + final ITupleCursor<E>[] srcs, final IIndex ndx) { super(flags, deleted, srcs); @@ -561,7 +561,7 @@ * Note: You must specify {@link IRangeQuery#CURSOR} in order for * {@link #remove()} to be supported. */ - @SuppressWarnings("unchecked") +// @SuppressWarnings("unchecked") @Override public void remove() { Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-6889 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/jsr166 ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/jsr166:6766-6889 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/jsr166:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/jsr166:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/jsr166:4814-4836 Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/relation/AbstractResource.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/relation/AbstractResource.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/relation/AbstractResource.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -49,6 +49,7 @@ import com.bigdata.journal.IJournal; import com.bigdata.journal.IResourceLock; import com.bigdata.journal.IResourceLockService; +import com.bigdata.journal.NoSuchIndexException; import com.bigdata.journal.TimestampUtility; import com.bigdata.rdf.rules.FastClosure; import com.bigdata.rdf.rules.FullClosure; @@ -700,7 +701,12 @@ final String name = itr.next(); - indexManager.dropIndex(name); + try { + indexManager.dropIndex(name); + } catch (NoSuchIndexException ex) { + // If the index does not exist, keep on going. + log.warn("Ignored: " + ex); + } } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -69,6 +69,7 @@ import com.bigdata.journal.IResourceLock; import com.bigdata.journal.ITx; import com.bigdata.journal.TimestampUtility; +import com.bigdata.rdf.lexicon.ITextIndexer.FullTextQuery; import com.bigdata.relation.AbstractRelation; import com.bigdata.relation.locator.DefaultResourceLocator; import com.bigdata.striterator.IChunkedOrderedIterator; @@ -439,7 +440,7 @@ /** * See {@link Options#HIT_CACHE_SIZE}. */ - private final ConcurrentWeakValueCacheWithTimeout<FullTextSearchQuery, Hit<V>[]> cache; + private final ConcurrentWeakValueCacheWithTimeout<FullTextQuery, Hit<V>[]> cache; // /** // * @see Options#DOCID_FACTORY_CLASS @@ -564,7 +565,7 @@ } this.cache = - new ConcurrentWeakValueCacheWithTimeout<FullTextSearchQuery, Hit<V>[]>( + new ConcurrentWeakValueCacheWithTimeout<FullTextQuery, Hit<V>[]>( hitCacheSize, hitCacheTimeoutMillis); { @@ -946,58 +947,36 @@ * iterator that is sent to the data service such that the search * terms are visited only when they occur in the matching field(s). */ - public Hiterator<Hit<V>> search(final String query, final String languageCode, - final boolean prefixMatch, - final double minCosine, final double maxCosine, - final int minRank, final int maxRank, - final boolean matchAllTerms, final boolean matchExact, - long timeout, final TimeUnit unit, final String regex) { + public Hiterator<Hit<V>> search(final FullTextQuery query) { - final Hit<V>[] a = _search(query, languageCode, prefixMatch, minCosine, - maxCosine, minRank, maxRank, matchAllTerms, matchExact, timeout, unit, regex); + final Hit<V>[] a = _search(query); - return new Hiterator<Hit<V>>(// - Arrays.asList(a)// -// ,minCosine,//0.0d,// minCosine -// Integer.MAX_VALUE // maxRank - ); + return new Hiterator<Hit<V>>(a); } - /** - * Used to support test cases. - */ - public int count(final String query, final String languageCode, - final boolean prefixMatch) { + public int count(final FullTextQuery query) { - return count(query, languageCode, prefixMatch, 0.0d, 1.0d, 1, 10000, - false, false, this.timeout,// - TimeUnit.MILLISECONDS, null); - - } - - - public int count(final String query, final String languageCode, - final boolean prefixMatch, - final double minCosine, final double maxCosine, - final int minRank, final int maxRank, - final boolean matchAllTerms, final boolean matchExact, - long timeout, final TimeUnit unit, final String regex) { - - final Hit[] a = _search(query, languageCode, prefixMatch, minCosine, - maxCosine, minRank, maxRank, matchAllTerms, matchExact, timeout, unit, regex); + final Hit[] a = _search(query); return a.length; } - private Hit<V>[] _search( - final String query, final String languageCode, - final boolean prefixMatch, - final double minCosine, final double maxCosine, - final int minRank, final int maxRank, - final boolean matchAllTerms, final boolean matchExact, - long timeout, final TimeUnit unit, final String regex) { + public Hit<V>[] _search(final FullTextQuery q) { + + final String query = q.getQuery(); + final String languageCode = q.getLanguageCode(); + final boolean prefixMatch = q.isPrefixMatch(); + final double minCosine = q.getMinCosine(); + final double maxCosine = q.getMaxCosine(); + final int minRank = q.getMinRank(); + final int maxRank = q.getMaxRank(); + final boolean matchAllTerms = q.isMatchAllTerms(); + final boolean matchExact = q.isMatchExact(); + final String regex = q.getMatchRegex(); + long timeout = q.getTimeout(); + final TimeUnit unit = q.getTimeUnit(); final long begin = System.currentTimeMillis(); @@ -1038,23 +1017,21 @@ } - final FullTextSearchQuery cacheKey = new FullTextSearchQuery( - query, matchAllTerms, matchExact, prefixMatch, timeout, unit, regex - ); + final FullTextQuery cacheKey = q; Hit<V>[] a; if (cache.containsKey(cacheKey)) { - if (log.isDebugEnabled()) - log.debug("found hits in cache"); + if (log.isInfoEnabled()) + log.info("found hits in cache"); a = cache.get(cacheKey); } else { - if (log.isDebugEnabled()) - log.debug("did not find hits in cache"); + if (log.isInfoEnabled()) + log.info("did not find hits in cache"); // tokenize the query. final TermFrequencyData<V> qdata; @@ -1495,89 +1472,4 @@ throw new UnsupportedOperationException(); } - private static final class FullTextSearchQuery { - - private final String search; - private final boolean matchAllTerms; - private final boolean matchExact; - private final boolean prefixMatch; - private final long timeout; - private final TimeUnit unit; - private final String regex; - - public FullTextSearchQuery( - final String search, - final boolean matchAllTerms, - final boolean matchExact, - final boolean prefixMatch, - final long timeout, - final TimeUnit unit, - final String regex) { - - this.search = search; - this.matchAllTerms = matchAllTerms; - this.matchExact = matchExact; - this.prefixMatch = prefixMatch; - this.timeout = timeout; - this.unit = unit; - this.regex = regex; - - } - - /** - * Generated by Eclipse. - */ - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + (matchAllTerms ? 1231 : 1237); - result = prime * result + (matchExact ? 1231 : 1237); - result = prime * result + (prefixMatch ? 1231 : 1237); - result = prime * result + ((regex == null) ? 0 : regex.hashCode()); - result = prime * result - + ((search == null) ? 0 : search.hashCode()); - result = prime * result + (int) (timeout ^ (timeout >>> 32)); - result = prime * result + ((unit == null) ? 0 : unit.hashCode()); - return result; - } - - /** - * Generated by Eclipse. - */ - @Override - public boolean equals(Object obj) { - if (this == obj) - return true; - if (obj == null) - return false; - if (getClass() != obj.getClass()) - return false; - FullTextSearchQuery other = (FullTextSearchQuery) obj; - if (matchAllTerms != other.matchAllTerms) - return false; - if (matchExact != other.matchExact) - return false; - if (prefixMatch != other.prefixMatch) - return false; - if (regex == null) { - if (other.regex != null) - return false; - } else if (!regex.equals(other.regex)) - return false; - if (search == null) { - if (other.search != null) - return false; - } else if (!search.equals(other.search)) - return false; - if (timeout != other.timeout) - return false; - if (unit != other.unit) - return false; - return true; - } - - - } - } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hiterator.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hiterator.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hiterator.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -1,8 +1,6 @@ package com.bigdata.search; -import java.util.Collection; import java.util.Iterator; -import java.util.NoSuchElementException; /** * Visits search results in order of decreasing relevance. @@ -12,159 +10,223 @@ */ public class Hiterator<A extends IHit> implements Iterator<A> { - final private Collection<A> hits; - final private Iterator<A> src; -// final private double minCosine; -// final private int maxRank; + /** + * The index into the array of hits wrapped by this iterator. + */ + private int rank = 0; + + /** + * The array of hits wrapped by this iterator. + */ + private final A[] hits; + + /** + * + * @param hits + */ + public Hiterator(final A[] hits) { + if (hits == null) + throw new IllegalArgumentException(); + + this.hits = hits; + + } + + public boolean hasNext() { + + return rank < hits.length; + + } + + public A next() { + + return hits[rank++]; + + } + /** - * The rank of the last hit returned by {@link #next()}. The rank is in - * [1:nhits] and is incremented as we return each hit in {@link #next()}. + * @throws UnsupportedOperationException */ - private int rank = 0; + public void remove() { + + throw new UnsupportedOperationException(); + + } + + public String toString() { + + return "Hiterator{nhits=" + hits.length + "} : " + + hits; + + } - /** set by {@link #hasNext()}. */ - private boolean exhausted = false; - - /** set by {@link #hasNext()}. */ - private A nextHit = null; - -// /** set by {@link #next()}. */ -// private A lastHit = null; + public int size() { + + return hits.length; + + } +// +// +// final private Collection<A> hits; +// final private Iterator<A> src; +//// final private double minCosine; +//// final private int maxRank; +// // /** -// * The minimum cosine that will be visited as specified to -// * {@link FullTextIndex#search(String, String, double, int)} +// * The rank of the last hit returned by {@link #next()}. The rank is in +// * [1:nhits] and is incremented as we return each hit in {@link #next()}. // */ -// public double minCosine() { +// private int rank = 0; +// +// /** set by {@link #hasNext()}. */ +// private boolean exhausted = false; +// +// /** set by {@link #hasNext()}. */ +// private A nextHit = null; +// +//// /** set by {@link #next()}. */ +//// private A lastHit = null; +// +//// /** +//// * The minimum cosine that will be visited as specified to +//// * {@link FullTextIndex#search(String, String, double, int)} +//// */ +//// public double minCosine() { +//// +//// return minCosine; +//// +//// } +//// +//// /** +//// * The maximum rank that will be visited as specified to +//// * {@link FullTextIndex#search(String, String, double, int)} +//// */ +//// public int maxRank() { +//// +//// return maxRank; +//// +//// } +// +// /** +// * +// * @param hits +// */ +// public Hiterator(final Collection<A> hits +//// ,final double minCosine, final int maxRank +// ) { +// +// if (hits == null) +// throw new IllegalArgumentException(); +// +//// if (minCosine < 0d || minCosine > 1d) +//// throw new IllegalArgumentException(); +// +//// if (maxRank <= 0) +//// throw new IllegalArgumentException(); +// +// this.hits = hits; +// +//// this.minCosine = minCosine; +//// +//// this.maxRank = maxRank; // -// return minCosine; +// this.src = hits.iterator(); // // } +// +//// /** +//// * The #of hits (approximate). +//// * +//// * @todo differentiate between the #of hits and the #of hits that satisfy +//// * the minCosine and maxRank criteria +//// * +//// * @todo this and other search engine metadata (elapsed time) might go on a +//// * different object from which we can obtain the {@link Hiterator}. +//// */ +//// public long size() { +//// +//// return hits.size(); +//// +//// } // +// public boolean hasNext() { +// +// if(exhausted) return false; +// +// if(nextHit!=null) return true; +// +// if(!src.hasNext()) { +// +// exhausted = true; +// +// return false; +// +// } +// +// nextHit = src.next(); +// +//// if (rank + 1 > maxRank || nextHit.getCosine() < minCosine) { +//// +//// exhausted = true; +//// +//// return false; +//// +//// } +// +// return true; +// +// } +// +// public A next() { +// +// if(!hasNext()) throw new NoSuchElementException(); +// +// final A tmp = nextHit; +// +// nextHit = null; +// +// rank++; +// +// return tmp; +// +// } +// // /** -// * The maximum rank that will be visited as specified to -// * {@link FullTextIndex#search(String, String, double, int)} +// * The rank of the last hit returned (origin ONE). +// * +// * @throws IllegalStateException +// * if nothing has been visited yet. // */ -// public int maxRank() { +// public int rank() { +// +// if (rank == 0) +// throw new IllegalStateException(); // -// return maxRank; +// return rank; // // } - - /** - * - * @param hits - */ - public Hiterator(final Collection<A> hits -// ,final double minCosine, final int maxRank - ) { - - if (hits == null) - throw new IllegalArgumentException(); - -// if (minCosine < 0d || minCosine > 1d) -// throw new IllegalArgumentException(); - -// if (maxRank <= 0) -// throw new IllegalArgumentException(); - - this.hits = hits; - -// this.minCosine = minCosine; +// +// /** +// * @throws UnsupportedOperationException +// */ +// public void remove() { // -// this.maxRank = maxRank; - - this.src = hits.iterator(); - - } - - /** - * The #of hits (approximate). - * - * @todo differentiate between the #of hits and the #of hits that satisfy - * the minCosine and maxRank criteria - * - * @todo this and other search engine metadata (elapsed time) might go on a - * different object from which we can obtain the {@link Hiterator}. - */ - public long size() { - - return hits.size(); - - } - - public boolean hasNext() { - - if(exhausted) return false; - - if(nextHit!=null) return true; - - if(!src.hasNext()) { - - exhausted = true; - - return false; - - } - - nextHit = src.next(); - -// if (rank + 1 > maxRank || nextHit.getCosine() < minCosine) { +// throw new UnsupportedOperationException(); +// +// } // -// exhausted = true; -// -// return false; -// -// } - - return true; - - } - - public A next() { - - if(!hasNext()) throw new NoSuchElementException(); - - final A tmp = nextHit; - - nextHit = null; - - rank++; - - return tmp; - - } +// public String toString() { +// +// return "Hiterator{"+/*minCosine=" + minCosine +// + ", maxRank=" + maxRank + ",*/" nhits=" + hits.size() + "} : " +// + hits; +// +// } +// +// public int size() { +// return hits.size(); +// } - /** - * The rank of the last hit returned (origin ONE). - * - * @throws IllegalStateException - * if nothing has been visited yet. - */ - public int rank() { - - if (rank == 0) - throw new IllegalStateException(); - - return rank; - - } - - /** - * @throws UnsupportedOperationException - */ - public void remove() { - - throw new UnsupportedOperationException(); - - } - - public String toString() { - - return "Hiterator{"+/*minCosine=" + minCosine - + ", maxRank=" + maxRank + ",*/" nhits=" + hits.size() + "} : " - + hits; - - } } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractFederation.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractFederation.java 2013-04-05 21:15:56 UTC (rev 7037) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractFederation.java 2013-04-06 18:20:04 UTC (rev 7038) @@ -80,6 +80,7 @@ import com.bigdata.ganglia.GangliaSlopeEnum; import com.bigdata.ganglia.IGangliaDefaults; import com.bigdata.ganglia.util.GangliaUtil; +import com.bigdata.journal.NoSuchIndexException; import com.bigdata.journal.TemporaryStore; import com.bigdata.journal.TemporaryStoreFactory; import com.bigdata.rawstore.Bytes; @@ -89,6 +90,7 @@ import com.bigdata.service.ndx.ScaleOutIndexCounters; import com.bigdata.sparse.GlobalRowStoreHelper; import com.bigdata.sparse.SparseRowStore; +import com.bigdata.util.InnerCause; import com.bigdata.util.concurrent.DaemonThreadFactory; import com.bigdata.util.concurrent.ShutdownHelper; import com.bigdata.util.concurrent.TaskCounters; @@ -1014,7 +1016,14 @@ getIndexCache().dropIndexFromCache(name); } catch (Exception e) { - + if(InnerCause.isInnerCause(e, NoSuchIndexException.class)) { + /* + * Wrap with the root cause per the API for dropIndex(). + */ + final NoSuchIndexException tmp = new NoSuchIndexException(name); + tmp.initCause(e); + throw tmp; + } throw new RuntimeException( e ); } Property changes on: branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/joinGraph ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/joinGraph:6766-6889 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/joinGraph:6766-7037 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANC... [truncated message content] |
From: <tho...@us...> - 2013-04-10 17:35:28
|
Revision: 7040 http://bigdata.svn.sourceforge.net/bigdata/?rev=7040&view=rev Author: thompsonbry Date: 2013-04-10 17:35:17 +0000 (Wed, 10 Apr 2013) Log Message: ----------- We fixed the problem with the transition to RunMet causing the asynchronous close of the SocketChannel for the write pipeline by NOT propagating the interrupt to the SendIncTask's Future in HAJournal. This was documented for both sendHALog() and sendHAStore(). It now successfully transitions to RunMet during a long load before the commit point. 100% Green on the CI HA test suite. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HAReceiveService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HASendService.java branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestAll.java branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive.java branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive3Nodes.java branches/READ_CACHE/bigdata/src/test/com/bigdata/io/TestCase3.java branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/log4j-template-A.properties branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/log4j-template-B.properties branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/log4j-template-C.properties Added Paths: ----------- branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceiveInterrupts.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumPipelineImpl.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -31,6 +31,7 @@ import java.nio.ByteBuffer; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.FutureTask; @@ -906,9 +907,9 @@ } catch (Throwable t) { - if (InnerCause.isInnerCause(t, InterruptedException.class)) { + if (InnerCause.isInnerCause(t, InterruptedException.class)||InnerCause.isInnerCause(t, CancellationException.class)) { - throw (InterruptedException) t; + throw new RuntimeException(t); } @@ -1020,9 +1021,9 @@ } catch (Throwable t) { - if (InnerCause.isInnerCause(t, InterruptedException.class)) { + if (InnerCause.isInnerCause(t, InterruptedException.class)||InnerCause.isInnerCause(t, CancellationException.class)) { - throw (InterruptedException) t; + throw new RuntimeException( t ); } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HAReceiveService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HAReceiveService.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HAReceiveService.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -28,6 +28,7 @@ import java.net.BindException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import java.nio.channels.AsynchronousCloseException; import java.nio.channels.SelectionKey; import java.nio.channels.Selector; import java.nio.channels.ServerSocketChannel; @@ -512,7 +513,7 @@ message = null; // [waitFuture] is available for receiveData(). - futureReady.signal(); + futureReady.signalAll(); } finally { @@ -828,70 +829,101 @@ } public Void call() throws Exception { - -// awaitAccept(); -// -// /* -// * Get the client connection and open the channel in a non-blocking -// * mode so we will read whatever is available and loop until all -// * data has been read. -// */ -// final SocketChannel client = server.accept(); -// client.configureBlocking(false); -// -// final Selector clientSelector = Selector.open(); + +// awaitAccept(); +// +// /* +// * Get the client connection and open the channel in a non-blocking +// * mode so we will read whatever is available and loop until all +// * data has been read. +// */ +// final SocketChannel client = server.accept(); +// client.configureBlocking(false); +// +// final Selector clientSelector = Selector.open(); // -// // must register OP_READ selector on the new client -// final SelectionKey clientKey = client.register(clientSelector, -// SelectionKey.OP_READ); +// // must register OP_READ selector on the new client +// final SelectionKey clientKey = client.register(clientSelector, +// SelectionKey.OP_READ); - Client client = clientRef.get(); + Client client = clientRef.get(); -// if (client != null) { +// if (client != null) { // -// /* -// * Note: We need to know when the client connection is no longer -// * valid. The code here does not appear to do the trick. -// * changeUpStream() is handling this instead. -// * -// * We need to decide whether the client is no longer valid -// * (either because the upstream HASendService has changed (our -// * predecessor in the pipeline might have died) or because it -// * has closed is socket connection to this HAReceiveService). -// * -// * Either way, we need to establish a client connection using -// * awaitAccept(). -// */ -// if (!client.client.isConnected()) { -// log.warn("Closing old client connection."); -// clientRef.set(client = null); -// } +// /* +// * Note: We need to know when the client connection is no longer +// * valid. The code here does not appear to do the trick. +// * changeUpStream() is handling this instead. +// * +// * We need to decide whether the client is no longer valid +// * (either because the upstream HASendService has changed (our +// * predecessor in the pipeline might have died) or because it +// * has closed is socket connection to this HAReceiveService). +// * +// * Either way, we need to establish a client connection using +// * awaitAccept(). +// */ +// if (!client.client.isConnected()) { +// log.warn("Closing old client connection."); +// clientRef.set(client = null); +// } // -// } - - if (client == null) { +// } + + if (client == null || !client.client.isOpen()) { - /* - * Accept and the initialize a connection from the upstream - * HASendService. - */ - - // Accept a client connection (blocks) - awaitAccept(); + final Client tmp = clientRef.getAndSet(null); + if (tmp != null) { + // Close existing connection if not open. + tmp.close(); + } - // New client connection. - client = new Client(server);//, sendService, addrNext); + /* + * Accept and the initialize a connection from the upstream + * HASendService. + */ + + // Accept a client connection (blocks) + awaitAccept(); - // save off reference. - clientRef.set(client); + // New client connection. + client = new Client(server);//, sendService, addrNext); + // save off reference. + clientRef.set(client); + } +// boolean success = false; +// try { + doReceiveAndReplicate(client); +// success = true; + // success. + return null; +// } finally { +// try { +// if(success) { +// ack(client); +// } else { +// nack(client); +// } +// } catch (IOException ex) { +// // log and ignore. +// log.error(ex, ex); +// } +// } + + } // call. + + private void doReceiveAndReplicate(final Client client) + throws Exception { + /* * We should now have parameters ready in the WriteMessage and can * begin transferring data from the stream to the writeCache. */ final long begin = System.currentTimeMillis(); + long mark = begin; // #of bytes remaining (to be received). int rem = message.getSize(); @@ -909,15 +941,44 @@ if (nkeys == 0) { - // Nothing available. - final long elapsed = System.currentTimeMillis() - begin; + /* + * Nothing available. + */ + // time since last mark. + final long now = System.currentTimeMillis(); + final long elapsed = now - mark; + if (elapsed > 10000) { // Issue warning if we have been blocked for a while. log.warn("Blocked: awaiting " + rem + " out of " + message.getSize() + " bytes."); + mark = now;// reset mark. } + + if (!client.client.isOpen() + || !client.clientSelector.isOpen()) { + + /* + * The channel has been closed. The request must be + * failed. TODO Or set EOF:=true? + * + * Note: The [callback] is NOT notified. The service + * that issued the RMI request to this service to + * receive the payload over the HAReceivedService will + * see this exception thrown back across the RMI + * request. + * + * @see HAReceiveService.receiveData(). + */ + + throw new AsynchronousCloseException(); + + } + // no keys. nothing to read. + continue; + } final Set<SelectionKey> keys = client.clientSelector @@ -938,7 +999,7 @@ + " bytes remaining."); if (rdlen > 0) { - reads++; + reads++; updateChk(rdlen); } @@ -1015,7 +1076,7 @@ + ", number of reads: " + reads + ", buffer: " + localBuffer); - if (message.getChk() != (int) chk.getValue()) { + if (message.getChk() != (int) chk.getValue()) { throw new ChecksumError("msg=" + message.toString() + ", actual=" + chk.getValue()); } @@ -1024,10 +1085,97 @@ callback.callback(message, localBuffer); } - // success. - return null; + } // call() - } // call() +// private void ack(final Client client) throws IOException { +// +// if (log.isTraceEnabled()) +// log.trace("Will ACK"); +// +// ack(client.client, HASendService.ACK); +// +// if (log.isTraceEnabled()) +// log.trace("Did ACK"); +// +// } +// +// private void nack(final Client client) throws IOException { +// +// if (log.isTraceEnabled()) +// log.trace("Will NACK"); +// +// ack(client.client, HASendService.NACK); +// +// if (log.isTraceEnabled()) +// log.trace("Did NACK"); +// +// } +// +// /** +// * ACK/NACK the payload. +// * +// * @param client +// * @throws IOException +// */ +// private void ack(final SocketChannel client, final byte ret) +// throws IOException { +// +// // FIXME optimize. +// final ByteBuffer b = ByteBuffer.wrap(new byte[] { ret /* ACK */}); +// +// // The #of bytes to transfer. +// final int remaining = b.remaining(); +// +//// if (log.isTraceEnabled()) +//// log.trace("Will send " + remaining + " bytes"); +// +//// try { +// +// int nwritten = 0; +// +// while (nwritten < remaining) { +// +// /* +// * Write the data. Depending on the channel, will either +// * block or write as many bytes as can be written +// * immediately (this latter is true for socket channels in a +// * non-blocking mode). IF it blocks, should block until +// * finished or until this thread is interrupted, e.g., by +// * shutting down the thread pool on which it is running. +// * +// * Note: If the SocketChannel is closed by an interrupt, +// * then the send request for the [data] payload will fail. +// * However, the SocketChannel will be automatically reopened +// * for the next request (unless the HASendService has been +// * terminated). +// */ +// +// final int nbytes = client.write(b); +// +// nwritten += nbytes; +// +//// if (log.isTraceEnabled()) +//// log.trace("Sent " + nbytes + " bytes with " + nwritten +//// + " of out " + remaining + " written so far"); +// +// } +// return; +// +//// while (client.isOpen()) { +//// +//// if (client.write(b) > 0) { +//// +//// // Sent (N)ACK byte. +//// return; +//// +//// } +//// +//// } +// +//// // channel is closed. +//// throw new AsynchronousCloseException(); +// +// } } // class ReadTask @@ -1069,7 +1217,7 @@ localBuffer = buffer;// DO NOT duplicate()! (side-effects required) localBuffer.limit(message.getSize()); localBuffer.position(0); - messageReady.signal(); + messageReady.signalAll(); if (log.isTraceEnabled()) log.trace("Will accept data for message: msg=" + msg); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HASendService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HASendService.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/pipeline/HASendService.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -82,6 +82,9 @@ private static final Logger log = Logger.getLogger(HASendService.class); +// static final byte ACK = 1; +// static final byte NACK = 0; + /** * The Internet socket address of the receiving service. */ @@ -295,20 +298,43 @@ if (log.isTraceEnabled()) log.trace("Will send " + buffer.remaining() + " bytes"); - /* +// reopenChannel(); + + return tmp.submit(newIncSendTask(buffer.asReadOnlyBuffer())); + + } + + /** + * A series of timeouts used when we need to re-open the + * {@link SocketChannel}. + */ + private final static long[] retryMillis = new long[] { 1, 5, 10, 50, 100 }; + + /** + * (Re-)open the {@link SocketChannel} if it is closed and this service is + * still running. + * + * @return The {@link SocketChannel}. + */ + private SocketChannel reopenChannel() { + + /* * Synchronize on the socketChannel object to serialize attempts to open * the SocketChannel. */ synchronized (socketChannel) { - SocketChannel sc = socketChannel.get(); + int tryno = 0; + + SocketChannel sc = null; - if (sc == null) { + while ((((sc = socketChannel.get()) == null) || !sc.isOpen()) + && isRunning()) { try { /* - * Open the SocketChannel. + * (Re-)open the SocketChannel. * * TODO we may have to retry or play with the timeout for * the socket connect request since the downstream node may @@ -333,24 +359,38 @@ * since B does not setup its receive service until after A * has seen the pipelineChange() event. */ - + socketChannel.set(sc = openChannel(addrNext.get())); - + } catch (IOException e) { + if (tryno < retryMillis.length) { + + try { + // sleep and retry. + Thread.sleep(retryMillis[tryno]); + tryno++; + continue; + } catch (InterruptedException e1) { + // rethrow original exception. + throw new RuntimeException(e); + } + + } + // do not wrap. throw new RuntimeException(e); - - } - } - - } + } // catch - return tmp.submit(newIncSendTask(buffer.asReadOnlyBuffer())); + } // while - } - + return socketChannel.get(); + + } // synchronized(socketChannel) + + } // reopenChannel() + /** * Factory for the {@link SendTask}. * @@ -362,7 +402,7 @@ */ protected Callable<Void> newIncSendTask(final ByteBuffer buffer) { - return new IncSendTask(socketChannel.get(), buffer); + return new IncSendTask(buffer); } @@ -376,7 +416,7 @@ * * @throws IOException */ - static protected SocketChannel openChannel(final InetSocketAddress addr) + static private SocketChannel openChannel(final InetSocketAddress addr) throws IOException { final SocketChannel socketChannel = SocketChannel.open(); @@ -410,20 +450,20 @@ * {@link ByteBuffer} to the receiving service on a specified * {@link InetSocketAddress}. */ - protected static class IncSendTask implements Callable<Void> { + protected /*static*/ class IncSendTask implements Callable<Void> { - private final SocketChannel socketChannel; +// private final SocketChannel socketChannel; private final ByteBuffer data; - public IncSendTask(final SocketChannel socketChannel, final ByteBuffer data) { + public IncSendTask(/*final SocketChannel socketChannel, */final ByteBuffer data) { - if (socketChannel == null) - throw new IllegalArgumentException(); +// if (socketChannel == null) +// throw new IllegalArgumentException(); if (data == null) throw new IllegalArgumentException(); - this.socketChannel = socketChannel; +// this.socketChannel = socketChannel; this.data = data; @@ -431,9 +471,21 @@ public Void call() throws Exception { + // defer until we actually run. + final SocketChannel socketChannel = reopenChannel(); + + if (!isRunning()) + throw new RuntimeException("Not Running."); + + if (socketChannel == null) + throw new AssertionError(); + // The #of bytes to transfer. final int remaining = data.remaining(); + if (log.isTraceEnabled()) + log.trace("Will send " + remaining + " bytes"); + try { int nwritten = 0; @@ -447,6 +499,12 @@ * non-blocking mode). IF it blocks, should block until * finished or until this thread is interrupted, e.g., by * shutting down the thread pool on which it is running. + * + * Note: If the SocketChannel is closed by an interrupt, + * then the send request for the [data] payload will fail. + * However, the SocketChannel will be automatically reopened + * for the next request (unless the HASendService has been + * terminated). */ final int nbytes = socketChannel.write(data); @@ -458,6 +516,14 @@ + " of out " + remaining + " written so far"); } + + /* + * The ACK by the receiver divides the HASend requests into + * distinct operations. Without this handshaking, the next + * available payload would be on the way as soon as the last + * byte of the current payload was written. + */ +// awaitAck(socketChannel); } finally { @@ -478,6 +544,145 @@ } +// /** +// * +// * @param socketChannel +// * @throws IOException +// */ +// private void awaitAck(final SocketChannel socketChannel) +// throws IOException { +// +// log.debug("Awaiting (N)ACK"); +// +// // FIXME Optimize. +// final ByteBuffer b = ByteBuffer.wrap(new byte[] { -1 }); +// +// while (socketChannel.isOpen()) { +// +// final int nread = socketChannel.read(b); +// +// if (nread == 1) { +// +// final byte ret = b.array()[0]; +// +// if (ret == ACK) { +// +// // Received ACK. +// log.debug("ACK"); +// return; +// +// } +// +// log.error("NACK"); +// return; +// +// } +// +// throw new IOException("Expecting ACK, not " + nread + " bytes"); +// +// } +// +// // channel is closed. +// throw new AsynchronousCloseException(); +// +//// /* +//// * We should now have parameters ready in the WriteMessage and can +//// * begin transferring data from the stream to the writeCache. +//// */ +//// final long begin = System.currentTimeMillis(); +//// long mark = begin; +//// +//// // #of bytes remaining (to be received). +//// int rem = b.remaining(); +//// +//// // End of stream flag. +//// boolean EOS = false; +//// +//// // for debug retain number of low level reads +//// int reads = 0; +//// +//// while (rem > 0 && !EOS) { +//// +//// // block up to the timeout. +//// final int nkeys = client.clientSelector.select(10000/* ms */); +//// +//// if (nkeys == 0) { +//// +//// /* +//// * Nothing available. +//// */ +//// +//// // time since last mark. +//// final long now = System.currentTimeMillis(); +//// final long elapsed = now - mark; +//// +//// if (elapsed > 10000) { +//// // Issue warning if we have been blocked for a while. +//// log.warn("Blocked: awaiting " + rem + " out of " +//// + message.getSize() + " bytes."); +//// mark = now;// reset mark. +//// } +//// +//// if (!client.client.isOpen() +//// || !client.clientSelector.isOpen()) { +//// +//// /* +//// * The channel has been closed. The request must be +//// * failed. TODO Or set EOF:=true? +//// * +//// * Note: The [callback] is NOT notified. The service +//// * that issued the RMI request to this service to +//// * receive the payload over the HAReceivedService will +//// * see this exception thrown back across the RMI +//// * request. +//// * +//// * @see HAReceiveService.receiveData(). +//// */ +//// +//// throw new AsynchronousCloseException(); +//// +//// } +//// +//// // no keys. nothing to read. +//// continue; +//// +//// } +//// +//// final Set<SelectionKey> keys = client.clientSelector +//// .selectedKeys(); +//// +//// final Iterator<SelectionKey> iter = keys.iterator(); +//// +//// while (iter.hasNext()) { +//// +//// iter.next(); +//// iter.remove(); +//// +//// final int rdlen = client.client.read(b); +//// +//// if (log.isTraceEnabled()) +//// log.trace("Read " + rdlen + " bytes of " +//// + (rdlen > 0 ? rem - rdlen : rem) +//// + " bytes remaining."); +//// +//// if (rdlen > 0) { +//// reads++; +//// } +//// +//// if (rdlen == -1) { +//// // The stream is closed? +//// EOS = true; +//// break; +//// } +//// +//// rem -= rdlen; +//// +//// } +//// +//// } // while( rem > 0 ) +// +// } + } } Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestAll.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestAll.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestAll.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -64,6 +64,9 @@ // Test of HASendService and HAReceiveService (2 nodes). suite.addTestSuite(TestHASendAndReceive.class); + // Test of HASendService and HAReceiveService w/ interrupts (2 nodes). + suite.addTestSuite(TestHASendAndReceiveInterrupts.class); + // Test of HASendService and HAReceiveService (3 nodes). suite.addTestSuite(TestHASendAndReceive3Nodes.class); Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -24,10 +24,7 @@ package com.bigdata.ha.pipeline; -import java.io.IOException; -import java.net.BindException; import java.net.InetSocketAddress; -import java.net.ServerSocket; import java.nio.ByteBuffer; import java.util.Random; import java.util.concurrent.ExecutionException; @@ -37,8 +34,6 @@ import com.bigdata.ha.msg.HAWriteMessageBase; import com.bigdata.ha.msg.IHAWriteMessageBase; -import com.bigdata.ha.pipeline.HAReceiveService; -import com.bigdata.ha.pipeline.HASendService; import com.bigdata.io.DirectBufferPool; import com.bigdata.io.IBufferAccess; import com.bigdata.io.TestCase3; @@ -52,64 +47,6 @@ */ public class TestHASendAndReceive extends TestCase3 { - /** - * A random number generated - the seed is NOT fixed. - */ - private final Random r = new Random(); - -// /** -// * Returns random data that will fit in N bytes. N is chosen randomly in -// * 1:256. -// * -// * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> of -// * random length and having random contents. -// */ -// public ByteBuffer getRandomData() { -// -// final int nbytes = r.nextInt(256) + 1; -// -// return getRandomData(nbytes); -// -// } - - /** - * Returns random data that will fit in <i>nbytes</i>. - * - * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> - * having random contents. - */ - private ByteBuffer getRandomData(final int nbytes) { - - final byte[] bytes = new byte[nbytes]; - - r.nextBytes(bytes); - - return ByteBuffer.wrap(bytes); - - } - - /** - * Returns random data that will fit in <i>nbytes</i>. - * - * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> - * having random contents. - */ - private ByteBuffer getRandomData(final ByteBuffer b, final int nbytes) { - - final byte[] a = new byte[nbytes]; - - r.nextBytes(a); - - b.limit(nbytes); - b.position(0); - b.put(a); - - b.flip(); - - return b; - - } - public TestHASendAndReceive() { } @@ -124,8 +61,11 @@ private HAReceiveService<IHAWriteMessageBase> receiveService; private ChecksumUtility chk; + @Override protected void setUp() throws Exception { + super.setUp(); + r = new Random(); chk = new ChecksumUtility(); /* @@ -149,8 +89,11 @@ } + @Override protected void tearDown() throws Exception { + super.tearDown(); + if (receiveService != null) { receiveService.terminate(); receiveService = null; @@ -167,35 +110,6 @@ } /** - * Return an open port on current machine. Try the suggested port first. If - * suggestedPort is zero, just select a random port - */ - private static int getPort(int suggestedPort) throws IOException { - - ServerSocket openSocket; - try { - openSocket = new ServerSocket(suggestedPort); - } catch (BindException ex) { - // the port is busy, so look for a random open port - openSocket = new ServerSocket(0); - } - - final int port = openSocket.getLocalPort(); - - openSocket.close(); - - if (suggestedPort != 0 && port != suggestedPort) { - - log.warn("suggestedPort is busy: suggestedPort=" + suggestedPort - + ", using port=" + port + " instead"); - - } - - return port; - - } - - /** * Should we expect concurrency of the Socket send and RMI? It seems that we * should be able to handle it whatever the logical argument. The only * constraint should be on the processing of each pair of socket/RMI @@ -217,16 +131,6 @@ final ByteBuffer rcv = ByteBuffer.allocate(2000); final Future<Void> futRec = receiveService.receiveData(msg1, rcv); final Future<Void> futSnd = sendService.send(tst1); -// while (!futSnd.isDone() && !futRec.isDone()) { -// try { -// futSnd.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// try { -// futRec.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// } futSnd.get(timeout,TimeUnit.MILLISECONDS); futRec.get(timeout,TimeUnit.MILLISECONDS); assertEquals(tst1, rcv); @@ -238,16 +142,6 @@ final ByteBuffer rcv2 = ByteBuffer.allocate(2000); final Future<Void> futSnd = sendService.send(tst2); final Future<Void> futRec = receiveService.receiveData(msg2, rcv2); -// while (!futSnd.isDone() && !futRec.isDone()) { -// try { -// futSnd.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// try { -// futRec.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// } futSnd.get(timeout,TimeUnit.MILLISECONDS); futRec.get(timeout,TimeUnit.MILLISECONDS); assertEquals(tst2, rcv2); @@ -275,16 +169,6 @@ // FutureTask return ensures remote ready for Socket data final Future<Void> futRec = receiveService.receiveData(msg, rcv); final Future<Void> futSnd = sendService.send(tst); -// while (!futSnd.isDone() && !futRec.isDone()) { -// try { -// futSnd.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignored) { -// } -// try { -// futRec.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignored) { -// } -// } futSnd.get(timeout,TimeUnit.MILLISECONDS); futRec.get(timeout,TimeUnit.MILLISECONDS); assertEquals(tst, rcv); // make sure buffer has been transmitted @@ -316,16 +200,6 @@ // FutureTask return ensures remote ready for Socket data final Future<Void> futRec = receiveService.receiveData(msg, rcv); final Future<Void> futSnd = sendService.send(tst); -// while (!futSnd.isDone() && !futRec.isDone()) { -// try { -// futSnd.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignored) { -// } -// try { -// futRec.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignored) { -// } -// } futSnd.get(timeout,TimeUnit.MILLISECONDS); futRec.get(timeout,TimeUnit.MILLISECONDS); // make sure buffer has been transmitted Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive3Nodes.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive3Nodes.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceive3Nodes.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -25,11 +25,8 @@ package com.bigdata.ha.pipeline; import java.io.IOException; -import java.net.BindException; import java.net.InetSocketAddress; -import java.net.ServerSocket; import java.nio.ByteBuffer; -import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -37,8 +34,6 @@ import com.bigdata.ha.msg.HAWriteMessageBase; import com.bigdata.ha.msg.IHAWriteMessageBase; -import com.bigdata.ha.pipeline.HAReceiveService; -import com.bigdata.ha.pipeline.HASendService; import com.bigdata.io.DirectBufferPool; import com.bigdata.io.IBufferAccess; import com.bigdata.io.TestCase3; @@ -54,66 +49,8 @@ */ public class TestHASendAndReceive3Nodes extends TestCase3 { - private ChecksumUtility chk = new ChecksumUtility(); + private ChecksumUtility chk; - /** - * A random number generated - the seed is NOT fixed. - */ - private Random r = new Random(); - -// /** -// * Returns random data that will fit in N bytes. N is chosen randomly in -// * 1:256. -// * -// * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> of -// * random length and having random contents. -// */ -// public ByteBuffer getRandomData() { -// -// final int nbytes = r.nextInt(256) + 1; -// -// return getRandomData(nbytes); -// -// } - - /** - * Returns random data that will fit in <i>nbytes</i>. - * - * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> - * having random contents. - */ - private ByteBuffer getRandomData(final int nbytes) { - - final byte[] bytes = new byte[nbytes]; - - r.nextBytes(bytes); - - return ByteBuffer.wrap(bytes); - - } - - /** - * Returns random data that will fit in <i>nbytes</i>. - * - * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> - * having random contents. - */ - private ByteBuffer getRandomData(final ByteBuffer b, final int nbytes) { - - final byte[] a = new byte[nbytes]; - - r.nextBytes(a); - - b.limit(nbytes); - b.position(0); - b.put(a); - - b.flip(); - - return b; - - } - public TestHASendAndReceive3Nodes() { } @@ -141,6 +78,10 @@ @Override protected void setUp() throws Exception { + super.setUp(); + + chk = new ChecksumUtility(); + /* * Setup C at the end of the pipeline. */ @@ -201,6 +142,8 @@ @Override protected void tearDown() throws Exception { + super.tearDown(); + if (receiveServiceB != null) { receiveServiceB.terminate(); receiveServiceB = null; @@ -219,38 +162,8 @@ chk = null; - r = null; - } - /** - * Return an open port on current machine. Try the suggested port first. If - * suggestedPort is zero, just select a random port - */ - private static int getPort(int suggestedPort) throws IOException { - - ServerSocket openSocket; - try { - openSocket = new ServerSocket(suggestedPort); - } catch (BindException ex) { - // the port is busy, so look for a random open port - openSocket = new ServerSocket(0); - } - - final int port = openSocket.getLocalPort(); - - openSocket.close(); - - if (suggestedPort != 0 && port != suggestedPort) { - - log.warn("suggestedPort is busy: suggestedPort=" + suggestedPort + ", using port=" + port + " instead"); - - } - - return port; - - } - public void testSimpleExchange() throws InterruptedException, ExecutionException, TimeoutException { @@ -263,20 +176,6 @@ final Future<Void> futRec1 = receiveServiceB.receiveData(msg1, rcv1); final Future<Void> futRec2 = receiveServiceC.receiveData(msg1, rcv2); final Future<Void> futSnd = sendServiceA.send(tst1); -// while (!futSnd.isDone() && !futRec2.isDone()) { -// try { -// futSnd.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// try { -// futRec1.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// try { -// futRec2.get(10L, TimeUnit.MILLISECONDS); -// } catch (TimeoutException ignore) { -// } -// } futSnd.get(timeout,TimeUnit.MILLISECONDS); futRec1.get(timeout,TimeUnit.MILLISECONDS); futRec2.get(timeout,TimeUnit.MILLISECONDS); @@ -300,7 +199,9 @@ futSnd.get(10L, TimeUnit.MILLISECONDS); } catch (TimeoutException ignore) { } catch (ExecutionException e) { - assertTrue(e.getCause().getMessage().equals("Checksum Error")); + if (!InnerCause.isInnerCause(e, ChecksumError.class)) { + fail("Expecting " + ChecksumError.class + ", not " + e, e); + } } try { futRec2.get(10L, TimeUnit.MILLISECONDS); Added: branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceiveInterrupts.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceiveInterrupts.java (rev 0) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/ha/pipeline/TestHASendAndReceiveInterrupts.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -0,0 +1,270 @@ +/** + +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 + */ + +package com.bigdata.ha.pipeline; + +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.channels.AsynchronousCloseException; +import java.nio.channels.SocketChannel; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; + +import com.bigdata.ha.msg.HAWriteMessageBase; +import com.bigdata.ha.msg.IHAWriteMessage; +import com.bigdata.ha.msg.IHAWriteMessageBase; +import com.bigdata.ha.pipeline.HAReceiveService.IHAReceiveCallback; +import com.bigdata.io.DirectBufferPool; +import com.bigdata.io.TestCase3; +import com.bigdata.util.ChecksumUtility; + +/** + * Test the raw socket protocol implemented by {@link HASendService} and + * {@link HAReceiveService}. This test suite looks at the interrupt handling + * behavior. Interrupts during an IO cause the {@link SocketChannel} to be + * closed. If the service is still open, then we need to transparently re-open + * the socket channel. + * <p> + * Interrupts wind up closing the {@link HASendService} side of the channel when + * a remote service makes a request to the quorum leader to send some data along + * the write pipeline, and then cancels that operation. For example, when a + * service needs to resynchronize with the quorum it will request either HALogs + * or a snapshot from the leader. Those data are sent over the write pipeline. + * There may be concurrent requests and data transfers in progress. Those + * requests will be serialized (one raw buffer of at a time). The + * {@link IHAWriteMessage}s for different requests (and their payloads) can be + * interleaved, but each {@link IHAWriteMessage} and payload is transmitted + * atomically (the packets sent along the socket pertain to only a single + * payload / {@link IHAWriteMessage}). When the service has caught up, if it + * enters an error state, or if it shuts down, then the service will cancel the + * remote future on the leader for the request (e.g., a "sendHALog()" request). + * This can cause an interrupt of a blocked {@link SocketChannel} IO and that + * will cause the {@link SocketChannel} to be closed asynchronously. + * <p> + * Both the {@link HASendService} and the {@link HAReceiveService} need to + * notice the asynchronous {@link SocketChannel} close and cure it (unless they + * are terminated or terminating). The {@link HAReceiveService} needs to + * propagate the {@link AsynchronousCloseException} through the + * {@link IHAReceiveCallback} and drop the paylaod. The {@link HASendService} + * needs to drop the payload and re-open the {@link SocketChannel}. The next + * {@link IHAWriteMessage} and payload should be traversed normally once the + * {@link SocketChannel} has been re-opened. + * + * @author martyn Cutcher + */ +public class TestHASendAndReceiveInterrupts extends TestCase3 { + + public TestHASendAndReceiveInterrupts() { + + } + + public TestHASendAndReceiveInterrupts(String name) { + + super(name); + + } + + private class MyCallback implements IHAReceiveCallback<IHAWriteMessageBase> { + + @Override + public void callback(IHAWriteMessageBase msg, ByteBuffer data) + throws Exception { + // TODO Auto-generated method stub + + } + + } + + private HASendService sendService; + private HAReceiveService<IHAWriteMessageBase> receiveService; + private ChecksumUtility chk; + private IHAReceiveCallback<IHAWriteMessageBase> callback; + + @Override + protected void setUp() throws Exception { + + super.setUp(); + + chk = new ChecksumUtility(); + + /* + * Note: ZERO (0) indicates that a random free port will be selected. If + * you use a fixed port then there is a danger that the port will not be + * able to be reopened immediately after it has been closed, in which + * case you will see something like: "bind address already in use". + */ + final int port = getPort(0);// 3000 + + if (log.isInfoEnabled()) + log.info("Using port=" + port); + + final InetSocketAddress addr = new InetSocketAddress(port); + + callback = new MyCallback(); + + receiveService = new HAReceiveService<IHAWriteMessageBase>(addr, + null/* nextAddr */, callback); + receiveService.start(); + + sendService = new HASendService(); + sendService.start(addr); + + } + + @Override + protected void tearDown() throws Exception { + + super.tearDown(); + + if (receiveService != null) { + receiveService.terminate(); + receiveService = null; + } + + if (sendService != null) { +// sendService.closeIncSend(); + sendService.terminate(); + sendService = null; + } + + chk = null; + + } + + public void testRecoverFromInterrupts() throws InterruptedException, + ExecutionException, TimeoutException { + + // replace receiveService with one with callback + final InetSocketAddress addrSelf = receiveService.getAddrSelf(); + final InetSocketAddress addrNext = null; + + receiveService.terminate(); + + // Callback will check fail value to determine if task should be + // interrupted + final AtomicReference<Future<Void>> receiveFail = new AtomicReference<Future<Void>>( + null); + + receiveService = new HAReceiveService<IHAWriteMessageBase>(addrSelf, + addrNext, new IHAReceiveCallback<IHAWriteMessageBase>() { + public void callback(final IHAWriteMessageBase msg, + final ByteBuffer data) throws Exception { + final Future<Void> task = receiveFail.get(); + if (task != null) { + task.cancel(true); + } + } + }); + + receiveService.start(); + + /* + * Note: Must be larger than the socket buffer / packet size or the 2nd + * message will get buffered before we can interrupt it. + * + * FIXME The problem here is that we are queuing the 2nd message + * immediately on the HASendService's Executor. The actual IncSendTask + * does not wait once it has finished deliverying data to the socket + * channel. It immediately exits, and the data transfer for the next + * queued request begins. In order to avoid messing up the payload for + * the next request, we might need to wait on the Future of the remote + * ReceiveService.receiveData() task before terminating the IncSendTask + * (or before scheduling the next one). Otherwise we may not be in a + * position where we can figure out whether or not to restart a given + * payload. If the Send Future (of the IncSendTask) was cancelled, then + * we want to drop the payload associated with that specific Future. + */ + final int msgSize = 256 + r.nextInt(1024); + final int receiveBufSize = msgSize + r.nextInt(128); + + final long timeout = 5000;// ms + { + // 1st xfer. + final ByteBuffer tst1 = getRandomData(msgSize); + final IHAWriteMessageBase msg1 = new HAWriteMessageBase(msgSize, + chk.checksum(tst1)); + final ByteBuffer rcv1 = ByteBuffer.allocate(receiveBufSize); + final Future<Void> futSnd1 = sendService.send(tst1); + + // 2nd xfer. + final ByteBuffer tst2 = getRandomData(msgSize); + final IHAWriteMessageBase msg2 = new HAWriteMessageBase(msgSize, + chk.checksum(tst2)); + final ByteBuffer rcv2 = ByteBuffer.allocate(receiveBufSize); + final Future<Void> futSnd2 = sendService.send(tst2); + + // We will interrupt the 2nd send. + receiveFail.set(futSnd2); + + // 3rd xfer. + final ByteBuffer tst3 = getRandomData(msgSize); + final IHAWriteMessageBase msg3 = new HAWriteMessageBase(msgSize, + chk.checksum(tst3)); + final ByteBuffer rcv3 = ByteBuffer.allocate(receiveBufSize); + final Future<Void> futSnd3 = sendService.send(tst3); + + final Future<Void> futRec1 = receiveService.receiveData(msg1, rcv1); + + futSnd2.get(); // should throw exception UNLESS IO done before + final Future<Void> futRec2 = receiveService.receiveData(msg2, rcv2); + + final Future<Void> futRec3 = receiveService.receiveData(msg3, rcv2); + + // first send. should be good. + futSnd1.get(timeout,TimeUnit.MILLISECONDS); + futRec1.get(timeout,TimeUnit.MILLISECONDS); + assertEquals(tst1, rcv1); + + // seecond 2nd. should be cancelled. + futSnd2.get(timeout,TimeUnit.MILLISECONDS); + futRec2.get(timeout,TimeUnit.MILLISECONDS); + assertEquals(tst1, rcv2); + + // 3rd send. should be good. TODO Longer sequences w/ variable message sizes to hit various timings. + futSnd3.get(timeout,TimeUnit.MILLISECONDS); + futRec3.get(timeout,TimeUnit.MILLISECONDS); + assertEquals(tst3, rcv3); + + } + + } + + public void testStressRecoverFromInterrupts() throws Exception { + tearDown(); + for (int i = 0; i < 100; i++) { + setUp(); + try { + testRecoverFromInterrupts(); + } catch (Throwable t) { + fail("Fail on pass " + i + " : " + t, t); + } finally { + tearDown(); + } + } + } + +} Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/io/TestCase3.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/io/TestCase3.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/io/TestCase3.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -27,7 +27,11 @@ package com.bigdata.io; +import java.io.IOException; +import java.net.BindException; +import java.net.ServerSocket; import java.nio.ByteBuffer; +import java.util.Random; import java.util.concurrent.TimeUnit; import junit.framework.AssertionFailedError; @@ -46,7 +50,13 @@ */ public class TestCase3 extends TestCase2 { + /** + * A random number generated - the seed is NOT fixed. + */ + protected Random r; + + /** * */ public TestCase3() { @@ -61,10 +71,19 @@ } + @Override + protected void setUp() throws Exception { + + r = new Random(); + + } + protected void tearDown() throws Exception { super.tearDown(); + r = null; + TestHelper.checkJournalsClosed(this); } @@ -253,4 +272,70 @@ } + /** + * Returns random data that will fit in <i>nbytes</i>. + * + * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> + * having random contents. + */ + protected ByteBuffer getRandomData(final int nbytes) { + + final byte[] bytes = new byte[nbytes]; + + r.nextBytes(bytes); + + return ByteBuffer.wrap(bytes); + + } + + /** + * Returns random data that will fit in <i>nbytes</i>. + * + * @return A new {@link ByteBuffer} wrapping a new <code>byte[]</code> + * having random contents. + */ + protected ByteBuffer getRandomData(final ByteBuffer b, final int nbytes) { + + final byte[] a = new byte[nbytes]; + + r.nextBytes(a); + + b.limit(nbytes); + b.position(0); + b.put(a); + + b.flip(); + + return b; + + } + + /** + * Return an open port on current machine. Try the suggested port first. If + * suggestedPort is zero, just select a random port + */ + protected static int getPort(int suggestedPort) throws IOException { + + ServerSocket openSocket; + try { + openSocket = new ServerSocket(suggestedPort); + } catch (BindException ex) { + // the port is busy, so look for a random open port + openSocket = new ServerSocket(0); + } + + final int port = openSocket.getLocalPort(); + + openSocket.close(); + + if (suggestedPort != 0 && port != suggestedPort) { + + log.warn("suggestedPort is busy: suggestedPort=" + suggestedPort + ", using port=" + port + " instead"); + + } + + return port; + + } + } Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/io/writecache/TestWORMWriteCacheService.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -29,9 +29,7 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; -import java.net.BindException; import java.net.InetSocketAddress; -import java.net.ServerSocket; import java.nio.ByteBuffer; import java.nio.channels.Channel; import java.nio.channels.FileChannel; @@ -2728,33 +2726,33 @@ * HA pipeline tests. */ - /** - * Return an open port on current machine. Try the suggested port first. If - * suggestedPort is zero, just select a random port - */ - private static int getPort(final int suggestedPort) throws IOException { +// /** +// * Return an open port on current machine. Try the suggested port first. If +// * suggestedPort is zero, just select a random port +// */ +// private static int getPort(final int suggestedPort) throws IOException { +// +// ServerSocket openSocket; +// try { +// openSocket = new ServerSocket(suggestedPort); +// } catch (BindException ex) { +// // the port is busy, so look for a random open port +// openSocket = new ServerSocket(0); +// } +// +// final int port = openSocket.getLocalPort(); +// +// openSocket.close(); +// +// if (suggestedPort != 0 && port != suggestedPort) { +// +// log.warn("suggestedPort is busy: suggestedPort=" + suggestedPort +// + ", using port=" + port + " instead"); +// +// } +// +// return port; +// +// } - ServerSocket openSocket; - try { - openSocket = new ServerSocket(suggestedPort); - } catch (BindException ex) { - // the port is busy, so look for a random open port - openSocket = new ServerSocket(0); - } - - final int port = openSocket.getLocalPort(); - - openSocket.close(); - - if (suggestedPort != 0 && port != suggestedPort) { - - log.warn("suggestedPort is busy: suggestedPort=" + suggestedPort - + ", using port=" + port + " instead"); - - } - - return port; - - } - } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-10 09:07:45 UTC (rev 7039) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-10 17:35:17 UTC (rev 7040) @@ -771,6 +771,7 @@ final IBufferAccess buf = DirectBufferPool.INSTANCE.acquire(); long nsent = 0; + boolean success = false; try { while (r.hasMoreBuffers()) { @@ -792,23 +793,48 @@ try { // wait for message to make it through the pipeline. ft.get(); + nsent++; } finally { - ft.cancel(true/* mayInterruptIfRunning */); + /* + * DO NOT PROPAGATE THE INTERRUPT TO THE PIPELINE. + * + * Note: Either the Future isDone... [truncated message content] |
From: <tho...@us...> - 2013-04-11 17:18:16
|
Revision: 7043 http://bigdata.svn.sourceforge.net/bigdata/?rev=7043&view=rev Author: thompsonbry Date: 2013-04-11 17:18:07 +0000 (Thu, 11 Apr 2013) Log Message: ----------- TimeUnit.DAYS.toMillis() does not work inside of a Configuration file. Updated configuration file and added ConfigMath.d2ms(). Added "MUST" to synchronized clocks in the readme. Modified Paths: -------------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/jini/util/ConfigMath.java branches/READ_CACHE/src/resources/HAJournal/HAJournal.config branches/READ_CACHE/src/resources/HAJournal/README Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/jini/util/ConfigMath.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/jini/util/ConfigMath.java 2013-04-10 17:37:44 UTC (rev 7042) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/jini/util/ConfigMath.java 2013-04-11 17:18:07 UTC (rev 7043) @@ -148,6 +148,20 @@ } /** + * Convert days to milliseconds. + * + * @param d + * days + * + * @return Milliseconds. + */ + public static long d2ms(final long m) { + + return TimeUnit.DAYS.toMillis(m); + + } + + /** * Return the absolute path for the file. * * @param file Modified: branches/READ_CACHE/src/resources/HAJournal/HAJournal.config =================================================================== --- branches/READ_CACHE/src/resources/HAJournal/HAJournal.config 2013-04-10 17:37:44 UTC (rev 7042) +++ branches/READ_CACHE/src/resources/HAJournal/HAJournal.config 2013-04-11 17:18:07 UTC (rev 7043) @@ -45,8 +45,6 @@ import com.bigdata.rdf.lexicon.LexiconRelation; import com.bigdata.rdf.lexicon.LexiconKeyOrder; import com.bigdata.rawstore.Bytes; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeUnit.*; /* * This is a sample configuration file for a highly available Journal. A @@ -105,7 +103,7 @@ private static snapshotPolicy = new DefaultSnapshotPolicy(200/*hhmm*/,20/*percent*/); // Restore policy - private static restorePolicy = new DefaultRestorePolicy(TimeUnit.DAYS.toMillis(7)); + private static restorePolicy = new DefaultRestorePolicy(ConfigMath.d2ms(7)); // one federation, multicast discovery. //static private groups = LookupDiscovery.ALL_GROUPS; Modified: branches/READ_CACHE/src/resources/HAJournal/README =================================================================== --- branches/READ_CACHE/src/resources/HAJournal/README 2013-04-10 17:37:44 UTC (rev 7042) +++ branches/READ_CACHE/src/resources/HAJournal/README 2013-04-11 17:18:07 UTC (rev 7043) @@ -6,7 +6,7 @@ The basic procedure is: -0. The nodes should have synchronized clocks, both for logging and to ensure +0. The nodes MUST have synchronized clocks, both for logging and to ensure that the transaction services have closely correlated clocks for assigning timestamps. Make sure that ntp or a similar service is running to synchronize the clocks. This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-12 12:43:11
|
Revision: 7045 http://bigdata.svn.sourceforge.net/bigdata/?rev=7045&view=rev Author: thompsonbry Date: 2013-04-12 12:43:04 +0000 (Fri, 12 Apr 2013) Log Message: ----------- Always start the scheduled executor service (not just if we are collecting platform statistics). Bug fix to the delay for the DefaultSnapshotPolicy. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-12 11:54:44 UTC (rev 7044) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-12 12:43:04 UTC (rev 7045) @@ -337,19 +337,19 @@ .getName() + ".executorService")); - if (Boolean.valueOf(properties.getProperty( - Options.COLLECT_QUEUE_STATISTICS, - Options.DEFAULT_COLLECT_QUEUE_STATISTICS))) { +// if (Boolean.valueOf(properties.getProperty( +// Options.COLLECT_QUEUE_STATISTICS, +// Options.DEFAULT_COLLECT_QUEUE_STATISTICS))) { scheduledExecutorService = Executors .newSingleThreadScheduledExecutor(new DaemonThreadFactory( getClass().getName() + ".sampleService")); - } else { - - scheduledExecutorService = null; - - } +// } else { +// +// scheduledExecutorService = null; +// +// } { @@ -2202,8 +2202,7 @@ /** * Used to sample and report on the queue associated with the - * {@link #executorService} and <code>null</code> if we will not be - * collecting data on task execution. + * {@link #executorService}. May be used to schedule other tasks as well. */ private final ScheduledExecutorService scheduledExecutorService; Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-12 11:54:44 UTC (rev 7044) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-12 12:43:04 UTC (rev 7045) @@ -49,6 +49,34 @@ final private int timeOfDay; final private int percentLogSize; + /** + * Convert a delay in milliseconds into an <code>hh:mm</code> format string + * representing the hours and minutes remaining in that delay. + * + * @param delay + * The delay in milliseconds. + * + * @return The string representation of the countdown. + */ + public static String toHHMM(final long delay) { + + final int hh = (int) TimeUnit.MILLISECONDS.toHours(delay); + final int mm = (int) (TimeUnit.MILLISECONDS.toMinutes(delay) % 60L); + + final StringBuilder sb = new StringBuilder(); + + final Formatter f = new Formatter(sb); + + f.format("%02d:%02d", hh, mm); + f.flush(); + f.close(); + + final String str = sb.toString(); + + return str; + + } + @Override public String toString() { @@ -62,8 +90,12 @@ final String todStr = sb.toString(); - return DefaultSnapshotPolicy.class.getSimpleName() + "{timeOfDay=" - + todStr + ", percentLogSize=" + percentLogSize + "%}"; + return DefaultSnapshotPolicy.class.getSimpleName() // + + "{timeOfDay=" + todStr // + + ", percentLogSize=" + percentLogSize + "%}" + // how long until the next snapshot check. + + ", countdown=" + toHHMM(delay(timeOfDay) + ); } @@ -105,17 +137,17 @@ public void init(final HAJournal jnl) { + // delay until next run (ms). final long initialDelay = delay(timeOfDay); + // run once per day. + final long delay = TimeUnit.DAYS.toMillis(1); + if (log.isInfoEnabled()) - log.info("initialDelay=" + initialDelay + "ms" + " (hours=" - + TimeUnit.MILLISECONDS.toHours(initialDelay) - + ", minutes=" - + (TimeUnit.MILLISECONDS.toMinutes(initialDelay) % 60L) - + ")"); + log.info("initialDelay=" + toHHMM(initialDelay)); - jnl.addScheduledTask(new SnapshotTask(jnl), initialDelay, 1/* delay */, - TimeUnit.DAYS); + jnl.addScheduledTask(new SnapshotTask(jnl), initialDelay, delay, + TimeUnit.MILLISECONDS); } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-15 13:22:23
|
Revision: 7049 http://bigdata.svn.sourceforge.net/bigdata/?rev=7049&view=rev Author: thompsonbry Date: 2013-04-15 13:22:11 +0000 (Mon, 15 Apr 2013) Log Message: ----------- Merge from the development branch to catch up on some significant changes in Journal (decoupling Ganglia). {{{ At revision 7048. merge https://bigdata.svn.sourceforge.net/svnroot/bigdata/branches/BIGDATA_RELEASE_1_2_0 /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD --- Merging r7037 through r7048 into /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/Journal.java A /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/PlatformStatsPlugIn.java A /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/IPlugIn.java A /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/QueueStatsPlugIn.java A /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/HttpPlugin.java A /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE_HEAD/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java Merge complete. ===== File Statistics: ===== Added: 5 Updated: 1 }}} Revision Links: -------------- http://bigdata.svn.sourceforge.net/bigdata/?rev=7048&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7037&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7048&view=rev Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java Added Paths: ----------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/HttpPlugin.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IPlugIn.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/PlatformStatsPlugIn.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/QueueStatsPlugIn.java Property Changed: ---------------- branches/READ_CACHE/ branches/READ_CACHE/bigdata/lib/jetty/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba/ branches/READ_CACHE/bigdata/src/java/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/test/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/util/httpd/ branches/READ_CACHE/bigdata-compatibility/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/attr/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/disco/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/util/config/ branches/READ_CACHE/bigdata-perf/ branches/READ_CACHE/bigdata-perf/btc/ branches/READ_CACHE/bigdata-perf/btc/src/resources/ branches/READ_CACHE/bigdata-perf/lubm/ branches/READ_CACHE/bigdata-perf/uniprot/ branches/READ_CACHE/bigdata-perf/uniprot/src/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/changesets/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/error/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/util/ branches/READ_CACHE/bigdata-rdf/src/samples/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/changesets/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/dsi-utils/ branches/READ_CACHE/dsi-utils/LEGAL/ branches/READ_CACHE/dsi-utils/lib/ branches/READ_CACHE/dsi-utils/src/ branches/READ_CACHE/dsi-utils/src/java/ branches/READ_CACHE/dsi-utils/src/java/it/ branches/READ_CACHE/dsi-utils/src/java/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/dsi/ branches/READ_CACHE/lgpl-utils/src/java/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/lgpl-utils/src/test/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/osgi/ branches/READ_CACHE/src/resources/bin/config/ Property changes on: branches/READ_CACHE ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-7037 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-7048 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata/lib/jetty ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-7037 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-7048 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-7037 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 Copied: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java (from rev 7048, branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java) =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/GangliaPlugIn.java 2013-04-15 13:22:11 UTC (rev 7049) @@ -0,0 +1,404 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.journal; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.Properties; +import java.util.concurrent.FutureTask; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.log4j.Logger; + +import com.bigdata.counters.AbstractStatisticsCollector; +import com.bigdata.counters.ganglia.BigdataGangliaService; +import com.bigdata.counters.ganglia.BigdataMetadataFactory; +import com.bigdata.counters.ganglia.HostMetricsCollector; +import com.bigdata.counters.ganglia.QueryEngineMetricsCollector; +import com.bigdata.ganglia.DefaultMetadataFactory; +import com.bigdata.ganglia.GangliaMetadataFactory; +import com.bigdata.ganglia.GangliaService; +import com.bigdata.ganglia.GangliaSlopeEnum; +import com.bigdata.ganglia.IGangliaDefaults; +import com.bigdata.ganglia.util.GangliaUtil; + +/** + * A plugin for ganglia. + * <p> + * Note: This plugin will not start (and will not be loaded from the classpath) + * unless {@link PlatformStatsPlugIn.Options#COLLECT_PLATFORM_STATISTICS} is set to + * <code>true</code>. + * + * @see <a href="https://sourceforge.net/apps/trac/bigdata/ticket/441"> Ganglia + * Integration</a> + * @see <a href="https://sourceforge.net/apps/trac/bigdata/ticket/609"> + * bigdata-ganglia is required dependency for Journal </a> + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public class GangliaPlugIn implements IPlugIn<Journal, GangliaService> { + + private static final Logger log = Logger.getLogger(GangliaPlugIn.class); + + /** + * Configuration options. + */ + public interface Options { + + // Listen + + /** + * The multicast group used to join the ganglia performance monitoring + * network. + */ + String GANGLIA_LISTEN_GROUP = Journal.class.getName() + + ".ganglia.listenGroup"; + + String DEFAULT_GANGLIA_LISTEN_GROUP = IGangliaDefaults.DEFAULT_GROUP; + + /** + * The port for the multicast group used to join the ganglia performance + * monitoring network. + */ + String GANGLIA_LISTEN_PORT = Journal.class.getName() + + ".ganglia.listenPort"; + + String DEFAULT_GANGLIA_LISTEN_PORT = Integer + .toString(IGangliaDefaults.DEFAULT_PORT); + + /** + * When <code>true</code>, the embedded {@link GangliaService} will + * listen on to the specified multicast group and build up an internal + * model of the metrics in the ganglia network. + * <p> + * Note: If both {@link #GANGLIA_LISTEN} and {@link #GANGLIA_REPORT} are + * <code>false</code> then the embedded {@link GangliaService} will not + * be started. + */ + String GANGLIA_LISTEN = Journal.class.getName() + + ".ganglia.listen"; + + String DEFAULT_GANGLIA_LISTEN = "false"; + + // Report + + /** + * When <code>true</code>, the embedded {@link GangliaService} will + * report performance metrics to the specified gmetad server(s). + * <p> + * Note: If both {@link #GANGLIA_LISTEN} and {@link #GANGLIA_REPORT} are + * <code>false</code> then the embedded {@link GangliaService} will not + * be started. + */ + String GANGLIA_REPORT = Journal.class.getName() + + ".ganglia.report"; + + String DEFAULT_GANGLIA_REPORT = "false"; + + /** + * An list of the metric servers (<code>gmetad</code> instances) to + * which metrics will be sent. The default is to send metrics to the + * well known multicast group for ganglia. Zero or more hosts may be + * specified, separated by whitespace or commas. The port for each host + * is optional and defaults to the well known port for ganglia. Each + * host may be either a unicast address or a multicast group. + */ + String GANGLIA_SERVERS = Journal.class.getName() + + ".ganglia.servers"; + + String DEFAULT_GANGLIA_SERVERS = IGangliaDefaults.DEFAULT_GROUP; + + /** + * The delay between reports of performance counters in milliseconds ( + * {@value #DEFAULT_REPORT_DELAY}). When ZERO (0L), performance counter + * reporting will be disabled. + * + * @see #DEFAULT_REPORT_DELAY + */ + String REPORT_DELAY = Journal.class.getName() + ".reportDelay"; + + /** + * The default {@link #REPORT_DELAY}. + */ + String DEFAULT_REPORT_DELAY = "" + (60 * 1000); + + } + + /** + * Future for an embedded {@link GangliaService} which listens to + * <code>gmond</code> instances and other {@link GangliaService}s and + * reports out metrics from {@link #getCounters()} to the ganglia network. + */ + private final AtomicReference<FutureTask<Void>> gangliaFuture = new AtomicReference<FutureTask<Void>>(); + + /** + * The embedded ganglia peer. + */ + private final AtomicReference<BigdataGangliaService> gangliaService = new AtomicReference<BigdataGangliaService>(); + + /** + * {@inheritDoc} + * <P> + * Start embedded Ganglia peer. It will develop a snapshot of the + * metrics in memory for all nodes reporting in the ganglia network + * and will self-report metrics from the performance counter + * hierarchy to the ganglia network. + */ + @Override + public void startService(final Journal journal) { + + final AbstractStatisticsCollector statisticsCollector = journal + .getPlatformStatisticsCollector(); + + if (statisticsCollector == null) + return; + + final Properties properties = journal.getProperties(); + + final boolean listen = Boolean.valueOf(properties.getProperty( + Options.GANGLIA_LISTEN, Options.DEFAULT_GANGLIA_LISTEN)); + + final boolean report = Boolean.valueOf(properties.getProperty( + Options.GANGLIA_REPORT, Options.DEFAULT_GANGLIA_REPORT)); + + if (!listen && !report) + return; + + try { + + final String hostName = AbstractStatisticsCollector.fullyQualifiedHostName; + + /* + * Note: This needs to be the value reported by the statistics + * collector since that it what makes it into the counter set + * path prefix for this service. + * + * TODO This implies that we can not enable the embedded ganglia + * peer unless platform level statistics collection is enabled. + * We should be able to separate out the collection of host + * metrics from whether or not we are collecting metrics from + * the bigdata service. Do this when moving the host and process + * (pidstat) collectors into the bigdata-ganglia module. + */ + final String serviceName = statisticsCollector.getProcessName(); + + final InetAddress listenGroup = InetAddress + .getByName(properties.getProperty( + Options.GANGLIA_LISTEN_GROUP, + Options.DEFAULT_GANGLIA_LISTEN_GROUP)); + + final int listenPort = Integer.valueOf(properties.getProperty( + Options.GANGLIA_LISTEN_PORT, + Options.DEFAULT_GANGLIA_LISTEN_PORT)); + +// final boolean listen = Boolean.valueOf(properties.getProperty( +// Options.GANGLIA_LISTEN, +// Options.DEFAULT_GANGLIA_LISTEN)); +// +// final boolean report = Boolean.valueOf(properties.getProperty( +// Options.GANGLIA_REPORT, +// Options.DEFAULT_GANGLIA_REPORT)); + + // Note: defaults to the listenGroup and port if nothing given. + final InetSocketAddress[] metricsServers = GangliaUtil.parse( + // server(s) + properties.getProperty( + Options.GANGLIA_SERVERS, + Options.DEFAULT_GANGLIA_SERVERS), + // default host (same as listenGroup) + listenGroup.getHostName(), + // default port (same as listenGroup) + listenPort + ); + + final int quietPeriod = IGangliaDefaults.QUIET_PERIOD; + + final int initialDelay = IGangliaDefaults.INITIAL_DELAY; + + /* + * Note: Use ZERO (0) if you are running gmond on the same host. + * That will prevent the GangliaService from transmitting a + * different heartbeat, which would confuse gmond and gmetad. + */ + final int heartbeatInterval = 0; // IFF using gmond. + // final int heartbeatInterval = + // IGangliaDefaults.HEARTBEAT_INTERVAL; + + // Use the report delay for the interval in which we scan the + // performance counters. + final int monitoringInterval = (int) TimeUnit.MILLISECONDS + .toSeconds(Long.parseLong(properties.getProperty( + Options.REPORT_DELAY, + Options.DEFAULT_REPORT_DELAY))); + + final String defaultUnits = IGangliaDefaults.DEFAULT_UNITS; + + final GangliaSlopeEnum defaultSlope = IGangliaDefaults.DEFAULT_SLOPE; + + final int defaultTMax = IGangliaDefaults.DEFAULT_TMAX; + + final int defaultDMax = IGangliaDefaults.DEFAULT_DMAX; + + // Note: Factory is extensible (application can add its own + // delegates). + final GangliaMetadataFactory metadataFactory = new GangliaMetadataFactory( + new DefaultMetadataFactory(// + defaultUnits,// + defaultSlope,// + defaultTMax,// + defaultDMax// + )); + + /* + * Layer on the ability to (a) recognize and align host + * bigdata's performance counters hierarchy with those declared + * by ganglia and; (b) provide nice declarations for various + * application counters of interest. + */ + metadataFactory.add(new BigdataMetadataFactory(hostName, + serviceName, defaultSlope, defaultTMax, defaultDMax, + heartbeatInterval)); + + // The embedded ganglia peer. + final BigdataGangliaService gangliaService = new BigdataGangliaService( + hostName, // + serviceName, // + metricsServers,// + listenGroup,// + listenPort, // + listen,// listen + report,// report + false,// mock, + quietPeriod, // + initialDelay, // + heartbeatInterval,// + monitoringInterval, // + defaultDMax,// globalDMax + metadataFactory); + + // Collect and report host metrics. + gangliaService.addMetricCollector(new HostMetricsCollector( + statisticsCollector)); + + // Collect and report QueryEngine metrics. + gangliaService + .addMetricCollector(new QueryEngineMetricsCollector( + journal, statisticsCollector)); + + /* + * TODO The problem with reporting per-service statistics is + * that ganglia lacks a facility to readily aggregate statistics + * across services on a host (SMS + anything). The only way this + * can readily be made to work is if each service has a distinct + * metric for the same value (e.g., Mark and Sweep GC). However, + * that causes a very large number of distinct metrics. I have + * commented this out for now while I think it through some + * more. Maybe we will wind up only reporting the per-host + * counters to ganglia? + * + * Maybe the right way to handle this is to just filter by the + * service type? Basically, that is what we are doing for the + * QueryEngine metrics. + */ + // Collect and report service metrics. +// gangliaService.addMetricCollector(new ServiceMetricsCollector( +// statisticsCollector, null/* filter */)); + + // Wrap as Future. + final FutureTask<Void> ft = new FutureTask<Void>( + gangliaService, (Void) null); + + // Save reference to future. + gangliaFuture.set(ft); + + // Set the state reference. + GangliaPlugIn.this.gangliaService.set(gangliaService); + + // Start the embedded ganglia service. + journal.getExecutorService().submit(ft); + + } catch (RejectedExecutionException t) { + + /* + * Ignore. + * + * Note: This occurs if the federation shutdown() before we + * start the embedded ganglia peer. For example, it is common + * when running a short lived utility service such as + * ListServices. + */ + + } catch (Throwable t) { + + log.error(t, t); + + } + + } + + /** + * {@inheritDoc} + * <p> + * Note: The embedded GangliaService is executed on the main thread pool. We + * need to terminate the GangliaService in order for the thread pool to + * shutdown. + */ + @Override + public void stopService(final boolean immediateShutdown) { + + final FutureTask<Void> ft = gangliaFuture.getAndSet(null); + + if (ft != null) { + + ft.cancel(immediateShutdown/* mayInterruptIfRunning */); + + } + + // Clear the state reference. + gangliaService.set(null); + + } + + @Override + public boolean isRunning() { + + final FutureTask<Void> ft = gangliaFuture.get(); + + if (ft == null || ft.isDone()) + return false; + + return true; + + } + + @Override + public GangliaService getService() { + + return gangliaService.get(); + + } + +} Copied: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/HttpPlugin.java (from rev 7048, branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/journal/HttpPlugin.java) =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/HttpPlugin.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/HttpPlugin.java 2013-04-15 13:22:11 UTC (rev 7049) @@ -0,0 +1,210 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.journal; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; + +import org.apache.log4j.Logger; + +import com.bigdata.counters.AbstractStatisticsCollector; +import com.bigdata.counters.CounterSet; +import com.bigdata.counters.httpd.CounterSetHTTPD; + +/** + * {@link CounterSetHTTPD} plug-in. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +@SuppressWarnings("deprecation") +public class HttpPlugin implements IPlugIn<Journal, CounterSetHTTPD> { + + private static final Logger log = Logger.getLogger(HttpPlugin.class); + + public interface Options { + + /** + * Integer option specifies the port on which an httpd service will be + * started that exposes the {@link CounterSet} for the client (default + * {@value #DEFAULT_HTTPD_PORT}). When ZERO (0), a random port will be + * used and the actual port selected may be discovered using + * {@link Journal#getHttpdURL()}. The httpd service may be disabled by + * specifying <code>-1</code> as the port. + */ + String HTTPD_PORT = Journal.class.getName() + ".httpdPort"; + + /** + * The default http service port is <code>-1</code>, which means + * performance counter reporting is disabled by default. + */ + String DEFAULT_HTTPD_PORT = "-1"; + + } + + /** + * httpd reporting the live counters -or- <code>null</code> if not enabled. + * <p> + * Note: Guarded by <code>synchronized(this)</code>. + * + * @see Options#HTTPD_PORT + */ + private CounterSetHTTPD httpd = null; + + /** + * The URL that may be used to access the httpd service exposed by this + * client -or- <code>null</code> if not enabled. + * <p> + * Note: Guarded by <code>synchronized(this)</code>. + */ + private String httpdURL = null; + + /** + * {@inheritDoc} + * <p> + * Start the local httpd service (if enabled). The service is started on the + * {@link Journal#getHttpdPort()}, on a randomly assigned port if the port + * is <code>0</code>, or NOT started if the port is <code>-1</code>. If the + * service is started, then the URL for the service is reported to the load + * balancer and also written into the file system. When started, the httpd + * service will be shutdown with the federation. + */ + @Override + public void startService(final Journal indexManager) { + + final int httpdPort = Integer.valueOf(indexManager.getProperty( + Options.HTTPD_PORT, Options.DEFAULT_HTTPD_PORT)); + + if (log.isInfoEnabled()) + log.info(Options.HTTPD_PORT + "=" + httpdPort + + (httpdPort == -1 ? " (disabled)" : "")); + + if (httpdPort == -1) { + + return; + + } + + final CounterSetHTTPD httpd; + try { + + httpd = new CounterSetHTTPD(httpdPort, indexManager); + + } catch (IOException e) { + + log.error("Could not start httpd: port=" + httpdPort, e); + + return; + + } + + final String httpdURL; + try { + + httpdURL = "http://" + + AbstractStatisticsCollector.fullyQualifiedHostName + ":" + + httpd.getPort() + "/?path=" + + URLEncoder.encode("", "UTF-8"); + + } catch (UnsupportedEncodingException ex) { + + log.error("Could not start httpd: " + ex, ex); + + httpd.shutdownNow(); + + return; + + } + + synchronized (this) { + + // save reference to the daemon. + this.httpd = httpd; + + // the URL that may be used to access the local httpd. + this.httpdURL = httpdURL; + + if (log.isInfoEnabled()) + log.info("Performance counters: " + httpdURL); + + } + + } + + @Override + public void stopService(final boolean immediateShutdown) { + + synchronized (this) { + + if (httpd != null) { + + httpd.shutdown(); + + httpd = null; + + httpdURL = null; + + } + + } + + } + + @Override + public CounterSetHTTPD getService() { + + synchronized (this) { + + return httpd; + + } + + } + + @Override + public boolean isRunning() { + + synchronized (this) { + + return httpd != null; + + } + + } + + /** + * The URL that may be used to access the httpd service exposed by this + * client -or- <code>null</code> if not enabled. + */ + final public String getHttpdURL() { + + synchronized (this) { + + return httpdURL; + + } + + } + +} Copied: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IPlugIn.java (from rev 7048, branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/journal/IPlugIn.java) =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IPlugIn.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/IPlugIn.java 2013-04-15 13:22:11 UTC (rev 7049) @@ -0,0 +1,64 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.journal; + + + +/** + * Interface for a "plug in" that can be registered with a {@link Journal}. + * + * TODO Support at {@link IIndexManager} interface? E.g., for the federation + * as well as the journal. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IPlugIn<M extends IIndexManager, S> { + + /** + * Start the service. + * + * @param indexManager + */ + public void startService(M indexManager); + + /** + * Stop the service. + * + * @param immediateShutdown + * <code>true</code> if the service should be stopped + * immediately. + */ + public void stopService(boolean immediateShutdown); + + /** + * Return the service. + */ + public S getService(); + + /** + * Return true iff the service is running. + */ + public boolean isRunning(); + +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-15 13:13:35 UTC (rev 7048) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-15 13:22:11 UTC (rev 7049) @@ -25,10 +25,6 @@ import java.io.File; import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.URLEncoder; import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; @@ -38,8 +34,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.FutureTask; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.Semaphore; @@ -65,18 +59,7 @@ import com.bigdata.config.LongValidator; import com.bigdata.counters.AbstractStatisticsCollector; import com.bigdata.counters.CounterSet; -import com.bigdata.counters.ICounterSet; -import com.bigdata.counters.ganglia.BigdataGangliaService; -import com.bigdata.counters.ganglia.BigdataMetadataFactory; -import com.bigdata.counters.ganglia.HostMetricsCollector; -import com.bigdata.counters.ganglia.QueryEngineMetricsCollector; import com.bigdata.counters.httpd.CounterSetHTTPD; -import com.bigdata.ganglia.DefaultMetadataFactory; -import com.bigdata.ganglia.GangliaMetadataFactory; -import com.bigdata.ganglia.GangliaService; -import com.bigdata.ganglia.GangliaSlopeEnum; -import com.bigdata.ganglia.IGangliaDefaults; -import com.bigdata.ganglia.util.GangliaUtil; import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumService; import com.bigdata.journal.jini.ha.HAJournal; @@ -99,7 +82,6 @@ import com.bigdata.util.concurrent.LatchedExecutor; import com.bigdata.util.concurrent.ShutdownHelper; import com.bigdata.util.concurrent.ThreadPoolExecutorBaseStatisticsTask; -import com.bigdata.util.httpd.AbstractHTTPD; /** * Concrete implementation suitable for a local and unpartitioned database. @@ -139,11 +121,16 @@ * Options understood by the {@link Journal}. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ */ public interface Options extends com.bigdata.journal.Options, com.bigdata.journal.ConcurrencyManager.Options, - com.bigdata.journal.TemporaryStoreFactory.Options { + com.bigdata.journal.TemporaryStoreFactory.Options, + com.bigdata.journal.QueueStatsPlugIn.Options, + com.bigdata.journal.PlatformStatsPlugIn.Options, + com.bigdata.journal.HttpPlugin.Options + // Note: Do not import. Forces bigdata-ganglia dependency. + // com.bigdata.journal.GangliaPlugIn.Options + { /** * The capacity of the {@link HardReferenceQueue} backing the @@ -188,129 +175,6 @@ String DEFAULT_READ_POOL_SIZE = "0"; - /* - * Performance counters options. - */ - - /** - * Boolean option for the collection of statistics from the underlying - * operating system (default - * {@value #DEFAULT_COLLECT_PLATFORM_STATISTICS}). - * - * @see AbstractStatisticsCollector#newInstance(Properties) - */ - String COLLECT_PLATFORM_STATISTICS = Journal.class.getName() - + ".collectPlatformStatistics"; - - String DEFAULT_COLLECT_PLATFORM_STATISTICS = "false"; - - /** - * Boolean option for the collection of statistics from the various - * queues using to run tasks (default - * {@link #DEFAULT_COLLECT_QUEUE_STATISTICS}). - */ - String COLLECT_QUEUE_STATISTICS = Journal.class.getName() - + ".collectQueueStatistics"; - - String DEFAULT_COLLECT_QUEUE_STATISTICS = "false"; - - /** - * Integer option specifies the port on which an httpd service will be - * started that exposes the {@link CounterSet} for the client (default - * {@value #DEFAULT_HTTPD_PORT}). When ZERO (0), a random port will be - * used and the actual port selected may be discovered using - * {@link Journal#getHttpdURL()}. The httpd service may be disabled by - * specifying <code>-1</code> as the port. - */ - String HTTPD_PORT = Journal.class.getName() + ".httpdPort"; - - /** - * The default http service port is <code>-1</code>, which means - * performance counter reporting is disabled by default. - */ - String DEFAULT_HTTPD_PORT = "-1"; - - /** - * The delay between reports of performance counters in milliseconds ( - * {@value #DEFAULT_REPORT_DELAY}). When ZERO (0L), performance counter - * reporting will be disabled. - * - * @see #DEFAULT_REPORT_DELAY - */ - String REPORT_DELAY = Journal.class.getName() + ".reportDelay"; - - /** - * The default {@link #REPORT_DELAY}. - */ - String DEFAULT_REPORT_DELAY = ""+(60*1000); - - /* - * Ganglia - */ - - // Listen - - /** - * The multicast group used to join the ganglia performance monitoring - * network. - */ - String GANGLIA_LISTEN_GROUP = Journal.class.getName() - + ".ganglia.listenGroup"; - - String DEFAULT_GANGLIA_LISTEN_GROUP = IGangliaDefaults.DEFAULT_GROUP; - - /** - * The port for the multicast group used to join the ganglia performance - * monitoring network. - */ - String GANGLIA_LISTEN_PORT = Journal.class.getName() - + ".ganglia.listenPort"; - - String DEFAULT_GANGLIA_LISTEN_PORT = Integer - .toString(IGangliaDefaults.DEFAULT_PORT); - - /** - * When <code>true</code>, the embedded {@link GangliaService} will - * listen on to the specified multicast group and build up an internal - * model of the metrics in the ganglia network. - * <p> - * Note: If both {@link #GANGLIA_LISTEN} and {@link #GANGLIA_REPORT} are - * <code>false</code> then the embedded {@link GangliaService} will not - * be started. - */ - String GANGLIA_LISTEN = Journal.class.getName() - + ".ganglia.listen"; - - String DEFAULT_GANGLIA_LISTEN = "false"; - - // Report - - /** - * When <code>true</code>, the embedded {@link GangliaService} will - * report performance metrics to the specified gmetad server(s). - * <p> - * Note: If both {@link #GANGLIA_LISTEN} and {@link #GANGLIA_REPORT} are - * <code>false</code> then the embedded {@link GangliaService} will not - * be started. - */ - String GANGLIA_REPORT = Journal.class.getName() - + ".ganglia.report"; - - String DEFAULT_GANGLIA_REPORT = "false"; - - /** - * An list of the metric servers (<code>gmetad</code> instances) to - * which metrics will be sent. The default is to send metrics to the - * well known multicast group for ganglia. Zero or more hosts may be - * specified, separated by whitespace or commas. The port for each host - * is optional and defaults to the well known port for ganglia. Each - * host may be either a unicast address or a multicast group. - */ - String GANGLIA_SERVERS = Journal.class.getName() - + ".ganglia.servers"; - - String DEFAULT_GANGLIA_SERVERS = IGangliaDefaults.DEFAULT_GROUP; - } /** @@ -943,12 +807,18 @@ final CounterSet root = new CounterSet(); // Host wide performance counters (collected from the OS). - if (platformStatisticsCollector != null) { + { - root.attach(platformStatisticsCollector.getCounters()); + final AbstractStatisticsCollector t = getPlatformStatisticsCollector(); - } + if (t != null) { + root.attach(t.getCounters()); + + } + + } + // JVM wide performance counters. { @@ -985,11 +855,25 @@ tmp.makePath(IJournalCounters.transactionManager) .attach(localTransactionManager.getCounters()); - if (queueSampleTask != null) { + { - tmp.makePath(IJournalCounters.executorService) - .attach(queueSampleTask.getCounters()); + final IPlugIn<Journal, ThreadPoolExecutorBaseStatisticsTask> plugin = pluginQueueStats + .get(); + if (plugin != null) { + + final ThreadPoolExecutorBaseStatisticsTask t = plugin + .getService(); + + if (t != null) { + + tmp.makePath(IJournalCounters.executorService).attach( + t.getCounters()); + + } + + } + } } @@ -1711,32 +1595,45 @@ */ localTransactionManager.shutdown(); - /* - * Note: The embedded GangliaService is executed on the main thread - * pool. We need to terminate the GangliaService in order for the thread - * pool to shutdown. - */ { - final FutureTask<Void> ft = gangliaFuture.getAndSet(null); - if (ft != null) { + final IPlugIn<?, ?> plugIn = pluginGanglia.get(); - ft.cancel(true/* mayInterruptIfRunning */); + if (plugIn != null) { + // stop if running. + plugIn.stopService(false/* immediateShutdown */); + } - // Clear the state reference. - gangliaService.set(null); } - if (platformStatisticsCollector != null) { + { + + final IPlugIn<?, ?> plugIn = pluginQueueStats.get(); - platformStatisticsCollector.stop(); + if (plugIn != null) { - platformStatisticsCollector = null; + // stop if running. + plugIn.stopService(false/* immediateShutdown */); + } + } + { + + final IPlugIn<?, ?> plugIn = pluginPlatformStats.get(); + + if (plugIn != null) { + + // stop if running. + plugIn.stopService(false/* immediateShutdown */); + + } + + } + if (scheduledExecutorService != null) { scheduledExecutorService.shutdown(); @@ -1744,14 +1641,17 @@ } // optional httpd service for the local counters. - if (httpd != null) { + { + + final IPlugIn<?, ?> plugIn = pluginHttpd.get(); - httpd.shutdown(); + if (plugIn != null) { - httpd = null; + // stop if running. + plugIn.stopService(false/* immediateShutdown */); - httpdURL = null; - + } + } /* @@ -1808,43 +1708,63 @@ return; /* - * Note: The embedded GangliaService is executed on the main thread - * pool. We need to terminate the GangliaService in order for the thread - * pool to shutdown. + * Note: The ganglia plug in is executed on the main thread pool. We + * need to terminate it in order for the thread pool to shutdown. */ { - final FutureTask<Void> ft = gangliaFuture.getAndSet(null); - if (ft != null) { + final IPlugIn<?, ?> plugIn = pluginGanglia.get(); - ft.cancel(true/* mayInterruptIfRunning */); + if (plugIn != null) { + // stop if running. + plugIn.stopService(true/* immediateShutdown */); + } + + } - // Clear the state reference. - gangliaService.set(null); + { + + final IPlugIn<?, ?> plugIn = pluginQueueStats.get(); + + if (plugIn != null) { + + // stop if running. + plugIn.stopService(true/* immediateShutdown */); + + } + } - if (platformStatisticsCollector != null) { + { + + final IPlugIn<?, ?> plugIn = pluginPlatformStats.get(); - platformStatisticsCollector.stop(); + if (plugIn != null) { - platformStatisticsCollector = null; + // stop if running. + plugIn.stopService(true/* immediateShutdown */); + } + } if (scheduledExecutorService != null) scheduledExecutorService.shutdownNow(); // optional httpd service for the local counters. - if (httpd != null) { + { + + final IPlugIn<?, ?> plugIn = pluginHttpd.get(); - httpd.shutdown(); + if (plugIn != null) { - httpd = null; + // stop if running. + plugIn.stopService(false/* immediateShutdown */); - httpdURL = null; - + } + } // Note: can be null if error in ctor. @@ -2206,56 +2126,46 @@ */ private final ScheduledExecutorService scheduledExecutorService; - /** - * Collects interesting statistics on the {@link #executorService}. - * - * @see Options#COLLECT_QUEUE_STATISTICS + /* + * plugins. */ - private ThreadPoolExecutorBaseStatisticsTask queueSampleTask = null; - + + private final AtomicReference<IPlugIn<Journal, ThreadPoolExecutorBaseStatisticsTask>> pluginQueueStats = new AtomicReference<IPlugIn<Journal,ThreadPoolExecutorBaseStatisticsTask>>(); + private final AtomicReference<IPlugIn<Journal, AbstractStatisticsCollector>> pluginPlatformStats = new AtomicReference<IPlugIn<Journal, AbstractStatisticsCollector>>(); + private final AtomicReference<IPlugIn<Journal, ?>> pluginHttpd = new AtomicReference<IPlugIn<Journal, ?>>(); + /** - * Collects interesting statistics on the host and process. + * An optional plug in for Ganglia. + * <p> + * Note: The plug in concept was introduced to decouple the ganglia + * component. Do not introduce imports into the {@link Journal} class that + * would make the ganglia code a required dependency! * - * @see Options#COLLECT_PLATFORM_STATISTICS + * @see <a href="https://sourceforge.net/apps/trac/bigdata/ticket/609"> + * bigdata-ganglia is required dependency for Journal </a> */ - private AbstractStatisticsCollector platformStatisticsCollector = null; + private final AtomicReference<IPlugIn<Journal, ?>> pluginGanglia = new AtomicReference<IPlugIn<Journal, ?>>(); /** - * httpd reporting the live counters -or- <code>null</code> if not enabled. + * Host wide performance counters (collected from the OS) (optional). * - * @see Options#HTTPD_PORT + * @see PlatformStatsPlugIn */ - private AbstractHTTPD httpd = null; - - /** - * The URL that may be used to access the httpd service exposed by this - * client -or- <code>null</code> if not enabled. - */ - private String httpdURL = null; + protected AbstractStatisticsCollector getPlatformStatisticsCollector() { - /** - * The URL that may be used to access the httpd service exposed by this - * client -or- <code>null</code> if not enabled. - */ - final public String getHttpdURL() { - - return httpdURL; - + final IPlugIn<Journal, AbstractStatisticsCollector> plugin = pluginPlatformStats + .get(); + + if (plugin == null) + return null; + + final AbstractStatisticsCollector t = plugin.getService(); + + return t; + } /** - * Future for an embedded {@link GangliaService} which listens to - * <code>gmond</code> instances and other {@link GangliaService}s and - * reports out metrics from {@link #getCounters()} to the ganglia network. - */ - private final AtomicReference<FutureTask<Void>> gangliaFuture = new AtomicReference<FutureTask<Void>>(); - - /** - * The embedded ganglia peer. - */ - private final AtomicReference<BigdataGangliaService> gangliaService = new AtomicReference<BigdataGangliaService>(); - - /** * An executor service used to read on the local disk. * * @todo This is currently used by prefetch. We should generalize this @@ -2286,12 +2196,6 @@ * * @author <a href="mailto:tho...@us...">Bryan * Thompson</a> - * - * FIXME Make sure that we disable this by default (including the - * http reporting, the process and host monitoring, and the ganglia - * monitoring) for the unit tests or we will have a bunch of - * sampling processes running! (Right now, these things are disabled - * by default in {@link Journal.Options}.) */ private class StartDeferredTasksTask implements Runnable { @@ -2327,376 +2231,68 @@ protected void startDeferredTasks() throws IOException { // start collection on various work queues. - startQueueStatisticsCollection(); - - // start collecting performance counters (if enabled). - startPlatformStatisticsCollection(); - - // start the local httpd service reporting on this service. - startHttpdService(); - - /* - * Start embedded Ganglia peer. It will develop a snapshot of the - * metrics in memory for all nodes reporting in the ganglia network - * and will self-report metrics from the performance counter - * hierarchy to the ganglia network. - */ { - final Properties properties = getProperties(); - - final boolean listen = Boolean - .valueOf(properties.getProperty(Options.GANGLIA_LISTEN, - Options.DEFAULT_GANGLIA_LISTEN)); - - final boolean report = Boolean - .valueOf(properties.getProperty(Options.GANGLIA_REPORT, - Options.DEFAULT_GANGLIA_REPORT)); - - if (listen || report) - startGangliaService(Journal.this.platformStatisticsCollector); - - } - - } - - /** - * Setup sampling on the client's thread pool. This collects interesting - * statistics about the thread pool for reporting to the load balancer - * service. - */ - protected void startQueueStatisticsCollection() { - - final boolean collectQueueStatistics = Boolean.valueOf(getProperty( - Options.COLLECT_QUEUE_STATISTICS, - Options.DEFAULT_COLLECT_QUEUE_STATISTICS)); - - if (log.isInfoEnabled()) - log.info(Options.COLLECT_QUEUE_STATISTICS + "=" - + collectQueueStatistics); - - if (!collectQueueStatistics) { - - return; - - } - - final long initialDelay = 0; // initial delay in ms. - final long delay = 1000; // delay in ms. - final TimeUnit unit = TimeUnit.MILLISECONDS; - - queueSampleTask = new ThreadPoolExecutorBaseStatisticsTask( - executorService); - - addScheduledTask(queueSampleTask, initialDelay, - delay, unit); - - } - - /** - * Start collecting performance counters from the OS (if enabled). - */ - protected void startPlatformStatisticsCollection() { - - final boolean collectPlatformStatistics = Boolean - .valueOf(getProperty(Options.COLLECT_PLATFORM_STATISTICS, - Options.DEFAULT_COLLECT_PLATFORM_STATISTICS)); - - if (log.isInfoEnabled()) - log.info(Options.COLLECT_PLATFORM_STATISTICS + "=" - + collectPlatformStatistics); - - if (!collectPlatformStatistics) { - - return; - - } - - final Properties p = getProperties(); - - if (p.getProperty(AbstractStatisticsCollector.Options.PROCESS_NAME) == null) { - - // Set default name for this process. - p.setProperty(AbstractStatisticsCollector.Options.PROCESS_NAME, - "service" + ICounterSet.pathSeparator - + Journal.class.getName()); - - } - - try { - - final AbstractStatisticsCollector tmp = AbstractStatisticsCollector - .newInstance(p); - - tmp.start(); - - // Note: synchronized(Journal.this) keeps find bugs happy. - synchronized(Journal.this) { - - Journal.this.platformStatisticsCollector = tmp; - - } + final IPlugIn<Journal, ThreadPoolExecutorBaseStatisticsTask> tmp = new QueueStatsPlugIn(); - if (log.isInfoEnabled()) - log.info("Collecting platform statistics."); - - } catch (Throwable t) { - - log.error(t, t); + tmp.startService(Journal.this); + // Save reference iff started. + pluginQueueStats.set(tmp); + } - - } - - /** - * Start the local httpd service (if enabled). The service is started on - * the {@link Journal#getHttpdPort()}, on a randomly assigned - * port if the port is <code>0</code>, or NOT started if the port is - * <code>-1</code>. If the service is started, then the URL for the - * service is reported to the load balancer and also written into the - * file system. When started, the httpd service will be shutdown with - ... [truncated message content] |
From: <tho...@us...> - 2013-04-16 12:40:33
|
Revision: 7050 http://bigdata.svn.sourceforge.net/bigdata/?rev=7050&view=rev Author: thompsonbry Date: 2013-04-16 12:40:25 +0000 (Tue, 16 Apr 2013) Log Message: ----------- Exposed some more methods on SnapshotManager. Debugged a problem where daily snapshots were not being taken (it was failing to strip off the filename extension for an HALog file correctly). Extended the status report to indicate whether or not a snapshot would be taken the next time the ISnapshotPolicy runs. Modified Paths: -------------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultSnapshotPolicy.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -32,6 +32,7 @@ import org.apache.log4j.Logger; import com.bigdata.ha.msg.HASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; /** @@ -151,6 +152,13 @@ } + @Override + public IHASnapshotRequest newSnapshotRequest() { + + return new HASnapshotRequest(percentLogSize); + + } + private class SnapshotTask implements Runnable { /** @@ -184,7 +192,7 @@ // Conditionally start a snapshot. final Future<IHASnapshotResponse> f = jnl.getSnapshotManager() - .takeSnapshot(new HASnapshotRequest(percentLogSize)); + .takeSnapshot(newSnapshotRequest()); if (f != null) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -2093,8 +2093,8 @@ } /* - * Atomic decision whether HALog *was* for live write set when rbs - * were obtained from leader. + * Atomic decision whether HALog *was* for live write set when root + * blocks were obtained from leader. * * Note: Even if true, it is possible that the write set could have * been committed since then. However, if false then guaranteed that @@ -2373,7 +2373,7 @@ if (haLog.isInfoEnabled()) haLog.info("Will attempt to join met quorum."); - final UUID leaderId = getQuorum().getLeaderId(); + final UUID leaderId = getQuorum().getLeaderId(); // FIXME HA TXS: REFACTOR COMMON CODE WITH resyncTransitionToMetQuorum // The vote cast by the leader. final long lastCommitTimeOfQuorumLeader = getQuorum() @@ -2393,7 +2393,7 @@ // Verify that the quorum is valid. getQuorum().assertQuorum(token); - + //FIXME HA TXS: We must have the releaseTime before calling setQuorumToken(). CRITICAL SECTION exclusion here. // Set the token on the journal. journal.setQuorumToken(token); @@ -2804,7 +2804,7 @@ // Vote the consensus for the met quorum. final Quorum<?, ?> quorum = getQuorum(); - final UUID leaderId = quorum.getLeaderId(); + final UUID leaderId = quorum.getLeaderId(); // FIXME HA TXS: RECONCILE AND COMBINE WITH doConditionJoinWithMetQuorum() final long token = msg.getQuorumToken(); quorum.assertQuorum(token); // Note: Concurrent quorum break will cause NPE here. @@ -2826,7 +2826,7 @@ * TODO What happens if we are blocked here? */ getActor().serviceJoin(); - + // FIXME HA TXS: We must have the releaseTime before calling setQuorumToken(). CRITICAL SECTION exclusion here. // Set the token on the journal. journal.setQuorumToken(token); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/ISnapshotPolicy.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -23,6 +23,8 @@ */ package com.bigdata.journal.jini.ha; +import com.bigdata.ha.msg.IHASnapshotRequest; + /** * The policy that specifies when a new snapshot will be taken. The decision to * take a snapshot is a local decision. The snapshot is assumed to be written to @@ -52,4 +54,10 @@ @Override String toString(); + /** + * Return a request for a snapshot using the configured parameters for this + * {@link ISnapshotPolicy}. + */ + IHASnapshotRequest newSnapshotRequest(); + } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/NoSnapshotPolicy.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -23,6 +23,9 @@ */ package com.bigdata.journal.jini.ha; +import com.bigdata.ha.msg.HASnapshotRequest; +import com.bigdata.ha.msg.IHASnapshotRequest; + /** * Policy NEVER takes a snapshot. * @@ -42,4 +45,10 @@ } + public IHASnapshotRequest newSnapshotRequest() { + + return new HASnapshotRequest(100/* percentLogSize */); + + } + } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -708,6 +708,12 @@ /** * Take a new snapshot. This is a NOP if a snapshot is already being made. + * <p> + * Note: The service must be joined with a met quorum to take a snapshot. + * This is checked here and also in HAJournal when we take the snapshot. + * This is necessary in order to ensure that the snapshots are copies of a + * journal state that the quorum agrees on, otherwise we could later attempt + * to restore from an invalid state. * * @return The {@link Future} if a snapshot is already being made -or- if a * snapshot was started by the request and <code>null</code> if no @@ -742,6 +748,17 @@ } + final long token = journal.getQuorum().token(); + + if (!journal.getQuorum().getClient().isJoinedMember(token)) { + + haLog.warn("Service not joined with met quorum."); + + // This service is not joined with a met quorum. + return null; + + } + if (!isReadyToSnapshot(req)) { // Pre-conditions are not met. @@ -819,29 +836,12 @@ * snapshot. If the size(halogs) as a percentage of the size(journal) is LTE * the given [percentLogSize], then we return [false] to indicate that no * snapshot should be taken. - * <p> - * Note: The service must be joined with a met quorum to take a snapshot. - * This is checked here and also in HAJournal when we take the snapshot. - * This is necessary in order to ensure that the snapshots are copies of a - * journal state that the quorum agrees on, otherwise we could later attempt - * to restore from an invalid state. */ - private boolean isReadyToSnapshot(final IHASnapshotRequest req) { + public boolean isReadyToSnapshot(final IHASnapshotRequest req) { - if(req == null) + if (req == null) throw new IllegalArgumentException(); - - final long token = journal.getQuorum().token(); - - if (!journal.getQuorum().getClient().isJoinedMember(token)) { - haLog.warn("Service not joined with met quorum."); - - // This service is not joined with a met quorum. - return false; - - } - final IRootBlockView snapshotRootBlock = snapshotIndex .getNewestSnapshot(); @@ -853,20 +853,72 @@ * We already have a snapshot for the most recent commit point on * the journal. */ - + return false; - + } + + final long sinceCommitCounter = snapshotRootBlock == null ? 0L + : snapshotRootBlock.getCommitCounter(); + + // Get HALog bytes on disk since that commit counter (strictly GT). + final long haLogBytesOnDisk = getHALogFileBytesSinceCommitCounter(sinceCommitCounter); + + /* + * Figure out the size of the HALog files written since the last + * snapshot as a percentage of the size of the journal. + */ + final long journalSize = journal.size(); + + // size(HALogs)/size(journal) as percentage. + final int actualPercentLogSize = (int) (100 * (((double) haLogBytesOnDisk) / ((double) journalSize))); + + final int thresholdPercentLogSize = req.getPercentLogSize(); + + final boolean takeSnapshot = (actualPercentLogSize >= thresholdPercentLogSize); + + if (haLog.isInfoEnabled()) { + + haLog.info("sinceCommitCounter=" + + sinceCommitCounter// + + ", haLogBytesOnDisk=" + + haLogBytesOnDisk// + + ", journalSize=" + + journalSize// + + ", percentLogSize=" + + actualPercentLogSize// + + "%, takeSnapshot=" + (takeSnapshot ? "" : " not") + + " be taken"); + + } + return takeSnapshot; + + } + + /** + * Return the #of bytes in the HALog files since a given commit point. + * <p> + * Note: The current (live) HALog file is NOT in the reported total. The + * total only reports the bytes on disk for the committed transactions. + * + * @param sinceCommitCounter + * The exclusive lower bound and <code>-1L</code> if the total + * bytes on disk for ALL HALog files should be reported. + * + * @return The #of bytes in those HALog files. + */ + public long getHALogFileBytesSinceCommitCounter(final long sinceCommitCounter) { + /* * List the HALog files for this service. */ final File[] files; { - // most recent snapshot commit counter or -1L if no snapshots exist. - final long snapshotCommitCounter = snapshotRootBlock == null ? -1L - : snapshotRootBlock.getCommitCounter(); +// // most recent snapshot commit counter or -1L if no snapshots exist. +// final long snapshotCommitCounter = snapshotRootBlock == null ? -1L +// : snapshotRootBlock.getCommitCounter(); final File currentLogFile = journal.getHALogWriter().getFile(); @@ -894,27 +946,28 @@ return false; } - // filter out the current log file if (currentLogFile != null && name.equals(currentLogFileName)) { - /* - * The caller requested that we NOT purge the - * current HALog, and this is it. - */ + // filter out the current log file return false; } // Strip off the filename extension. final String logFileBaseName = name.substring(0, - IHALogReader.HA_LOG_EXT.length()); + name.length() - IHALogReader.HA_LOG_EXT.length()); // Closing commitCounter for HALog file. final long logCommitCounter = Long .parseLong(logFileBaseName); - if (logCommitCounter >= snapshotCommitCounter) { + if (logCommitCounter > sinceCommitCounter) { /* * HALog is more recent than the current snapshot + * + * Note: We do not include the HALog file if it was for + * the commit point of the snapshot. We are only + * counting HALog file bytes that have NOT yet been + * incorporated into a snapshot. */ return true; } @@ -941,35 +994,14 @@ } - /* - * Figure out the size of the HALog files written since the last - * snapshot as a percentage of the size of the journal. - */ - - final long journalSize = journal.size(); + if (haLog.isInfoEnabled()) + haLog.info("sinceCommitCounter=" + sinceCommitCounter + ", files=" + + files.length + ", bytesOnDisk=" + totalBytes); - // size(HALogs)/size(journal) as percentage. - final int actualPercentLogSize = (int) (100 * (((double) totalBytes) / ((double) journalSize))); - - final int thresholdPercentLogSize = req.getPercentLogSize(); - - final boolean takeSnapshot = (actualPercentLogSize >= thresholdPercentLogSize); - - if (haLog.isInfoEnabled()) { - - haLog.info("There are " + files.length - + " HALog files since the last snapshot occupying " - + totalBytes + " bytes. The journal is currently " - + journalSize + " bytes. The HALogs are " + actualPercentLogSize - + " of the journal on the disk. A new snapshot should" - + (takeSnapshot ? "" : " not") + " be taken"); - - } + return totalBytes; - return takeSnapshot; - } - + /** * Take a snapshot. * Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-15 13:22:11 UTC (rev 7049) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java 2013-04-16 12:40:25 UTC (rev 7050) @@ -51,9 +51,11 @@ import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.RootBlockView; import com.bigdata.journal.jini.ha.HAJournal; +import com.bigdata.journal.jini.ha.ISnapshotPolicy; import com.bigdata.journal.jini.ha.SnapshotManager; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.zk.ZKQuorumImpl; import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.zookeeper.DumpZookeeper; @@ -168,10 +170,30 @@ { // snapshot policy. - p.text("Service: snapshotPolicy=" - + journal.getSnapshotManager().getSnapshotPolicy()) - .node("br").close(); - + { + final SnapshotManager mgr = journal.getSnapshotManager(); +// final IRootBlockView lastSnapshotRB = mgr +// .getNewestSnapshot(); +// final long sinceCommitCounter = lastSnapshotRB == null ? -1L +// : lastSnapshotRB.getCommitCounter(); +// final long haLogBytesOnDiskSinceLastSnapshot = mgr +// .getHALogFileBytesSinceCommitCounter(sinceCommitCounter); + final ISnapshotPolicy snapshotPolicy = mgr + .getSnapshotPolicy(); + final boolean takeSnapshot = mgr + .isReadyToSnapshot(snapshotPolicy + .newSnapshotRequest()); + p.text("Service"// + + ": snapshotPolicy=" + + snapshotPolicy// + + ", shouldSnapshot=" + + takeSnapshot// +// + ", lastSnapshotCommitCounter=" +// + sinceCommitCounter// +// + ", HALogFileBytesOnDiskSinceLastSnapshot=" +// + haLogBytesOnDiskSinceLastSnapshot// + ).node("br").close(); + } // restore policy. p.text("Service: restorePolicy=" + journal.getSnapshotManager().getRestorePolicy()) @@ -212,11 +234,23 @@ } final long commitCounter = journal.getRootBlockView() .getCommitCounter(); + // TODO Move this stuff to a TXS Status section? + long releaseTime = -1; + try { + // Note: Can throw exception if quorum is not met. + releaseTime = journal.getTransactionService() + .getReleaseTime(); + } catch (QuorumException ex) { + // Ignore. + } p.text("HAJournal: file=" + file // + ", commitCounter=" + commitCounter // + ", nbytes=" + journal.size()// - + (digestStr == null ? "" : ", md5=" + digestStr)) - .node("br").close(); + + (digestStr == null ? "" : ", md5=" + digestStr)// + + (releaseTime != -1L ? ", releaseTime=" + + RootBlockView.toString(releaseTime)// + : "")// + ).node("br").close(); } } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-17 23:06:44
|
Revision: 7054 http://bigdata.svn.sourceforge.net/bigdata/?rev=7054&view=rev Author: thompsonbry Date: 2013-04-17 23:06:33 +0000 (Wed, 17 Apr 2013) Log Message: ----------- Refactor to replace the delegation pattern by the followers to the quorum leader for new tx starts and aborts. This is intended to make queries execute 100% locally on the follower in an HA replication cluster and thus provide linear scaling for queries based on the #of nodes in the cluster. The leader engages the followers in a consensus protocol during the commit to decide on the new releaseTime, which governs both what allocation slots will be recycled and what commit points will be visible. This commit is 100% green on the HA test suite and the RWStore Journal test suite. There are no new test failures for either the AST test suite or the TestBigdataSailWithQuads test suite. See https://sourceforge.net/apps/trac/bigdata/ticket/623 (HA TXS) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumCommitImpl.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/JournalTransactionService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractTransactionService.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java Added Paths: ----------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAGatherReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeResponse.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHAGatherReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeResponse.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/util/ClocksNotSynchronizedException.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-16 18:01:59 UTC (rev 7053) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -44,7 +44,6 @@ import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.journal.AbstractJournal; -import com.bigdata.journal.ITransactionService; import com.bigdata.journal.Journal; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.QuorumException; @@ -66,7 +65,7 @@ * the standard jini smart proxy naming pattern. */ public interface HAGlue extends HAGlueBase, HAPipelineGlue, HAReadGlue, - HACommitGlue, ITransactionService, IService { + HACommitGlue, HATXSGlue, IService { /* * Administrative Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-16 18:01:59 UTC (rev 7053) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -29,6 +29,7 @@ import java.security.DigestException; import java.security.NoSuchAlgorithmException; import java.util.UUID; +import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -38,12 +39,15 @@ import com.bigdata.ha.msg.IHA2PhasePrepareMessage; import com.bigdata.ha.msg.IHADigestRequest; import com.bigdata.ha.msg.IHADigestResponse; +import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; import com.bigdata.ha.msg.IHAGlobalWriteLockRequest; import com.bigdata.ha.msg.IHALogDigestRequest; import com.bigdata.ha.msg.IHALogDigestResponse; import com.bigdata.ha.msg.IHALogRequest; import com.bigdata.ha.msg.IHALogRootBlocksRequest; import com.bigdata.ha.msg.IHALogRootBlocksResponse; +import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; import com.bigdata.ha.msg.IHAReadRequest; import com.bigdata.ha.msg.IHAReadResponse; import com.bigdata.ha.msg.IHARebuildRequest; @@ -55,10 +59,10 @@ import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; +import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; import com.bigdata.ha.msg.IHAWriteSetStateResponse; -import com.bigdata.journal.ValidationError; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.QuorumException; @@ -166,41 +170,60 @@ } @Override - public long nextTimestamp() throws IOException { - return delegate.nextTimestamp(); + public Future<Void> gatherMinimumVisibleCommitTime( + final IHAGatherReleaseTimeRequest req) throws IOException { + return delegate.gatherMinimumVisibleCommitTime(req); } @Override - public long newTx(long timestamp) throws IOException { - return delegate.newTx(timestamp); + public IHANotifyReleaseTimeResponse notifyEarliestCommitTime( + final IHANotifyReleaseTimeRequest req) throws IOException, + InterruptedException, BrokenBarrierException { + return delegate.notifyEarliestCommitTime(req); } @Override - public long commit(long tx) throws ValidationError, IOException { - return delegate.commit(tx); + public Future<Void> getTXSCriticalSectionLockOnLeader( + final IHATXSLockRequest req) throws IOException { + return delegate.getTXSCriticalSectionLockOnLeader(req); } - @Override - public void abort(long tx) throws IOException { - delegate.abort(tx); - } +// @Override +// public long nextTimestamp() throws IOException { +// return delegate.nextTimestamp(); +// } +// +// @Override +// public long newTx(long timestamp) throws IOException { +// return delegate.newTx(timestamp); +// } +// +// @Override +// public long commit(long tx) throws ValidationError, IOException { +// return delegate.commit(tx); +// } +// +// @Override +// public void abort(long tx) throws IOException { +// delegate.abort(tx); +// } +// +// @Override +// public void notifyCommit(long commitTime) throws IOException { +// delegate.notifyCommit(commitTime); +// } +// +// @Override +// public long getLastCommitTime() throws IOException { +// return delegate.getLastCommitTime(); +// } +// +// @Override +// public long getReleaseTime() throws IOException { +// return delegate.getReleaseTime(); +// } @Override - public void notifyCommit(long commitTime) throws IOException { - delegate.notifyCommit(commitTime); - } - - @Override - public long getLastCommitTime() throws IOException { - return delegate.getLastCommitTime(); - } - - @Override - public long getReleaseTime() throws IOException { - return delegate.getReleaseTime(); - } - - @Override public IHALogRootBlocksResponse getHALogRootBlocksForWriteSet( IHALogRootBlocksRequest msg) throws IOException { return delegate.getHALogRootBlocksForWriteSet(msg); Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,109 @@ +/** + +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 +*/ +package com.bigdata.ha; + +import java.io.IOException; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.Future; +import java.util.concurrent.locks.Lock; + +import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; +import com.bigdata.ha.msg.IHATXSLockRequest; +import com.bigdata.journal.ITransactionService; + +/** + * RMI interface for the {@link ITransactionService} for HA. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * + * @see <a + * href="https://docs.google.com/document/d/14FO2yJFv_7uc5N0tvYboU-H6XbLEFpvu-G8RhAzvxrk/edit?pli=1#" + * > HA TXS Design Document </a> + * + * @see <a href="https://sourceforge.net/apps/trac/bigdata/ticket/623" > HA TXS + * / TXS Bottleneck </a> + */ +public interface HATXSGlue { + + /** + * Message used to request information about the earliest commit point that + * is pinned on a follower. This is used by the leader to make a decision + * about the new release time for the replication cluster. + * <p> + * Note: This message is used as part of a pattern where the leader + * instructs the followers to message the leader with their earliest commit + * point pinned by either a transaction or the minReleaseAge of their + * {@link ITransactionService} using + * {@link #notifyEarliestCommitTime(IHANotifyReleaseTimeResponse)}. + * + * @param req + * The request from the leader. + * + * @see #notifyEarliestCommitTime(IHANotifyReleaseTimeResponse) + */ + Future<Void> gatherMinimumVisibleCommitTime(IHAGatherReleaseTimeRequest req) + throws IOException; + + /** + * Message used by the follower to notify the leader about the new release + * time that will be visible for new transaction starts. The leader then + * chooses the mimimum across itself and the followers. + * + * @param rsp + * The earliest pinned commit point on the follower. + * + * @return The earliest pinned commit point across the services joined with + * the met quorum. + * + * @throws BrokenBarrierException + * @throws InterruptedException + */ + IHANotifyReleaseTimeResponse notifyEarliestCommitTime( + IHANotifyReleaseTimeRequest req) throws IOException, + InterruptedException, BrokenBarrierException; + + /** + * Return an asynchronous {@link Future} for a task executing on the quorum + * leader that holds the {@link Lock} protecting the critical section + * contended by + * {@link #gatherMinimumVisibleCommitTime(IHAGatherReleaseTimeRequest)}. + * This lock is used by a service that wishes to join a met quorum to + * prevent a service join that is concurrent with the critical section in + * which the already joined services agree on the new <i>releaseTime</i>. + * <p> + * Note: The leader MAY impose a timeout on this lock such that a service + * which does not complete its work in a timely fashion will lose the lock. + * The service MUST check the {@link Future} and verify that it is still + * running. If the {@link Future#isCancelled()}, then the service MUST NOT + * complete its service join (or must do a service leave to exit the + * quorum). + * + * @return The {@link Future} for the task holding that lock. + */ + Future<Void> getTXSCriticalSectionLockOnLeader(IHATXSLockRequest req) + throws IOException; + +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumCommitImpl.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumCommitImpl.java 2013-04-16 18:01:59 UTC (rev 7053) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/QuorumCommitImpl.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -461,7 +461,7 @@ /* * If there were any errors, then throw an exception listing them. * - * TODO But only throw the exception if the errors were for a joined + * FIXME But only throw the exception if the errors were for a joined * service. Otherwise just log. */ if (!causes.isEmpty()) { Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAGatherReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAGatherReleaseTimeRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HAGatherReleaseTimeRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,53 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +public class HAGatherReleaseTimeRequest implements + IHAGatherReleaseTimeRequest { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private final long token; + private final long timestampOnLeader; + + public HAGatherReleaseTimeRequest(final long token, + final long timestampOnLeader) { + this.token = token; + this.timestampOnLeader = timestampOnLeader; + } + + @Override + public long token() { + return token; + } + + @Override + public long getTimestampOnLeader() { + return timestampOnLeader; + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,69 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +import java.util.UUID; + +public class HANotifyReleaseTimeRequest implements IHANotifyReleaseTimeRequest { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private final UUID serviceUUID; + private final long commitTime; + private final long commitCounter; + private final long timestampOnFollower; + + public HANotifyReleaseTimeRequest(final UUID serviceUUID, + final long commitTime, final long commitCounter, + final long timestampOnFollower) { + this.serviceUUID = serviceUUID; + this.commitTime = commitTime; + this.commitCounter = commitCounter; + this.timestampOnFollower = timestampOnFollower; + } + + @Override + public UUID getServiceUUID() { + return serviceUUID; + } + + @Override + public long getCommitTime() { + return commitTime; + } + + @Override + public long getCommitCounter() { + return commitCounter; + } + + @Override + public long getTimestampOnFollower() { + return timestampOnFollower; + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeResponse.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,53 @@ +/** + +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 +*/ +package com.bigdata.ha.msg; + +public class HANotifyReleaseTimeResponse implements + IHANotifyReleaseTimeResponse { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private final long commitTime; + private final long commitCounter; + + public HANotifyReleaseTimeResponse(final long commitTime, + final long commitCounter) { + this.commitTime = commitTime; + this.commitCounter = commitCounter; + } + + @Override + public long getCommitTime() { + return commitTime; + } + + @Override + public long getCommitCounter() { + return commitCounter; + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,44 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha.msg; + +public class HATXSLockRequest implements IHATXSLockRequest { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private final long token; + + public HATXSLockRequest(final long token) { + this.token = token; + } + + @Override + public long token() { + return token; + } + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHAGatherReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHAGatherReleaseTimeRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHAGatherReleaseTimeRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,58 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha.msg; + +/** + * Message used to request information about the earliest commit point that is + * pinned on a follower. This is used by the leader to make a decision about the + * new release time for the replication cluster. The message causes the follower + * to send an {@link IHANotifyReleaseTimeRequest} back to the leader. That + * message is sent from within the thread on the follower that is handling the + * RMI for the {@link IHAGatherReleaseTimeRequest} in order to synchronize the + * protocol across the leader and the followers. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHAGatherReleaseTimeRequest extends IHAMessage { + + /** + * The token for which this request is valid. + */ + public long token(); + +// /** +// * The timestamp associated with the earliest pinned commit point on the +// * leader. +// */ +// public long getLeadersValue(); + + /** + * A timestamp on the leader at the start of the protocol used to agree on + * the new release time (this can be the commitTime that will be assigned by + * the leader to the new commit point). This is used to detect problems + * where the clocks are not synchronized on the services. + */ + public long getTimestampOnLeader(); + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,72 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha.msg; + +import java.util.UUID; + +import com.bigdata.journal.ITransactionService; + +/** + * Message from a follower to the leader in which the follower specifies the + * earliest commit point that is pinned on the follower by an active transaction + * or the minReleaseAge associated with its local {@link ITransactionService}. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHANotifyReleaseTimeRequest extends IHAMessage { + + /** + * The service that provided this information. + */ + public UUID getServiceUUID(); + + /** + * The earliest pinned commit time on the follower. + */ + public long getCommitTime(); + + /** + * The earliest pinned commit counter on the follower. + */ + public long getCommitCounter(); + +// /** +// * The readsOnCommitTime of the earliest active transaction on the follower. +// */ +// public long getReadsOnCommitTimeForEarliestActiveTx(); +// +// /** +// * The minReleaseAge on the follower (this should be the same on all +// * services in a quorum). +// */ +// public long getMinReleaseAge(); + + /** + * A timestamp on the follower taken during the protocol used to agree on + * the new release time. This is used to detect problems where the clocks + * are not synchronized on the services. + */ + public long getTimestampOnFollower(); + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeResponse.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeResponse.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeResponse.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,44 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha.msg; + +/** + * Message from the leader providing the followers with the new agreed on + * earliest visible commit point. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHANotifyReleaseTimeResponse extends IHAMessage { + + /** + * The earliest visible commit time. + */ + public long getCommitTime(); + + /** + * The earliest visible commit counter. + */ + public long getCommitCounter(); + +} Added: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -0,0 +1,41 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha.msg; + +/** + * Message used to request a lock on the quorum leader to make a serviceJoin + * mutex with the critical section for the services joined with the met quorum + * make a decision concerning the new minimum visible commit point on the + * quorum. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public interface IHATXSLockRequest extends IHAMessage { + + /** + * The quorum token for which this request is valid. + */ + public long token(); + +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-16 18:01:59 UTC (rev 7053) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -49,6 +49,7 @@ import java.util.Properties; import java.util.Set; import java.util.UUID; +import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -95,6 +96,7 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.Instrument; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HATXSGlue; import com.bigdata.ha.QuorumService; import com.bigdata.ha.RunState; import com.bigdata.ha.msg.HAReadResponse; @@ -106,12 +108,15 @@ import com.bigdata.ha.msg.IHA2PhasePrepareMessage; import com.bigdata.ha.msg.IHADigestRequest; import com.bigdata.ha.msg.IHADigestResponse; +import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; import com.bigdata.ha.msg.IHAGlobalWriteLockRequest; import com.bigdata.ha.msg.IHALogDigestRequest; import com.bigdata.ha.msg.IHALogDigestResponse; import com.bigdata.ha.msg.IHALogRequest; import com.bigdata.ha.msg.IHALogRootBlocksRequest; import com.bigdata.ha.msg.IHALogRootBlocksResponse; +import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; import com.bigdata.ha.msg.IHAReadRequest; import com.bigdata.ha.msg.IHAReadResponse; import com.bigdata.ha.msg.IHARebuildRequest; @@ -123,6 +128,7 @@ import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; +import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; import com.bigdata.ha.msg.IHAWriteSetStateResponse; @@ -153,6 +159,7 @@ import com.bigdata.rwstore.IRWStrategy; import com.bigdata.rwstore.sector.MemStrategy; import com.bigdata.rwstore.sector.MemoryManager; +import com.bigdata.service.AbstractHATransactionService; import com.bigdata.service.IBigdataFederation; import com.bigdata.stream.Stream; import com.bigdata.util.ChecksumUtility; @@ -2844,26 +2851,45 @@ rootAddrs[PREV_ROOTBLOCK] = this.m_rootBlockCommitter .handleCommit(commitTime); - /* - * Write the commit record onto the store. - * - * @todo Modify to log the current root block and set the address of - * that root block in the commitRecord. This will be of use solely - * in disaster recovery scenarios where your root blocks are toast, - * but good root blocks can be found elsewhere in the file. - */ + if ((_bufferStrategy instanceof IHABufferStrategy) + && quorum != null && quorum.isHighlyAvailable()) { - final IRootBlockView old = _rootBlock; + try { + /** + * CRITICAL SECTION. We need obtain a distributed consensus + * for the services joined with the met quorum concerning + * the earliest commit point that is pinned by the + * combination of the active transactions and the + * minReleaseAge on the TXS. New transaction starts during + * this critical section will block (on the leader or the + * folllower) unless they are guaranteed to be allowable, + * e.g., based on the current minReleaseAge, the new tx + * would read from the most recent commit point, the new tx + * would ready from a commit point that is already pinned by + * an active transaction on that node, etc. + * + * @see <a href= + * "https://docs.google.com/document/d/14FO2yJFv_7uc5N0tvYboU-H6XbLEFpvu-G8RhAzvxrk/edit?pli=1#" + * > HA TXS Design Document </a> + * + * @see <a + * href="https://sourceforge.net/apps/trac/bigdata/ticket/623" + * > HA TXS / TXS Bottleneck </a> + */ + + ((AbstractHATransactionService) getLocalTransactionManager() + .getTransactionService()) + .updateReleaseTimeConsensus(); + + } catch (Exception ex) { - final long newCommitCounter = old.getCommitCounter() + 1; - - final ICommitRecord commitRecord = new CommitRecord(commitTime, - newCommitCounter, rootAddrs); + // Wrap and rethrow. + throw new RuntimeException(ex); + + } + + } - final long commitRecordAddr = write(ByteBuffer - .wrap(CommitRecordSerializer.INSTANCE - .serialize(commitRecord))); - /* * Before flushing the commitRecordIndex we need to check for * deferred frees that will prune the index. @@ -2879,6 +2905,26 @@ } /* + * Write the commit record onto the store. + * + * @todo Modify to log the current root block and set the address of + * that root block in the commitRecord. This will be of use solely + * in disaster recovery scenarios where your root blocks are toast, + * but good root blocks can be found elsewhere in the file. + */ + + final IRootBlockView old = _rootBlock; + + final long newCommitCounter = old.getCommitCounter() + 1; + + final ICommitRecord commitRecord = new CommitRecord(commitTime, + newCommitCounter, rootAddrs); + + final long commitRecordAddr = write(ByteBuffer + .wrap(CommitRecordSerializer.INSTANCE + .serialize(commitRecord))); + + /* * Add the commit record to an index so that we can recover * historical states efficiently. */ @@ -3416,6 +3462,50 @@ } + /** + * Resolve the {@link ICommitRecord} for the earliest visible commit point + * based on the current {@link ITransactionService#getReleaseTime()}. + */ + protected ICommitRecord getEarliestVisibleCommitRecord() { + + final ReadLock lock = _fieldReadWriteLock.readLock(); + + lock.lock(); + + try { + + final long releaseTime = getLocalTransactionManager() + .getTransactionService().getReleaseTime(); + + if (releaseTime == 0) { + + // Nothing committed yet. + return null; + + } + + final CommitRecordIndex commitRecordIndex = _commitRecordIndex; + + if (commitRecordIndex == null) + throw new AssertionError(); + + final ICommitRecord commitRecord = commitRecordIndex + .findNext(releaseTime); + + return commitRecord; + + } catch (IOException e) { + + // Note: Should not be thrown. Local method call. + throw new RuntimeException(e); + + } finally { + + lock.unlock(); + + } + } + /** * Returns a read-only view of the most recently committed * {@link ICommitRecord} containing the root addresses. @@ -4941,7 +5031,7 @@ * We do not need to discard read-only tx since the committed * state should remain valid even when a quorum is lost. * - * FIXME HA : QUORUM TX INTEGRATION (discard running read/write tx). + * FIXME HA TXS INTEGRATION (discard running read/write tx). */ // local abort (no quorum, so we can do 2-phase abort). @@ -6323,45 +6413,54 @@ * ITransactionService. * * Note: API is mostly implemented by Journal/HAJournal. + * + * Note: We should either not expose the ITransactionService or we + * should delegate the rest of this API. I am leaning toward NOT + * exposing the ITransactionService interface since (a) it does not + * appear to be necessary to start transactions on a specific service; + * and (b) if we do, then we really need to track remote transactions + * (by the remote Service UUID) and cancel them if the remote service + * leaves the met quorum. All we really need to expose here is the + * HATXSGlue interface and that DOES NOT need to extend the + * ITransactionService interface. */ @Override - public long newTx(long timestamp) throws IOException { - throw new UnsupportedOperationException(); + public Future<Void> gatherMinimumVisibleCommitTime( + final IHAGatherReleaseTimeRequest req) throws IOException { + + final Future<Void> ft = ((HATXSGlue) AbstractJournal.this + .getLocalTransactionManager().getTransactionService()) + .gatherMinimumVisibleCommitTime(req); + + return getProxy(ft, true/* asynchFuture */); + } @Override - public long commit(long tx) throws ValidationError, IOException { - throw new UnsupportedOperationException(); + public IHANotifyReleaseTimeResponse notifyEarliestCommitTime( + final IHANotifyReleaseTimeRequest req) throws IOException, + InterruptedException, BrokenBarrierException { + + return ((HATXSGlue) AbstractJournal.this + .getLocalTransactionManager().getTransactionService()) + .notifyEarliestCommitTime(req); + } @Override - public void abort(long tx) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void notifyCommit(long commitTime) throws IOException { - throw new UnsupportedOperationException(); - } + public Future<Void> getTXSCriticalSectionLockOnLeader( + final IHATXSLockRequest req) throws IOException { - @Override - public long getLastCommitTime() throws IOException { + final Future<Void> f = ((HATXSGlue) AbstractJournal.this + .getLocalTransactionManager().getTransactionService()) + .getTXSCriticalSectionLockOnLeader(req); - return AbstractJournal.this.getLastCommitTime(); - - } + // Note: MUST be an asynchronous Future!!! + return getProxy(f, true/* asynchronousFuture */); - @Override - public long getReleaseTime() throws IOException { - throw new UnsupportedOperationException(); } - @Override - public long nextTimestamp() throws IOException { - throw new UnsupportedOperationException(); - } - /* * IService */ Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-16 18:01:59 UTC (rev 7053) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-17 23:06:33 UTC (rev 7054) @@ -27,19 +27,28 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collection; +import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import org.apache.log4j.Logger; @@ -61,9 +70,18 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.httpd.CounterSetHTTPD; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HATXSGlue; import com.bigdata.ha.QuorumService; +import com.bigdata.ha.msg.HAGatherReleaseTimeRequest; +import com.bigdata.ha.msg.HANotifyReleaseTimeRequest; +import com.bigdata.ha.msg.HANotifyReleaseTimeResponse; +import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; +import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; +import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.journal.jini.ha.HAJournal; import com.bigdata.quorum.Quorum; +import com.bigdata.quorum.QuorumException; import com.bigdata.rawstore.IRawStore; import com.bigdata.relation.locator.DefaultResourceLocator; import com.bigdata.relation.locator.ILocatableResource; @@ -71,14 +89,17 @@ import com.bigdata.resources.IndexManager; import com.bigdata.resources.ResourceManager; import com.bigdata.resources.StaleLocatorReason; -import com.bigdata.rwstore.IRWStrategy; +import com.bigdata.rwstore.IHistoryManager; import com.bigdata.rwstore.IRawTx; +import com.bigdata.rwstore.RWStore; import com.bigdata.service.AbstractTransactionService; import com.bigdata.service.DataService; import com.bigdata.service.IBigdataFederation; import com.bigdata.sparse.GlobalRowStoreHelper; import com.bigdata.sparse.SparseRowStore; +import com.bigdata.util.ClocksNotSynchronizedException; import com.bigdata.util.concurrent.DaemonThreadFactory; +import com.bigdata.util.concurrent.ExecutionExceptions; import com.bigdata.util.concurrent.LatchedExecutor; import com.bigdata.util.concurrent.ShutdownHelper; import com.bigdata.util.concurrent.ThreadPoolExecutorBaseStatisticsTask; @@ -285,431 +306,1239 @@ } - protected AbstractLocalTransactionManager newLocalTransactionManager() { + /** + * Inner class used to coordinate the distributed protocol for achieving an + * atomic consensus on the new <i>releaseTime</i> for the services joined + * with a met quorum. + * + * @author <a href="mailto:tho...@us...">Bryan + * Thompson</a> + */ + private class BarrierState implements Runnable { + + /** + * The token that must remain valid. + * + * TODO We should also verify that the responses we collect are for the + * same request. This could be done using a request UUID or one-up + * request counter. That would guard against having a service reconnect + * and respond late once the leader had gotten to another commit point. + */ + final private long token; + + /** + * Local HA service implementation (non-Remote). + */ + final private QuorumService<HAGlue> quorumService; + + /** The services joined with the met quorum, in their join order. */ + final private UUID[] joinedServiceIds; + + /** + * {@link CyclicBarrier} used to coordinate the protocol for achiving an + * atomic consensus on the new <i>releaseTime</i> for the services + * joined with a met quorum. + * <p> + * Note: The {@link #barrier} + * provides visibilty for the fields that are modified by {@link #run()} + * so we do not need additional locks or atomics for synchronizing these + * state updates. + */ + final private CyclicBarrier barrier; - final JournalTransactionService abstractTransactionService = new JournalTransactionService( - checkProperties(properties), this) { +// /** +// * The {@link Future} for the RMI to each follower that is joined with +// * the met quorum. +// */ +// final private Map<UUID, Future<Void>> futures = new HashMap<UUID, Future<Void>>(); - /* - * @see http://sourceforge.net/apps/trac/bigdata/ticket/445 (RWStore - * does not track tx release correctly) - */ - final private ConcurrentHashMap<Long, IRawTx> m_rawTxs = new ConcurrentHashMap<Long, IRawTx>(); + /** + * A timestamp taken on the leader when we start the protocol to + * discover the new releaseTime consensus. + */ + final private long timestampOnLeader; + + /** + * This is the earliest visible commit point on the leader. + */ + final private IHANotifyReleaseTimeRequest leadersValue; + + /** + * The message from each of those followers providing their local + * earliest visible commit point. + */ + final private Map<UUID, IHANotifyReleaseTimeRequest> responses = new ConcurrentHashMap<UUID, IHANotifyReleaseTimeRequest>(); - // Note: This is the implicit constructor call. - { - - final long lastCommitTime = Journal.this.getLastCommitTime(); - - if (lastCommitTime != 0L) { + /** + * The value from {@link #responses} associated with the earliest commit + * point. This is basis for the "censensus" across the services. + */ + private IHANotifyReleaseTimeRequest minimumResponse = null; - /* - * Notify the transaction service on startup so it can set - * the effective release time based on the last commit time - * for the store. - */ - updateReleaseTimeForBareCommit(lastCommitTime); - + /** + * The consensus value. This is a restatement of the data in from the + * {@link #minimumResponse}. + */ + protected IHANotifyReleaseTimeResponse consensus = null; + +// private Quorum<HAGlue,QuorumService<HAGlue>> getQuorum() { +// +// return Journal.this.getQuorum(); +// +// } + + private HATXSGlue getService(final UUID serviceId) { + + return quorumService.getService(serviceId); + + } + + /** + * Cancel the requests on the remote services (RMI). This is a best effort + * implementation. Any RMI related errors are trapped and ignored in order + * to be robust to failures in RMI when we try to cancel the futures. + */ + private <F extends Future<T>, T> void cancelRemoteFutures( + final List<F> remoteFutures) { + + if (log.isInfoEnabled()) + log.info(""); + + for (F rf : remoteFutures) { + + try { + + rf.cancel(true/* mayInterruptIfRunning */); + + } catch (Throwable t) { + + // ignored (to be robust). + } - + } + } + + public BarrierState() { + + token = getQuorum().token(); + + getQuorum().assertLeader(token); + + // Local HA service implementation (non-Remote). + quorumService = getQuorum().getClient(); + + // The services joined with the met quorum, in their join order. + joinedServiceIds = getQuorum().getJoined(); + + // Note: Local method call. + timestampOnLeader = getTransactionManager().nextTimestamp(); + + final ICommitRecord commitRecord = getEarliestVisibleCommitRecord(); + + final long commitTime = commitRecord == null ? 0L : commitRecord + .getTimestamp(); + + final long commitCounter = commitRecord == null ? 0L : commitRecord + .getCommitCounter(); + + this.leadersValue = new HANotifyReleaseTimeRequest( + quorumService.getServiceId(), commitTime, commitCounter, + timestampOnLeader); + /* - * HA Quorum Overrides. - * - * Note: The basic pattern is that the quorum must be met, a leader - * executes the operation directly, and a follower delegates the - * operation to the leader. This centralizes the decisions about the - * open transactions, and the read locks responsible for pinning - * commit points, on the leader. - * - * If the journal is not highly available, then the request is - * passed to the base class (JournalTransactionService, which - * extends AbstractTransactionService). + * Only the followers will countDown() at the barrier. The leader + * will await() until the barrier breaks. */ - - @Override - public long newTx(final long timestamp) { + final int nparties = joinedServiceIds.length - 1; - final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); + barrier = new CyclicBarrier(nparties, this); - if (quorum == null) { + } - // Not HA. - return this._newTx(timestamp); + /** + * Find the minimum value across the responses when the {@link #barrier} + * breaks. + * + * TODO Check the timestamps for validity on the follower as well. + */ + @Override + public void run() { - } + // This is the timestamp from the BarrierState ctor. + final long timeLeader = leadersValue.getTimestampOnFollower(); + + // Start with the leader's value (from ctor). + minimumResponse = leadersValue; - final long token = getQuorumToken(); + for (IHANotifyReleaseTimeRequest response : responses.values()) { - if (quorum.getMember().isLeader(token)) { + if (minimumResponse.getCommitCounter() > response + .getCommitCounter()) { - // HA and this is the leader. - return this._newTx(timestamp); + minimumResponse = response; } - + /* - * The transaction needs to be allocated by the leader. - * - * Note: Heavy concurrent query on a HAJournal will pin history - * on the leader. However, the lastReleaseTime will advance - * since clients will tend to read against the then current - * lastCommitTime, so we will still recycle the older commit - * points once there is no longer an active reader for those - * commit points. + * Verify that the timestamp from the ctor is BEFORE the + * timestamp assigned by the follower for its response. */ - - final HAGlue leaderService = quorum.getMember() - .getLeader(token); - - final long tx; - try { + assertBefore(timeLeader, response.getTimestampOnFollower()); - // delegate to the quorum leader. - tx = leaderService.newTx(timestamp); + } - } catch (IOException e) { - - throw new RuntimeException(e); - - } + // Restate the consensus as an appropriate message object. + consensus = new HANotifyReleaseTimeResponse( + minimumResponse.getCommitTime(), + minimumResponse.getCommitCounter()); - // Make sure the quorum is still valid. - quorum.assertQuorum(token); + } - return tx; + /** + * Send an {@link IHAGatherReleaseTimeRequest} message to each follower. + * Block until the responses are received. + * + * TODO Timeout on duration that we will wait for the followers to + * response? (Probably not, this is very similar to the 2-phase commit). + * + * FIXME Like the 2-phase commit, the overall protocol should succeed if + * we can get ((k+1)/2) services that do not fail this step. Thus for + * HA3, we should allow one error on a follower, the leader is sending + * the messages and is presumed to succeed, and one follower COULD fail + * without failing the protocol. If the protocol does fail we have to + * fail the commit, to getting this right is NECESSARY. At a mimimum, we + * must not fail if all joined services on entry to this method respond + * without failing (that is, succeed if no services fail during this + * protocol). [Review further whether we can allow the 2-phase commit to + * rely on a service that was not joined when we took this step. We + * probably can given that the serviceJoin() code path of the follower + * is MUTEX with the negotiation of the consensus releaseTime value so + * long as the service uses an appropriate releaseTime when it joins. + * Which it should do, but review this closely.] + */ + private void messageFollowers(final long token) throws IOException { - } - - /** - * Core impl. - * <p> - * This code pre-increments the active transaction count within the - * RWStore before requesting a new transaction from the transaction - * service. This ensures that the RWStore does not falsely believe - * that there are no open transactions during the call to - * AbstractTransactionService#newTx(). - * <p> - * Note: This code was moved into the inner class extending the - * {@link JournalTransactionService} in order to ensure that we - * follow this pre-incremental pattern for an {@link HAJournal} as - * well. - * - * @see <a - * href="https://sourceforge.net/apps/trac/bigdata/ticket/440#comment:13"> - * BTree can not be case to Name2Addr </a> - * @see <a - * href="https://sourceforge.net/apps/trac/bigdata/ticket/530"> - * Journal HA </a> - */ - private final long _newTx(final long timestamp) { + getQuorum().assertLeader(token); - IRawTx tx = null; - try { - - if (getBufferStrategy() instanceof IRWStrategy) { + final List<Future<Void>> remoteFutures = new LinkedList<Future<Void>>(); + + try { - // pre-increment the active tx count. - tx = ((IRWStrategy) getBufferStrategy()).newTx(); - } + final IHAGatherReleaseTimeRequest msg = new HAGatherReleaseTimeRequest( + token, timestampOnLeader); - return super.newTx(timestamp); + // Do not send message to self (leader is at index 0). + for (int i = 1; i < joinedServiceIds.length; i++) { - } finally { + final UUID serviceId = joinedServiceIds[i]; - if (tx != null) { + /* + * Runnable which will execute this message on the remote + * service. + */ + final HATXSGlue service = getService(serviceId); + final Future<Void> rf = service.gatherMinimumVisibleCommitTime(msg); - /* - * If we had pre-incremented the transaction counter in - * the RWStore, then we decrement it before leaving this - * method. - */ + // add to list of futures we will check. + remoteFutures.add(rf); - tx.close(); + } + /* + * Check the futures for the other services in the quorum. + */ + final List<Throwable> causes = new LinkedList<Throwable>(); + for (Future<Void> rf : remoteFutures) { + boolean success = false; + try { + rf.get(); + success = true; + } catch (InterruptedException ex) { + log.error(ex, ex); + causes.add(ex); + } catch (ExecutionException ex) { + log.error(ex, ex); + causes.add(ex); + } finally { + if (!success) { + // Cancel the req... [truncated message content] |
From: <tho...@us...> - 2013-04-18 11:58:35
|
Revision: 7055 http://bigdata.svn.sourceforge.net/bigdata/?rev=7055&view=rev Author: thompsonbry Date: 2013-04-18 11:58:27 +0000 (Thu, 18 Apr 2013) Log Message: ----------- Added HAGlue.getHAStatus() method to the RMI interface to provide simplified reporting through RMI. Refactored code in the HAStatusServlet. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java Added Paths: ----------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAStatusEnum.java Removed Paths: ------------- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -148,16 +148,26 @@ int getNSSPort() throws IOException; /** - * The {@link RunState} of the service. + * The {@link RunState} of the service - this does NOT tell you whether the + * service is ready to act as a leader or follower. */ RunState getRunState() throws IOException; /** - * The extended run state of the service. + * The extended run state of the service - this embeds more information but + * is not designed for progamatic interpretation. */ String getExtendedRunState() throws IOException; /** + * A simplified summary of the HA status of the service. This may be used to + * reliably decide whether the service is the {@link HAStatusEnum#Leader}, a + * {@link HAStatusEnum#Follower}, or {@link HAStatusEnum#NotReady}. This is + * exposed both here (an RMI interface) and by the REST API. + */ + HAStatusEnum getHAStatus() throws IOException; + + /** * Compute the digest of the entire backing store - <strong>THIS METHOD IS * ONLY FOR DIAGNOSTIC PURPOSES.</strong> * <p> Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -251,6 +251,11 @@ } @Override + public HAStatusEnum getHAStatus() throws IOException { + return delegate.getHAStatus(); + } + + @Override public Future<IHASendStoreResponse> sendHAStore(IHARebuildRequest msg) throws IOException { return delegate.sendHAStore(msg); Copied: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAStatusEnum.java (from rev 7048, branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java) =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAStatusEnum.java (rev 0) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAStatusEnum.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -0,0 +1,45 @@ +/** +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.ha; + +/** + * An enumeration of HA status values. The set of enumerated states available + * here is intentionally a simplification of the internal states of the HA + * service. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public enum HAStatusEnum { + /** + * Ready to handle requests as the quorum leader. + */ + Leader, + /** + * Ready to handle requests as a quorum follower. + */ + Follower, + /** + * Not ready to handle HA read/write requests. + */ + NotReady; +} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -96,6 +96,7 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.Instrument; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.HATXSGlue; import com.bigdata.ha.QuorumService; import com.bigdata.ha.RunState; @@ -5638,6 +5639,57 @@ } @Override + public HAStatusEnum getHAStatus() { + + final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); + + final QuorumService<HAGlue> quorumService = quorum.getClient(); + + // check, but do not wait. + final long haReadyToken = AbstractJournal.this.getHAReady(); + + final HAStatusEnum status; + + if (haReadyToken == Quorum.NO_QUORUM) { + + // Quorum is not met (as percieved by the HAJournal). + status = HAStatusEnum.NotReady; + + } else { + + if (quorumService.isLeader(haReadyToken)) { + + // Service is leader. + status = HAStatusEnum.Leader; + + } else if (quorumService.isFollower(haReadyToken)) { + + // Service is follower. + status = HAStatusEnum.Follower; + + } else { + + /* + * awaitHAReady() should only return successfully (and hence + * haReadyToken should only be a valid token) if the service was + * elected as either a leader or a follower. However, it is + * possible for the service to have concurrently left the met + * quorum, in which case the if/then/else pattern will fall + * through to this code path. + */ + + // Quorum is not met (as percieved by the HAJournal). + status = HAStatusEnum.NotReady; + + } + + } + + return status; + + } + + @Override public long awaitHAReady(final long timeout, final TimeUnit units) throws AsynchronousQuorumCloseException, InterruptedException, TimeoutException { Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -66,6 +66,7 @@ import org.apache.zookeeper.data.ACL; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.RunState; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.ha.msg.HASnapshotDigestRequest; @@ -86,7 +87,6 @@ import com.bigdata.quorum.QuorumClient; import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.zk.ZKQuorumImpl; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.service.jini.JiniClientConfig; import com.bigdata.service.jini.RemoteDestroyAdmin; import com.bigdata.util.InnerCause; Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -53,6 +53,7 @@ import com.bigdata.btree.BytesUtil; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.msg.HADigestRequest; import com.bigdata.ha.msg.HALogDigestRequest; import com.bigdata.ha.msg.HARootBlockRequest; @@ -64,7 +65,6 @@ import com.bigdata.rdf.sail.webapp.NanoSparqlServer; import com.bigdata.rdf.sail.webapp.client.ConnectOptions; import com.bigdata.rdf.sail.webapp.client.DefaultClientConnectionManagerFactory; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.HttpException; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; import com.bigdata.util.InnerCause; Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -34,10 +34,10 @@ import org.apache.log4j.Logger; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.journal.IRootBlockView; import com.bigdata.quorum.Quorum; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -36,11 +36,11 @@ import net.jini.config.Configuration; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.journal.AbstractJournal; import com.bigdata.quorum.Quorum; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -34,11 +34,11 @@ import net.jini.config.Configuration; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.ha.msg.HASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.journal.IRootBlockView; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; /** * Test suites for the {@link IRestorePolicy}. Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -36,6 +36,7 @@ import net.jini.config.Configuration; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.ha.msg.HASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; @@ -43,7 +44,6 @@ import com.bigdata.journal.IRootBlockView; import com.bigdata.journal.Journal; import com.bigdata.quorum.Quorum; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; /** Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -34,8 +34,8 @@ import net.jini.core.lookup.ServiceID; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.msg.HADigestRequest; -import com.bigdata.rdf.sail.webapp.client.HAStatusEnum; /** * Life cycle and related tests for a single remote {@link HAJournalServer} out Deleted: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java 2013-04-17 23:06:33 UTC (rev 7054) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/client/HAStatusEnum.java 2013-04-18 11:58:27 UTC (rev 7055) @@ -1,45 +0,0 @@ -/** -Copyright (C) SYSTAP, LLC 2006-2007. 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 -*/ -package com.bigdata.rdf.sail.webapp.client; - -/** - * An enumeration of HA status values available from the REST API. The set of - * enumerated states available here is intentionally a simplification of the - * internal states of the HA service. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - */ -public enum HAStatusEnum { - /** - * Ready to handle requests as the quorum leader. - */ - Leader, - /** - * Ready to handle requests as a quorum follower. - */ - Follower, - /** - * Not ready to handle HA read/write requests. - */ - NotReady; -} This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-18 15:44:11
|
Revision: 7058 http://bigdata.svn.sourceforge.net/bigdata/?rev=7058&view=rev Author: thompsonbry Date: 2013-04-18 15:44:00 +0000 (Thu, 18 Apr 2013) Log Message: ----------- Modified the NSS to reject read requests and write requests based on getHAStatus() and added that method to the HAGlue interface. The NSS uses a local method call rather than the proxy for the HAGlue interface so this is a low latency test. It is also more correct than the previous test which could allow an operation before the haReady token was set. Protected the serviceJoin() when joining with a met quorum by running it in a critical section protected by the barrierLock in the InnerJournalTransactionService. The barrierLock is held locally by the follower through gather() so this is a local method call rather than an RMI back to the leader. Removed the IHATXSLockRequest and associated method since they are no longer necessary with this change. Added doLocalAbort() into the AbstractJournal.setQuorumToken() method on a quorum break. Extracted abortAllTx() method from AbstractTransactionService. It is now invoked from setQuorumToken() to cancel running transactions if the quorum breaks. See https://sourceforge.net/apps/trac/bigdata/ticket/623 (HA TXS) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractTransactionService.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataServlet.java Removed Paths: ------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlue.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -230,7 +230,11 @@ * if a timeout expires while awaiting the global lock. * @throws InterruptedException * if interrupted while awaiting the lock. + * + * @deprecated This is no longer necessary to support backups since we can + * now take snapshots without suspending writers. */ + @Deprecated Future<Void> globalWriteLock(IHAGlobalWriteLockRequest req) throws IOException, TimeoutException, InterruptedException; Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -59,7 +59,6 @@ import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; -import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; import com.bigdata.ha.msg.IHAWriteSetStateResponse; @@ -182,11 +181,11 @@ return delegate.notifyEarliestCommitTime(req); } - @Override - public Future<Void> getTXSCriticalSectionLockOnLeader( - final IHATXSLockRequest req) throws IOException { - return delegate.getTXSCriticalSectionLockOnLeader(req); - } +// @Override +// public Future<Void> getTXSCriticalSectionLockOnLeader( +// final IHATXSLockRequest req) throws IOException { +// return delegate.getTXSCriticalSectionLockOnLeader(req); +// } // @Override // public long nextTimestamp() throws IOException { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HATXSGlue.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -26,12 +26,10 @@ import java.io.IOException; import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.Future; -import java.util.concurrent.locks.Lock; import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; -import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.journal.ITransactionService; /** @@ -85,25 +83,25 @@ IHANotifyReleaseTimeRequest req) throws IOException, InterruptedException, BrokenBarrierException; - /** - * Return an asynchronous {@link Future} for a task executing on the quorum - * leader that holds the {@link Lock} protecting the critical section - * contended by - * {@link #gatherMinimumVisibleCommitTime(IHAGatherReleaseTimeRequest)}. - * This lock is used by a service that wishes to join a met quorum to - * prevent a service join that is concurrent with the critical section in - * which the already joined services agree on the new <i>releaseTime</i>. - * <p> - * Note: The leader MAY impose a timeout on this lock such that a service - * which does not complete its work in a timely fashion will lose the lock. - * The service MUST check the {@link Future} and verify that it is still - * running. If the {@link Future#isCancelled()}, then the service MUST NOT - * complete its service join (or must do a service leave to exit the - * quorum). - * - * @return The {@link Future} for the task holding that lock. - */ - Future<Void> getTXSCriticalSectionLockOnLeader(IHATXSLockRequest req) - throws IOException; +// /** +// * Return an asynchronous {@link Future} for a task executing on the quorum +// * leader that holds the {@link Lock} protecting the critical section +// * contended by +// * {@link #gatherMinimumVisibleCommitTime(IHAGatherReleaseTimeRequest)}. +// * This lock is used by a service that wishes to join a met quorum to +// * prevent a service join that is concurrent with the critical section in +// * which the already joined services agree on the new <i>releaseTime</i>. +// * <p> +// * Note: The leader MAY impose a timeout on this lock such that a service +// * which does not complete its work in a timely fashion will lose the lock. +// * The service MUST check the {@link Future} and verify that it is still +// * running. If the {@link Future#isCancelled()}, then the service MUST NOT +// * complete its service join (or must do a service leave to exit the +// * quorum). +// * +// * @return The {@link Future} for the task holding that lock. +// */ +// Future<Void> getTXSCriticalSectionLockOnLeader(IHATXSLockRequest req) +// throws IOException; } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HANotifyReleaseTimeRequest.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -62,7 +62,7 @@ } @Override - public long getTimestampOnFollower() { + public long getTimestamp() { return timestampOnFollower; } Deleted: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/HATXSLockRequest.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -1,44 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2007. 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 -*/ -package com.bigdata.ha.msg; - -public class HATXSLockRequest implements IHATXSLockRequest { - - /** - * - */ - private static final long serialVersionUID = 1L; - - private final long token; - - public HATXSLockRequest(final long token) { - this.token = token; - } - - @Override - public long token() { - return token; - } - -} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHANotifyReleaseTimeRequest.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -67,6 +67,6 @@ * the new release time. This is used to detect problems where the clocks * are not synchronized on the services. */ - public long getTimestampOnFollower(); + public long getTimestamp(); } Deleted: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/msg/IHATXSLockRequest.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -1,41 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2007. 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 -*/ -package com.bigdata.ha.msg; - -/** - * Message used to request a lock on the quorum leader to make a serviceJoin - * mutex with the critical section for the services joined with the met quorum - * make a decision concerning the new minimum visible commit point on the - * quorum. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - */ -public interface IHATXSLockRequest extends IHAMessage { - - /** - * The quorum token for which this request is valid. - */ - public long token(); - -} Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -129,7 +129,6 @@ import com.bigdata.ha.msg.IHASnapshotRequest; import com.bigdata.ha.msg.IHASnapshotResponse; import com.bigdata.ha.msg.IHASyncRequest; -import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.ha.msg.IHAWriteMessage; import com.bigdata.ha.msg.IHAWriteSetStateRequest; import com.bigdata.ha.msg.IHAWriteSetStateResponse; @@ -161,6 +160,7 @@ import com.bigdata.rwstore.sector.MemStrategy; import com.bigdata.rwstore.sector.MemoryManager; import com.bigdata.service.AbstractHATransactionService; +import com.bigdata.service.AbstractTransactionService; import com.bigdata.service.IBigdataFederation; import com.bigdata.stream.Stream; import com.bigdata.util.ChecksumUtility; @@ -5031,13 +5031,15 @@ * * We do not need to discard read-only tx since the committed * state should remain valid even when a quorum is lost. - * - * FIXME HA TXS INTEGRATION (discard running read/write tx). + * However, it would be a bit odd to leave read-only + * transactions running if you could not start a new read-only + * because the quorum is not met. */ + ((AbstractTransactionService) getLocalTransactionManager() + .getTransactionService()).abortAllTx(); - // local abort (no quorum, so we can do 2-phase abort). -// _abort(); -// getLocalTransactionManager(). + // local abort (no quorum, so 2-phase abort not required). + doLocalAbort(); // FIXME HA : local abort on quorum break -or- service leave? /* * Note: We can not re-cast our vote until our last vote is @@ -5643,6 +5645,13 @@ final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); + if (quorum == null) { + + // Not HA. + return null; + + } + final QuorumService<HAGlue> quorumService = quorum.getClient(); // check, but do not wait. @@ -6500,19 +6509,19 @@ } - @Override - public Future<Void> getTXSCriticalSectionLockOnLeader( - final IHATXSLockRequest req) throws IOException { +// @Override +// public Future<Void> getTXSCriticalSectionLockOnLeader( +// final IHATXSLockRequest req) throws IOException { +// +// final Future<Void> f = ((HATXSGlue) AbstractJournal.this +// .getLocalTransactionManager().getTransactionService()) +// .getTXSCriticalSectionLockOnLeader(req); +// +// // Note: MUST be an asynchronous Future!!! +// return getProxy(f, true/* asynchronousFuture */); +// +// } - final Future<Void> f = ((HATXSGlue) AbstractJournal.this - .getLocalTransactionManager().getTransactionService()) - .getTXSCriticalSectionLockOnLeader(req); - - // Note: MUST be an asynchronous Future!!! - return getProxy(f, true/* asynchronousFuture */); - - } - /* * IService */ Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -41,7 +41,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.FutureTask; -import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.Semaphore; @@ -70,6 +69,7 @@ import com.bigdata.counters.CounterSet; import com.bigdata.counters.httpd.CounterSetHTTPD; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.HATXSGlue; import com.bigdata.ha.QuorumService; import com.bigdata.ha.msg.HAGatherReleaseTimeRequest; @@ -78,7 +78,6 @@ import com.bigdata.ha.msg.IHAGatherReleaseTimeRequest; import com.bigdata.ha.msg.IHANotifyReleaseTimeRequest; import com.bigdata.ha.msg.IHANotifyReleaseTimeResponse; -import com.bigdata.ha.msg.IHATXSLockRequest; import com.bigdata.journal.jini.ha.HAJournal; import com.bigdata.quorum.Quorum; import com.bigdata.quorum.QuorumException; @@ -460,14 +459,12 @@ /** * Find the minimum value across the responses when the {@link #barrier} * breaks. - * - * TODO Check the timestamps for validity on the follower as well. */ @Override public void run() { // This is the timestamp from the BarrierState ctor. - final long timeLeader = leadersValue.getTimestampOnFollower(); + final long timeLeader = leadersValue.getTimestamp(); // Start with the leader's value (from ctor). minimumResponse = leadersValue; @@ -485,7 +482,7 @@ * Verify that the timestamp from the ctor is BEFORE the * timestamp assigned by the follower for its response. */ - assertBefore(timeLeader, response.getTimestampOnFollower()); + assertBefore(timeLeader, response.getTimestamp()); } @@ -503,20 +500,21 @@ * TODO Timeout on duration that we will wait for the followers to * response? (Probably not, this is very similar to the 2-phase commit). * - * FIXME Like the 2-phase commit, the overall protocol should succeed if - * we can get ((k+1)/2) services that do not fail this step. Thus for - * HA3, we should allow one error on a follower, the leader is sending - * the messages and is presumed to succeed, and one follower COULD fail - * without failing the protocol. If the protocol does fail we have to - * fail the commit, to getting this right is NECESSARY. At a mimimum, we - * must not fail if all joined services on entry to this method respond - * without failing (that is, succeed if no services fail during this - * protocol). [Review further whether we can allow the 2-phase commit to - * rely on a service that was not joined when we took this step. We - * probably can given that the serviceJoin() code path of the follower - * is MUTEX with the negotiation of the consensus releaseTime value so - * long as the service uses an appropriate releaseTime when it joins. - * Which it should do, but review this closely.] + * FIXME HA TXS: Like the 2-phase commit, the overall protocol should + * succeed if we can get ((k+1)/2) services that do not fail this step. + * Thus for HA3, we should allow one error on a follower, the leader is + * sending the messages and is presumed to succeed, and one follower + * COULD fail without failing the protocol. If the protocol does fail we + * have to fail the commit, to getting this right is NECESSARY. At a + * mimimum, we must not fail if all joined services on entry to this + * method respond without failing (that is, succeed if no services fail + * during this protocol). [Review further whether we can allow the + * 2-phase commit to rely on a service that was not joined when we took + * this step. We probably can given that the serviceJoin() code path of + * the follower is MUTEX with the negotiation of the consensus + * releaseTime value so long as the service uses an appropriate + * releaseTime when it joins. Which it should do, but review this + * closely.] */ private void messageFollowers(final long token) throws IOException { @@ -624,32 +622,30 @@ } - /** - * The maximum error allowed (milliseconds) in the clocks. - * - * TODO Should this be zero? - */ - private static final long epsilon = 3; - - /** - * Assert that t1 LT t2. - * - * @param t1 - * @param t2 - * - * @throws ClocksNotSynchronizedException - */ - private void assertBefore(final long t1, final long t2) { + } - if (t1 < t2) - return; +// /** +// * The maximum error allowed (milliseconds) in the clocks. +// */ +// private static final long epsilon = 3; + + /** + * Assert that t1 LT t2. + * + * @param t1 + * @param t2 + * + * @throws ClocksNotSynchronizedException + */ + private void assertBefore(final long t1, final long t2) { - throw new ClocksNotSynchronizedException(); + if (t1 < t2) + return; - } - - } + throw new ClocksNotSynchronizedException(); + } + /** * {@inheritDoc} * <p> @@ -710,6 +706,8 @@ */ final private ReentrantLock barrierLock = new ReentrantLock(); +// final private Condition barrierBroke = barrierLock.newCondition(); + /** * This is used to coordinate the protocol for achiving an atomic * consensus on the new <i>releaseTime</i> for the services joined with @@ -717,6 +715,18 @@ */ final private AtomicReference<BarrierState> barrierRef = new AtomicReference<BarrierState>(); + @Override + public void executeWithBarrierLock(final Runnable r) { + + barrierLock.lock(); + try { + r.run(); + } finally { + barrierLock.unlock(); + } + + } + /** * {@inheritDoc} * <p> @@ -745,7 +755,7 @@ barrierLock.lock(); try { - + getQuorum().assertLeader(token); if (!barrierRef.compareAndSet(null/* expectedValue */, @@ -755,38 +765,40 @@ } - } finally { + try { - barrierLock.unlock(); - - } + /* + * Message the followers and block until the barrier breaks. + */ + barrierState.messageFollowers(token); - try { + } finally { + // Clear the barrierRef. + if (!barrierRef.compareAndSet(barrierState/* expected */, + null)) { + + throw new AssertionError(); + + } + + } + /* - * Message the followers and block until the barrier breaks. + * Update the release time on the leader */ - barrierState.messageFollowers(token); - } finally { + final long consensusValue = barrierState.consensus + .getCommitTime(); - // Clear the barrierRef. - if (!barrierRef.compareAndSet(barrierState/* expected */, null)) { + setReleaseTime(Math.max(0L, consensusValue - 1)); - throw new AssertionError(); + } finally { - } + barrierLock.unlock(); } - /* - * Update the release time on the leader - */ - - final long consensusValue = barrierState.consensus.getCommitTime(); - - setReleaseTime(Math.max(0L, consensusValue - 1)); - } /** @@ -809,8 +821,6 @@ if (newValue < oldValue) { /* - * FIXME HA TXS: Rollback releaseTime if commit fails. - * * Note: We might have to allow this to roll back the * release time if a commit fails. */ @@ -834,7 +844,12 @@ } } - + + /** + * {@inheritDoc} + * <p> + * "Gather" runs on the follower. + */ @Override public Future<Void> gatherMinimumVisibleCommitTime( final IHAGatherReleaseTimeRequest req) throws IOException { @@ -880,9 +895,14 @@ public Void call() throws Exception { + final long now = nextTimestamp(); + + // Verify event on leader occurs before event on follower. + assertBefore(req.getTimestampOnLeader(), now); + final long token = req.token(); - barrierLock.lock(); + barrierLock.lock(); // take lock on follower! try { @@ -982,146 +1002,168 @@ } + /** + * Helper method returns the {@link HAStatusEnum} -or- <code>null</code> + * if this is not HA. This is a <em>low latency local</em> method call. + * The code path is against the local (non-remote) HAGlue obeject. It is + * NOT an RMI. + */ + private final HAStatusEnum getHAStatus() { + + // Quorum iff HA. + final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); + + if(quorum == null) { + + // Not HA. + return null; + + } + + // Note: Invocation against local HAGlue object (NOT RMI). + try { + + return quorum.getClient().getService().getHAStatus(); + + } catch (IOException ex) { + + // Note: Exception is never thrown (not RMI). + throw new RuntimeException(ex); + + } + + } + @Override - public Future<Void> getTXSCriticalSectionLockOnLeader( - final IHATXSLockRequest req) throws IOException { + public long newTx(final long timestamp) { - getQuorum().assertLeader(req.token()); + if (TimestampUtility.isReadWriteTx(timestamp)) { - /* - * We need to submit a task that owns the lock for the critical - * section. - * - * Note: The task needs to own the lock BEFORE it runs and unlock - * the lock when it is done. It needs to own the lock before it runs - * since the receipt of the Future for this task by the remote - * service is its permission to do the serviceJoin. - * - * Note: If we can not submit the task to the executor service, then - * we need to to release the lock (RejectedExecutionException). - * - * FIXME HA TXS: This code is broken. We are not asking the lock - * before the tasks executes and, if we did, then we would not own - * it in the thread in which we are executing the task and hence - * would be unable to release the lock in that thread. This may need - * to be coordinated using a Condition so we block in the RMI Thread - * (here) until the submitted task gains the lock in its own Thread. - * That condition could probably use a different Lock object to - * communicate, but it might be possible to do this using the - * barrierLock. - * - * FIXME HA TXS: An HAJournalServer that will join with a met quorum - * MUST obtain this lock on the leader before executing the - * serviceJoin. They MUST cancel the Future obtained from this - * method regardless. A timeout on the leader is necessary in case a - * network partition or sudden power failure otherwise prevents the - * remote service from releasing this lock since the leader can not - * go through a commit point while this lock is held. The service - * that is trying to join MUST verify that the obtained Future is - * not yet done (!isDone()) after it has successfully joined. If the - * Future.isDone() when it checks, then it effectively lost the lock - * and MUST do a serviceLeave() and then retry. - */ + // The caller has provided a TxId, not a timestamp. + throw new IllegalArgumentException(); + + } - // Note: Takes lock *here*. - final BarrierLockTask task = new BarrierLockTask(); + // The HAStatusEnum -or- null if not HA. + final HAStatusEnum haStatus = getHAStatus(); - try { + if (haStatus == null) { + + // Not HA. + return _newTx(timestamp); + + } - // Wrap. - final FutureTask<Void> ft = new FutureTask<Void>(task); + if (haStatus == HAStatusEnum.NotReady) { - // Submit for evaluation. - getExecutorService().submit(ft); + // Not ready. + throw new QuorumException(); - // Return Future to caller. Will be async RMI Future. - return ft; + } - } catch (RejectedExecutionException ex) { + if (timestamp == ITx.UNISOLATED && haStatus != HAStatusEnum.Leader) { - // Release the lock if we could not submit the task. - barrierLock.unlock(); + // Read/Write Tx starts are only allowed on the Leader. + throw new QuorumException("Not quorum leader"); - // Rethrow the exception. - throw ex; - } - } + if (timestamp == ITx.UNISOLATED || timestamp == ITx.READ_COMMITTED) { - /** - * Helper class used to implement the - * {@link HATXSGlue#getTXSCriticalSectionLockOnLeader(IHATXSLockRequest)} - * method. - */ - private class BarrierLockTask implements Callable<Void> { - - public BarrierLockTask() { + /* + * A read-write tx reads on the current commit point. + * + * A read-committed tx reads on the current commit point. + * + * The current commit point is always visible, so these requests + * are non-blocking. + * + * Note: We have verified that this service is the quorum leader + * above if the request is for a read-write tx. + */ + + return _newTx(timestamp); } + + /* + * The request is a read-only tx against some specific historical + * commit point. It will be allowed (without blocking at the + * barrier) if the commit point is known to be pinned based on + * either the minReleaseAge or the earliestActiveTx. We use the + * AbstractTransactionService's lock to make these inspections + * atomic. + */ - public Void call() throws Exception { + lock.lock(); // Note: AbstractTransactionService.lock - // block until we get the lock. - barrierLock.lock(); - try { - // Sleep until interrupted. - Thread.sleep(Long.MAX_VALUE); - } finally { - barrierLock.unlock(); - } + try { - return null; + final long now = nextTimestamp(); + + /* + * TODO Should we reject tx starts against future history? The + * AbstractTransactionService used to reject this case but that + * code has been commented out. Perhaps because of minor clock + * differences that could arise? + */ +// if (timestamp > now) { +// // Timestamp is in the future. +// throw new IllegalArgumentException(); +// } - } - - } // class BarrierLockTask - - @Override - public long newTx(final long timestamp) { + final long minReleaseAge = getMinReleaseAge(); - if (timestamp == ITx.UNISOLATED) { + final long ageOfTxView = now - timestamp; + if (ageOfTxView < minReleaseAge) { + + // Start tx. Commit point pinned by minReleaseAge. + return _newTx(timestamp); + + } + /* - * This is a request for a read/write transaction. + * Handle commit point pinned by earliestActiveTx's + * readsOnCommitTime. */ + { - final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); + final TxState state = getEarlestActiveTx(); - if (quorum != null) { + if (state != null && state.readsOnCommitTime <= timestamp) { - /* - * We must be the quorum leader. - */ + // Start Tx. Commit point pinned by earliestActiveTx. + return _newTx(timestamp); - final long token = getQuorumToken(); + } - getQuorum().assertLeader(token); + } + final IRootBlockView rootBlock = getRootBlockView(); + + if (rootBlock.getCommitCounter() == 0L) { + + // Start Tx. No commits so nothing could be released. + return _newTx(timestamp); + } + if (rootBlock.getLastCommitTime() <= timestamp) { + + // Tx reads on most recent commit point. + return _newTx(timestamp); + + } + + } finally { + + lock.unlock(); + } /* - * FIXME HA TXS : OPTIMIZE newTx() : We do need an HA code path - * here. However, it needs to be concerned with blocking the calling - * Thread IF the request can not be satisified in a non-blocking - * manner. - * - * FIXME It is safe if we always take the barrierLock here, but that - * is (MUCH) too conservative. We only need it if the TXS can not - * start the new Tx based on local state. - * - * FIXME A tx SHOULD start without this lock as long as its - * readsOnCommitTime would be GTE the readsOnCommitTime of the - * earliestActiveTx -OR- its readsOnCommitTime is LTE - * now-releaseAge. This decision needs to be pushed down into the - * TXS implementation class using AbstractTransactionService#lock. - * - * Leader: block on the barrierLock. - * - * Follower: block on the barrierLock (but the follower needs to - * hold this in gather()). + * Must block at barrier. */ barrierLock.lock(); @@ -1136,55 +1178,6 @@ } -// final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); -// -// if (quorum == null) { -// -// // Not HA. -// return this._newTx(timestamp); -// -// } -// -// final long token = getQuorumToken(); -// -// if (quorum.getMember().isLeader(token)) { -// -// // HA and this is the leader. -// return this._newTx(timestamp); -// -// } -// -// /* -// * The transaction needs to be allocated by the leader. -// * -// * Note: Heavy concurrent query on a HAJournal will pin history -// * on the leader. However, the lastReleaseTime will advance -// * since clients will tend to read against the then current -// * lastCommitTime, so we will still recycle the older commit -// * points once there is no longer an active reader for those -// * commit points. -// */ -// -// final HAGlue leaderService = quorum.getMember() -// .getLeader(token); -// -// final long tx; -// try { -// -// // delegate to the quorum leader. -// tx = leaderService.newTx(timestamp); -// -// } catch (IOException e) { -// -// throw new RuntimeException(e); -// -// } -// -// // Make sure the quorum is still valid. -// quorum.assertQuorum(token); -// -// return tx; - } /** @@ -1263,233 +1256,8 @@ return super.commit(tx); -// if (quorum.getMember().isLeader(token)) { -// -// // HA and this is the leader. -// return super.commit(tx); -// -// } -// -// /* -// * Delegate to the quorum leader. -// */ -// -// final HAGlue leaderService = quorum.getMember() -// .getLeader(token); -// -// final long commitTime; -// try { -// -// // delegate to the quorum leader. -// commitTime = leaderService.commit(tx); -// -// } catch (IOException e) { -// -// throw new RuntimeException(e); -// -// } -// -// // Make sure the quorum is still valid. -// quorum.assertQuorum(token); -// -// return commitTime; - } - -// @Override -// public void abort(final long tx) { -// -// final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); -// -// if (quorum != null) { -// -// final long token = getQuorumToken(); -// -// getQuorum().assertLeader(token); -// -// } -// -// super.abort(tx); -// -// return; -// -//// if (quorum.getMember().isLeader(token)) { -//// -//// // HA and this is the leader. -//// super.abort(tx); -//// -//// return; -//// -//// } -//// -//// /* -//// * Delegate to the quorum leader. -//// */ -//// -//// final HAGlue leaderService = quorum.getMember() -//// .getLeader(token); -//// -//// try { -//// -//// // delegate to the quorum leader. -//// leaderService.abort(tx); -//// -//// } catch (IOException e) { -//// -//// throw new RuntimeException(e); -//// -//// } -//// -//// // Make sure the quorum is still valid. -//// quorum.assertQuorum(token); -//// -//// return; -// -// } - -// @Override -// public void notifyCommit(final long commitTime) { -// -// final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); -// -// if (quorum != null) { -// -// final long token = getQuorumToken(); -// -// getQuorum().assertLeader(token); -// -// } -// -// super.notifyCommit(commitTime); -// -//// if (quorum.getMember().isLeader(token)) { -//// -//// // HA and this is the leader. -//// super.notifyCommit(commitTime); -//// -//// return; -//// -//// } -//// -//// /* -//// * Delegate to the quorum leader. -//// */ -//// -//// final HAGlue leaderService = quorum.getMember() -//// .getLeader(token); -//// -//// try { -//// -//// // delegate to the quorum leader. -//// leaderService.notifyCommit(commitTime); -//// -//// } catch (IOException e) { -//// -//// throw new RuntimeException(e); -//// -//// } -//// -//// // Make sure the quorum is still valid. -//// quorum.assertQuorum(token); -//// -//// return; -// -// } - -// @Override -// public long getReleaseTime() { -// -// final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); -// -// if (quorum == null) { -// -// // Not HA. -// return super.getReleaseTime(); -// -// } -// -// final long token = getQuorumToken(); -// -// if (quorum.getMember().isLeader(token)) { -// -// // HA and this is the leader. -// return super.getReleaseTime(); -// -// } -// -// /* -// * Delegate to the quorum leader. -// */ -// -// final HAGlue leaderService = quorum.getMember() -// .getLeader(token); -// -// final long releaseTime; -// try { -// -// // delegate to the quorum leader. -// releaseTime = leaderService.getReleaseTime(); -// -// } catch (IOException e) { -// -// throw new RuntimeException(e); -// -// } -// -// // Make sure the quorum is still valid. -// quorum.assertQuorum(token); -// -// return releaseTime; -// -// } -// -// @Override -// public long nextTimestamp(){ -// -// final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); -// -// if (quorum == null) { -// -// // Not HA. -// return super.nextTimestamp(); -// -// } -// -// final long token = getQuorumToken(); -// -// if (quorum.getMember().isLeader(token)) { -// -// // HA and this is the leader. -// return super.nextTimestamp(); -// -// } -// -// /* -// * Delegate to the quorum leader. -// */ -// -// final HAGlue leaderService = quorum.getMember() -// .getLeader(token); -// -// final long nextTimestamp; -// try { -// -// // delegate to the quorum leader. -// nextTimestamp = leaderService.nextTimestamp(); -// -// } catch (IOException e) { -// -// throw new RuntimeException(e); -// -// } -// -// // Make sure the quorum is still valid. -// quorum.assertQuorum(token); -// -// return nextTimestamp; -// -// } - + protected void activateTx(final TxState state) { if (txLog.isInfoEnabled()) txLog.info("OPEN : txId=" + state.tx @@ -2226,51 +1994,6 @@ */ final public long newTx(final long timestamp) { -// IRawTx tx = null; -// try { -// if (getBufferStrategy() instanceof IRWStrategy) { -// -// /* -// * This code pre-increments the active transaction count within -// * the RWStore before requesting a new transaction from the -// * transaction service. This ensures that the RWStore does not -// * falsely believe that there are no open transactions during -// * the call to AbstractTransactionService#newTx(). -// * -// * @see https://sourceforge.net/apps/trac/bigdata/ticket/440#comment:13 -// */ -// tx = ((IRWStrategy) getBufferStrategy()).newTx(); -// } -// try { -// -// return getTransactionService().newTx(timestamp); -// -// } catch (IOException ioe) { -// -// /* -// * Note: IOException is declared for RMI but will not be thrown -// * since the transaction service is in fact local. -// */ -// -// throw new RuntimeException(ioe); -// -// } -// -// } finally { -// -// if (tx != null) { -// -// /* -// * If we had pre-incremented the transaction counter in the -// * RWStore, then we decrement it before leaving this method. -// */ -// -// tx.close(); -// -// } -// -// } - /* * Note: The RWStore native tx pre-increment logic is now handled by * _newTx() in the inner class that extends JournalTransactionService. Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -55,4 +55,9 @@ abstract public void updateReleaseTimeConsensus() throws IOException, TimeoutException, InterruptedException; + /** + * Used to make a serviceJoin() MUTEX with the consensus protocol. + */ + abstract public void executeWithBarrierLock(Runnable r); + } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractTransactionService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractTransactionService.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractTransactionService.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -505,6 +505,30 @@ setRunState(TxServiceRunState.ShutdownNow); + // Abort all active transactions. + abortAllTx(); + + super.shutdownNow(); + + setRunState(TxServiceRunState.Halted); + + } finally { + + lock.unlock(); + + } + + } + + /** + * Abort all active transactions. + */ + public void abortAllTx() { + + lock.lock(); + + try { + for (long tx : activeTx.keySet()) { final TxState state = activeTx.get(tx); @@ -533,15 +557,15 @@ abortImpl(state); assert state.isAborted() : state.toString(); - - } catch(Throwable t) { - + + } catch (Throwable t) { + log.error(state.toString(), t); - + } finally { deactivateTx(state); - + } } @@ -555,7 +579,7 @@ * need to acquire it here. */ updateReleaseTime(Math.abs(state.tx)); - + } } // foreach tx in activeTx @@ -571,19 +595,14 @@ + activeTx.size()); } - - super.shutdownNow(); - - setRunState(TxServiceRunState.Halted); - } finally { lock.unlock(); } - + } - + /** * Immediate/fast shutdown of the service and then destroys any persistent * state associated with the service. @@ -812,6 +831,29 @@ // return earliestTxStartTime; // // } + + /** + * Return the {@link TxState} for the earliest active Tx -or- + * <code>null</code> if there is no active tx. + * <p> + * Note: The {@link #lock} is required in order to make atomic decisions + * about the earliest active tx. Without the {@link #lock}, the tx could + * stop or a new tx could start, thereby invalidating the "easliest active" + * guarantee. + * + * @throws IllegalMonitorStateException + * unless the {@link #lock} is held by the caller. + */ + protected TxState getEarlestActiveTx() { + + if (!lock.isHeldByCurrentThread()) + throw new IllegalMonitorStateException(); + + final TxState state = getTxState(earliestOpenTxId); + + return state; + + } /** * The minimum over the absolute values of the active transactions and ZERO Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -425,7 +425,7 @@ * * TODO Lift into HAJournalServer.quorumBreak() handler? * - * TODO This will not be called if the quorum remains met but the + * FIXME This will not be called if the quorum remains met but the * local service leaves the quorum. However, we should still cancel * a running snapshot if that occurs. */ @@ -1054,11 +1054,11 @@ * connections distinct KBs per the ticket below, then we will need to * have a different global write lock - perhaps via the * {@link WriteExecutorService}. - * <p> - * In fact, we could deprecate this method. It is no longer necessary to - * support backups since we can now take snapshots without suspending - * writers. * + * @deprecated This method is no longer necessary to support backups + * since we can now take snapshots without suspending + * writers. + * * @see https://sourceforge.net/apps/trac/bigdata/ticket/566 ( * Concurrent unisolated operations against multiple KBs on the * same Journal) Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -99,6 +99,7 @@ import com.bigdata.rdf.sail.webapp.ConfigParams; import com.bigdata.rdf.sail.webapp.NanoSparqlServer; import com.bigdata.rwstore.RWStore; +import com.bigdata.service.AbstractHATransactionService; import com.bigdata.service.jini.FakeLifeCycle; import com.bigdata.service.jini.RemoteAdministrable; import com.bigdata.service.jini.RemoteDestroyAdmin; @@ -1125,27 +1126,6 @@ } -// @Override -// public void start(final Quorum<?,?> quorum) { -// -// if (haLog.isTraceEnabled()) -// log.trace("START"); -// -// super.start(quorum); -// -// // Note: It appears to be a problem to do this here. Maybe because -// // the watcher is not running yet? Could submit a task that could -// // await an appropriate condition to start.... -//// final QuorumActor<?, ?> actor = quorum.getActor(); -//// actor.memberAdd(); -//// actor.pipelineAdd(); -//// actor.castVote(journal.getLastCommitTime()); -// -//// // Inform the Journal about the current token (if any). -//// journal.setQuorumToken(quorum.token()); -// -// } - @Override public void quorumMeet(final long token, final UUID leaderId) { @@ -1229,6 +1209,44 @@ * <em>EXCEPT</em> the current one. */ @Override + public void serviceLeave() { + + super.serviceLeave(); + + // FIXME serviceLeave() needs event handler. +// // Submit task to handle this event. +// server.singleThreadExecutor.execute(new MonitoredFutureTask<Void>( +// new ServiceLeaveTask())); + } + + private class ServiceLeaveTask implements Callable<Void> { + public Void call() throws Exception { + /* + * Set token. Journal will notice that it is no longer + * "HA Ready" + * + * FIXME AbstractJournal.setQuorumToken() must detect case where + * it transitions from a met quorum through a service leave and + * clears its haReady token. + */ + journal.setQuorumToken(getQuorum().token()); + try { + journal.getHALogWriter().disable(); + } catch (IOException e) { + haLog.error(e, e); + } + enterRunState(new SeekConsensusTask()); + return null; + } + } + + /** + * {@inheritDoc} + * <p> + * If there is a fully met quorum, then we can purge all HA logs + * <em>EXCEPT</em> the current one. + */ + @Override public void serviceJoin() { super.serviceJoin(); @@ -2373,6 +2391,10 @@ /* * Cast the leader's vote, join with the met quorum, and * transition to RunMet. + * + * Note: We are blocking the write pipeline on this code path + * because the [logLock] is held. We verify the pre-conditions + * for the join with the met quorum while holding the [logLock]. */ doCastLeadersVoteAndServiceJoin(token); @@ -2782,6 +2804,14 @@ /** * Cast the leader's vote, join with the met quorum, and transition to * RunMet. + * <p> + * Note: The write pipeline will be blocked during this method. This is + * achieved via two different mechanisms. If the call stack goes through + * {@link #handleReplicatedWrite(IHASyncRequest, IHAWriteMessage, ByteBuffer)} + * , then the write pipeline is naturally blocked. For + * {@link #conditionalJoinWithMetQuorum(HAGlue, long, long)}, we are + * holding the {@link #logLock} which ensures that new write messages + * can not be processed. * * @param token * The token that must remain valid throughout this @@ -2821,24 +2851,40 @@ * Note: This will throw an exception if this services is not in the * consensus. * - * Note: We are BLOCKING the pipeline while we wait here (since we - * are handling a replicated write). + * Note: The write pipeline is BLOCKED. Either we are handling a + * replicated write -or- we are holding the logLock (or both). * + * Note: The serviceJoin() needs to be MUTEX with the critical + * section of the consensus protocol to identify the new release + * time. This is necessary to ensure that the follower does not + * start a new Tx against a commit point after the follower has + * notified the leader about its earliest visible commit point and + * before the leader has notified the followers about the new + * consensus release time. + * * TODO What happens if we are blocked here? - * - * FIXME HA TXS: CRITICAL SECTION exclusion here. */ - getActor().serviceJoin(); + ((AbstractHATransactionService) journal.getTransactionService()) + .executeWithBarrierLock(new Runnable() { - // Verify that the quorum is valid. - getQuorum().assertQuorum(token); + public void run() { + + // Verify that the quorum is valid. + getQuorum().assertQuorum(token); - // Set the token on the journal. - journal.setQuorumToken(token); + getActor().serviceJoin(); - // Verify that the quorum is valid. - getQuorum().assertQuorum(token); - + // Verify that the quorum is valid. + getQuorum().assertQuorum(token); + + // Set the token on the journal. + journal.setQuorumToken(token); + + // Verify that the quorum is valid. + getQuorum().assertQuorum(token); + } + }); + // Transition to RunMet. enterRunState(new RunMetTask(token, leaderId)); Modified: branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataServlet.java =================================================================== --- branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataServlet.java 2013-04-18 14:21:23 UTC (rev 7057) +++ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataServlet.java 2013-04-18 15:44:00 UTC (rev 7058) @@ -38,6 +38,7 @@ import org.apache.log4j.Logger; import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; import com.bigdata.ha.QuorumService; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.Journal; @@ -146,7 +147,7 @@ */ protected boolean isWritable(final HttpServletRequest req, final HttpServletResponse resp) throws IOException { - + final Quorum<HAGlue, QuorumService<HAGlue>> quorum = getQuorum(); if (quorum == null) { @@ -155,17 +156,7 @@ return true; } - - if (quorum.getClient().isLeader(quorum.token())) { - - /* - * There is a quorum. The quorum is met. ... [truncated message content] |
From: <mar...@us...> - 2013-04-18 17:44:16
|
Revision: 7059 http://bigdata.svn.sourceforge.net/bigdata/?rev=7059&view=rev Author: martyncutcher Date: 2013-04-18 17:44:07 +0000 (Thu, 18 Apr 2013) Log Message: ----------- synchronize updates to tests and fix resetforHARootBlocks to release historical cache through commits Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java branches/READ_CACHE/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/quorum/zk/ZKQuorumImpl.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/HALogWriter.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -457,9 +457,15 @@ throw new IllegalStateException("lastCommitTime=" + m_rootBlock.getLastCommitTime() + ", but msg=" + msg); - if (m_nextSequence != msg.getSequence()) + if (m_nextSequence != msg.getSequence()) { + if (true) {// DEBUG ignore!! + haLog.warn("Ignoring sequence error"); + return; + } + throw new IllegalStateException("nextSequence=" + m_nextSequence + ", but msg=" + msg); + } if (haLog.isInfoEnabled()) haLog.info("msg=" + msg + ", position=" + m_position); Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -2801,6 +2801,8 @@ lock.lock(); + final IRootBlockView old = _rootBlock; + try { assertOpen(); @@ -2855,6 +2857,10 @@ if ((_bufferStrategy instanceof IHABufferStrategy) && quorum != null && quorum.isHighlyAvailable()) { + final long newCommitCounter = old.getCommitCounter() + 1; + + final ICommitRecord commitRecord = new CommitRecord(commitTime, + newCommitCounter, rootAddrs); try { /** * CRITICAL SECTION. We need obtain a distributed consensus @@ -2914,8 +2920,6 @@ * but good root blocks can be found elsewhere in the file. */ - final IRootBlockView old = _rootBlock; - final long newCommitCounter = old.getCommitCounter() + 1; final ICommitRecord commitRecord = new CommitRecord(commitTime, @@ -3206,6 +3210,16 @@ return commitTime; + } catch (Throwable t) { + if (_bufferStrategy instanceof IHABufferStrategy) { + log.warn("BufferStrategy reset from root block after commit failure", t); + + ((IHABufferStrategy) _bufferStrategy).resetFromHARootBlock(old); + } else { + log.error("BufferStrategy does not support recovery from commit failure: " + _bufferStrategy); + } + + throw new RuntimeException(t); // wrap and rethrow } finally { lock.unlock(); @@ -3997,7 +4011,7 @@ return true; // no index available } - + } return false; @@ -4164,6 +4178,9 @@ if (ndx != null) { if (isHistoryGone(commitTime)) { + + if (log.isTraceEnabled()) + log.trace("Removing entry from cache: " + name); /* * No longer visible. @@ -4486,7 +4503,7 @@ final long offset = getPhysicalAddress(checkpointAddr); ICommitter ndx = historicalIndexCache.get(offset); - + if (ndx == null) { /* @@ -4497,6 +4514,12 @@ ndx = Checkpoint .loadFromCheckpoint(this, checkpointAddr, true/* readOnly */); + if (log.isTraceEnabled()) + log.trace("Adding checkpoint to historical index at " + checkpointAddr); + + } else { + if (log.isTraceEnabled()) + log.trace("Found historical index at " + checkpointAddr + ", historicalIndexCache.size(): " + historicalIndexCache.size()); } // Note: putIfAbsent is used to make concurrent requests atomic. Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/RWStore.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -53,6 +53,8 @@ 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; import org.apache.log4j.Logger; @@ -101,6 +103,7 @@ import com.bigdata.rawstore.IPSOutputStream; import com.bigdata.rawstore.IRawStore; import com.bigdata.service.AbstractTransactionService; +import com.bigdata.util.ChecksumError; import com.bigdata.util.ChecksumUtility; /** @@ -546,7 +549,9 @@ * a store-wide allocation lock when creating new allocation areas, but * significant contention may be avoided. */ - final private ReentrantLock m_allocationLock = new ReentrantLock(); + final private ReentrantReadWriteLock m_allocationLock = new ReentrantReadWriteLock(); + final private WriteLock m_allocationWriteLock = m_allocationLock.writeLock(); + final private ReadLock m_allocationReadLock = m_allocationLock.readLock(); /** * The deferredFreeList is simply an array of releaseTime,freeListAddrs @@ -897,7 +902,7 @@ if (latchedAddr == 0) return; - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { FixedAllocator alloc = null; try { @@ -937,7 +942,7 @@ } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -954,7 +959,7 @@ if (latchedAddr == 0) return; - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { // assert m_commitList.size() == 0; @@ -975,7 +980,7 @@ // assert m_commitList.size() == 0; } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -1398,24 +1403,16 @@ final FixedAllocator allocator; final ArrayList<? extends Allocator> freeList; assert allocSize > 0; + + final int slotSizeIndex = slotSizeIndex(allocSize); + + if (slotSizeIndex == -1) { + throw new IllegalStateException("Unexpected allocation size of: " + allocSize); + } - // m_minFixedAlloc and m_maxFixedAlloc may not be set since - // as finals they must be set in the constructor. Therefore - // recalculate for local load - final int minFixedAlloc = 64 * m_allocSizes[0]; - final int maxFixedAlloc = 64 * m_allocSizes[m_allocSizes.length-1]; - int index = 0; - int fixedSize = minFixedAlloc; - while (fixedSize < allocSize && fixedSize < maxFixedAlloc) - fixedSize = 64 * m_allocSizes[++index]; - - if (allocSize != fixedSize) { - throw new IllegalStateException("Unexpected allocator size: " - + allocSize + " != " + fixedSize); - } allocator = new FixedAllocator(this, allocSize);//, m_writeCache); - freeList = m_freeFixed[index]; + freeList = m_freeFixed[slotSizeIndex]; allocator.read(strBuf); final int chk = ChecksumUtility.getCHK().checksum(buf, @@ -1450,6 +1447,83 @@ } /** + * Computes the slot size index given the absolute slot size. + * + * If the slotSizes are [1,2,4] this corresponds to absolute sizes by + * multiplying by 64 of [64, 128, 256], so slotSizeIndex(64) would return 0, + * and any parameter other than 64, 128 or 256 would return -1. + * + * @param allocSize - absolute slot size + * @return + */ + private int slotSizeIndex(final int allocSize) { + if (allocSize % 64 != 0) + return -1; + + final int slotSize = allocSize / 64; + int slotSizeIndex = -1; + for (int index = 0; index < m_allocSizes.length; index++) { + if (m_allocSizes[index] == slotSize) { + slotSizeIndex = index; + break; + } + } + + return slotSizeIndex; + } + + /** + * Required for HA to support post commit message to synchronize allocators + * with new state. By this time the new allocator state will have been flushed + * to the disk, so should be 1) On disk, 2) Probably in OS cache and 3) Possibly + * in the WriteCache. + * + * For efficiency we do not want to default to reading from disk. + * + * If there is an existing allocator, then we can compare the old with the new state + * to determine which addresses have been freed and hence which addresses should be + * removed from the external cache. + * + * @param index of Alloctor to be updated + * @param addr on disk to be read + * @throws InterruptedException + * @throws ChecksumError + * @throws IOException + */ + private void updateFixedAllocator(final int index, final long addr) throws ChecksumError, InterruptedException, IOException { + final ByteBuffer buf = m_writeCacheService.read(addr, ALLOC_BLOCK_SIZE); + + final ByteArrayInputStream baBuf = new ByteArrayInputStream(buf.array()); + final DataInputStream strBuf = new DataInputStream(baBuf); + + final int allocSize = strBuf.readInt(); // if Blob < 0 + assert allocSize > 0; + + final int slotIndex = slotSizeIndex(allocSize); + if (slotIndex == -1) + throw new IllegalStateException("Invalid allocation size: " + allocSize); + + final FixedAllocator allocator = new FixedAllocator(this, allocSize); + final ArrayList<? extends Allocator> freeList = m_freeFixed[slotIndex]; + + if (index < m_allocs.size()) { + final FixedAllocator old = m_allocs.get(index); + freeList.remove(old); + + m_allocs.set(index, allocator); + allocator.setFreeList(freeList); + + // Need to iterate over all allocated bits in "old" and see if they + // are clear in "new". If so then clear from externalCache + + } else { + assert index == m_allocs.size(); + m_allocs.add(allocator); + } + + } + + /** * Called from ContextAllocation when no free FixedAllocator is immediately * available. First the free list will be checked to see if one is * available, otherwise it will be created. When the calling @@ -1573,38 +1647,43 @@ * @return */ public ByteBuffer getData(final long rwaddr, final int sze) { - // must allow for checksum - if (sze > (m_maxFixedAlloc-4) || m_writeCacheService == null) { - final byte buf[] = new byte[sze + 4]; // 4 bytes for checksum - - getData(rwaddr, buf, 0, sze+4); - - return ByteBuffer.wrap(buf, 0, sze); - } else { - final long paddr = physicalAddress((int) rwaddr); + m_allocationReadLock.lock(); // protection against resetFromHARootBlock!! + try { + // must allow for checksum + if (sze > (m_maxFixedAlloc-4) || m_writeCacheService == null) { + final byte buf[] = new byte[sze + 4]; // 4 bytes for checksum - if (paddr == 0) { - - assertAllocators(); - - throw new PhysicalAddressResolutionException(rwaddr); - + getData(rwaddr, buf, 0, sze+4); + + return ByteBuffer.wrap(buf, 0, sze); + } else { + final long paddr = physicalAddress((int) rwaddr); + + if (paddr == 0) { + + assertAllocators(); + + throw new PhysicalAddressResolutionException(rwaddr); + + } + + assert paddr > 0; + try { + return m_writeCacheService.read(paddr, sze+4); + } catch (Throwable e) { + /* + * Note: ClosedByInterruptException can be thrown out of + * FileChannelUtility.readAll(), typically because the LIMIT on + * a query was satisfied, but we do not want to log that as an + * error. + */ + // log.error(e,e); + throw new RuntimeException("addr=" + rwaddr + " : cause=" + e, e); + + } } - - assert paddr > 0; - try { - return m_writeCacheService.read(paddr, sze+4); - } catch (Throwable e) { - /* - * Note: ClosedByInterruptException can be thrown out of - * FileChannelUtility.readAll(), typically because the LIMIT on - * a query was satisfied, but we do not want to log that as an - * error. - */ -// log.error(e,e); - throw new RuntimeException("addr=" + rwaddr + " : cause=" + e, e); - - } + } finally { + m_allocationReadLock.unlock(); } } @@ -1921,7 +2000,7 @@ case -2: return; } - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { if (m_lockAddresses != null && m_lockAddresses.containsKey((int)laddr)) throw new IllegalStateException("address locked: " + laddr); @@ -1987,7 +2066,7 @@ } } } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2021,7 +2100,7 @@ */ boolean isSessionProtected() { - if (!m_allocationLock.isHeldByCurrentThread()) { + if (!m_allocationWriteLock.isHeldByCurrentThread()) { /* * In order for changes to m_activeTxCount to be visible the caller * MUST be holding the lock. @@ -2115,7 +2194,7 @@ new ByteArrayInputStream(hdr, 0, hdr.length-4) ); // retain lock for all frees - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final int allocs = instr.readInt(); int rem = sze; @@ -2130,7 +2209,7 @@ } catch (IOException ioe) { throw new RuntimeException(ioe); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2154,7 +2233,7 @@ return; } - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final FixedAllocator alloc = getBlockByAddress(addr); final int addrOffset = getOffset(addr); @@ -2200,7 +2279,7 @@ m_recentAlloc = true; } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2221,7 +2300,7 @@ */ void removeFromExternalCache(final long clr, final int slotSize) { - assert m_allocationLock.isLocked(); + assert m_allocationWriteLock.isHeldByCurrentThread(); if (m_externalCache == null) return; @@ -2278,7 +2357,7 @@ throw new IllegalArgumentException("Allocation size to big: " + size + " > " + m_maxFixedAlloc); } - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { try { final FixedAllocator allocator; @@ -2347,7 +2426,7 @@ throw new RuntimeException(t); } } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2386,7 +2465,7 @@ public long alloc(final byte buf[], final int size, final IAllocationContext context) { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final long begin = System.nanoTime(); @@ -2468,7 +2547,7 @@ return newAddr; } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2548,7 +2627,7 @@ if (log.isInfoEnabled()) { log.info("RWStore Reset"); } - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { assertOpen(); // assertNoRebuild(); @@ -2639,7 +2718,7 @@ } catch (Exception e) { throw new IllegalStateException("Unable to reset the store", e); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -2737,7 +2816,7 @@ checkCoreAllocations(); // take allocation lock to prevent other threads allocating during commit - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { @@ -2815,7 +2894,8 @@ } } - m_commitList.clear(); + // DO NOT clear the commit list until the writes have been flushed + // m_commitList.clear(); writeMetaBits(); @@ -2836,6 +2916,9 @@ m_metaTransientBits = (int[]) m_metaBits.clone(); + // It is now safe to clear the commit list + m_commitList.clear(); + // if (m_commitCallback != null) { // m_commitCallback.commitComplete(); // } @@ -2845,12 +2928,8 @@ } catch (IOException e) { throw new StorageTerminalError("Unable to commit transaction", e); } finally { - try { - // m_committing = false; - m_recentAlloc = false; - } finally { - m_allocationLock.unlock(); - } + m_recentAlloc = false; + m_allocationWriteLock.unlock(); } checkCoreAllocations(); @@ -2874,7 +2953,7 @@ * This may have adverse effects wrt concurrency deadlock issues, but * none have been noticed so far. */ - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { /** @@ -2948,7 +3027,7 @@ } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } @@ -3188,7 +3267,7 @@ void metaFree(final int bit) { - if (!m_allocationLock.isHeldByCurrentThread()) { + if (!m_allocationWriteLock.isHeldByCurrentThread()) { /* * Must hold the allocation lock while allocating or clearing * allocations. @@ -4089,7 +4168,7 @@ * DeferredFrees are written to the deferred PSOutputStream */ public void deferFree(final int rwaddr, final int sze) { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { if (sze > (this.m_maxFixedAlloc-4)) { m_deferredFreeOut.writeInt(-rwaddr); @@ -4101,7 +4180,7 @@ throw new RuntimeException("Could not free: rwaddr=" + rwaddr + ", size=" + sze, e); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -4134,7 +4213,7 @@ // } public long saveDeferrals() { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { if (m_deferredFreeOut.getBytesWritten() == 0) { return 0; @@ -4152,7 +4231,7 @@ } catch (IOException e) { throw new RuntimeException("Cannot write to deferred free", e); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -4171,7 +4250,7 @@ final byte[] buf = new byte[sze+4]; // allow for checksum getData(addr, buf); final DataInputStream strBuf = new DataInputStream(new ByteArrayInputStream(buf)); - m_allocationLock.lock(); + m_allocationWriteLock.lock(); int totalFreed = 0; try { int nxtAddr = strBuf.readInt(); @@ -4203,7 +4282,7 @@ } catch (IOException e) { throw new RuntimeException("Problem freeing deferrals", e); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } return totalFreed; @@ -4318,11 +4397,11 @@ * @param context */ public void registerContext(IAllocationContext context) { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { establishContextAllocation(context); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -4340,7 +4419,7 @@ */ public void detachContext(final IAllocationContext context) { assertOpen(); - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final ContextAllocation alloc = m_contexts.remove(context); @@ -4355,7 +4434,7 @@ releaseSessions(); } } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -4370,7 +4449,7 @@ */ public void abortContext(final IAllocationContext context) { assertOpen(); - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final ContextAllocation alloc = m_contexts.remove(context); @@ -4380,7 +4459,7 @@ } } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -4542,7 +4621,7 @@ * The allocation lock MUST be held to make changes in the membership of * m_contexts atomic with respect to free(). */ - assert m_allocationLock.isHeldByCurrentThread(); + assert m_allocationWriteLock.isHeldByCurrentThread(); ContextAllocation ret = m_contexts.get(context); @@ -5271,10 +5350,10 @@ writeCache.closeForWrites(); /* - * Setup buffer for writing. We receive the buffer with pos=0, \xCA + * Setup buffer for writing. We receive the buffer with pos=0, * limit=#ofbyteswritten. However, flush() expects pos=limit, will * clear pos to zero and then write bytes up to the limit. So, - * we set the position to the limit before calling flush. \xCA \xCA \xCA + * we set the position to the limit before calling flush. */ final ByteBuffer bb = b.buffer(); final int limit = bb.limit(); @@ -5284,7 +5363,7 @@ * Flush the scattered writes in the write cache to the backing * store. */ - m_allocationLock.lock(); // TODO This lock is not necessary (verify!) + m_allocationReadLock.lock(); // TODO This lock is not necessary (verify!) try { // Flush writes. writeCache.flush(false/* force */); @@ -5292,7 +5371,7 @@ // install reads into readCache (if any) m_writeCacheService.installReads(writeCache); } finally { - m_allocationLock.unlock(); + m_allocationReadLock.unlock(); } } @@ -5558,19 +5637,22 @@ } private void activateTx() { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { m_activeTxCount++; if(log.isInfoEnabled()) log.info("#activeTx="+m_activeTxCount); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } private void deactivateTx() { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { + if (log.isInfoEnabled()) + log.info("Deactivating TX " + m_activeTxCount); + if (m_activeTxCount == 0) { throw new IllegalStateException("Tx count must be positive!"); } @@ -5582,7 +5664,7 @@ releaseSessions(); } } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -5630,12 +5712,12 @@ final ConcurrentWeakValueCache<Long, ICommitter> externalCache, final int dataSize) { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { m_externalCache = externalCache; m_cachedDatasize = getSlotSize(dataSize); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -5651,7 +5733,7 @@ * @return <code>true</code> iff the address is currently committed. */ public boolean isCommitted(final int rwaddr) { - m_allocationLock.lock(); + m_allocationWriteLock.lock(); try { final FixedAllocator alloc = getBlockByAddress(rwaddr); @@ -5661,7 +5743,7 @@ return alloc.isCommitted(offset); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } @@ -5694,26 +5776,38 @@ */ public void resetFromHARootBlock(final IRootBlockView rootBlock) { + final Lock writeLock = this.m_extensionLock.writeLock(); + + writeLock.lock(); try { - - // should not be any dirty allocators - // assert m_commitList.size() == 0; - - // Remove all current allocators - m_allocs.clear(); - - assert m_nextAllocation != 0; - - m_nextAllocation = 0; - - initfromRootBlock(rootBlock); - - assert m_nextAllocation != 0; + m_allocationWriteLock.lock(); + try { + // should not be any dirty allocators + // assert m_commitList.size() == 0; + + // Remove all current allocators + m_allocs.clear(); + + assert m_nextAllocation != 0; + + m_nextAllocation = 0; + + initfromRootBlock(rootBlock); + + // KICK external cache into touch - FIXME: handle with improved Allocator synchronization + m_externalCache.clear(); + + assert m_nextAllocation != 0; + } finally { + m_allocationWriteLock.unlock(); + } } catch (IOException e) { throw new RuntimeException(e); + } finally { + writeLock.unlock(); } } @@ -5891,7 +5985,7 @@ getData(addr, buf); final DataInputStream strBuf = new DataInputStream( new ByteArrayInputStream(buf)); - m_allocationLock.lock(); + m_allocationWriteLock.lock(); // int totalFreed = 0; try { int nxtAddr = strBuf.readInt(); @@ -5934,7 +6028,7 @@ } catch (IOException e) { throw new RuntimeException("Problem checking deferrals: " + e, e); } finally { - m_allocationLock.unlock(); + m_allocationWriteLock.unlock(); } } Modified: branches/READ_CACHE/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata/src/test/com/bigdata/rwstore/TestRWJournal.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -186,6 +186,44 @@ return properties; } + + /** + * The RWStore relies on several bit manipulation methods to manage both FixedAllocators + * and meta allocations. + * <p> + * This test stresses these methods. + */ + public void testRWBits() { + final int bSize = 32 << 1; // a smaller array stresses more than a larger - say 8192 + final int[] bits = new int[bSize]; + final int nbits = bSize * 32; + + // Set all the bits one at a time + for (int i = 0; i < nbits; i++) { + final int b = RWStore.fndBit(bits, bSize); + assertTrue(b != -1); + assertFalse(RWStore.tstBit(bits, b)); + RWStore.setBit(bits, b); + assertTrue(RWStore.tstBit(bits, b)); + } + + // check that all are set + assertTrue(-1 == RWStore.fndBit(bits, bSize)); + + // now loop around clearing a random bit, then searching and setting it + for (int i = 0; i < 30 * 1024 * 1024; i++) { + final int b = r.nextInt(nbits); + assertTrue(RWStore.tstBit(bits, b)); + RWStore.clrBit(bits, b); + assertFalse(RWStore.tstBit(bits, b)); + + assertTrue(b == RWStore.fndBit(bits, bSize)); + RWStore.setBit(bits, b); + assertTrue(RWStore.tstBit(bits, b)); + } + + assertTrue(-1 == RWStore.fndBit(bits, bSize)); + } /** * Verify normal operation and basic assumptions when creating a new journal Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/quorum/zk/ZKQuorumImpl.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/quorum/zk/ZKQuorumImpl.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/quorum/zk/ZKQuorumImpl.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -809,7 +809,7 @@ false/* watch */, null/* stat */)); if (serviceId.equals(state.serviceUUID())) { // Found this service. - log.warn("Service already joined"); + log.warn("Service " + serviceId + " already joined in quorum of " + children.length); return; } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -2200,8 +2200,33 @@ return fullyMetBeforeLoadDone; } - + /** + * IMHO a simpler, clearer implementation + * + * @param token + * @param ft + * @return + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + protected boolean awaitFullyMetDuringLOAD2(final long token, + final Future<Void> ft) throws InterruptedException, + ExecutionException, TimeoutException { + + try { + assertTrue(token == awaitFullyMetQuorum((int) (longLoadTimeoutMillis/awaitQuorumTimeout))); + } catch (AsynchronousQuorumCloseException e) { + throw new RuntimeException(e); + } catch (IOException e) { + throw new RuntimeException(e); + } + + return !ft.isDone(); + + } + /** * Remove files in the directory, except the "open" log file. * * @param dir Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-18 15:44:00 UTC (rev 7058) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-18 17:44:07 UTC (rev 7059) @@ -27,12 +27,19 @@ package com.bigdata.journal.jini.ha; import java.io.File; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import junit.framework.AssertionFailedError; + import net.jini.config.Configuration; import com.bigdata.ha.HAGlue; @@ -40,6 +47,7 @@ import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.journal.AbstractJournal; +import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; @@ -1468,7 +1476,117 @@ } } + public void testABCMultiTransactionFollowerReads() throws Exception { + // doABCMultiTransactionFollowerReads(2000/*nTransactions*/, 20/*delay per transaction*/); // STRESS + doABCMultiTransactionFollowerReads(200/*nTransactions*/, 20/*delay per transaction*/); + } + /** + * Tests multiple concurrent reads on followers in presence of multiple updates. + * @throws Exception + */ + protected void doABCMultiTransactionFollowerReads(final int nTransactions, final long transactionDelay) throws Exception { + + final long timeout = TimeUnit.MINUTES.toMillis(4); + try { + + // Start all services. + final ABC services = new ABC(true/* sequential */); + + + // Wait for a quorum meet. + final long token = quorum.awaitQuorum(awaitQuorumTimeout, + TimeUnit.MILLISECONDS); + + awaitFullyMetQuorum(); + + final HAGlue leader = quorum.getClient().getLeader(token); + + // Verify assumption in this test. + assertEquals(leader, services.serverA); + + // Wait until leader is ready. + leader.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + + // start concurrent task to load for specified transactions + final Callable<Void> task = new Callable<Void>() { + public Void call() throws Exception { + for (int n = 0; n < nTransactions; n++) { + + final StringBuilder sb = new StringBuilder(); + sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); + sb.append("INSERT DATA {\n"); + sb.append(" <http://example/book" + n + + "> dc:title \"A new book\" ;\n"); + sb.append(" dc:creator \"A.N.Other\" .\n"); + sb.append("}\n"); + + final String updateStr = sb.toString(); + + final HAGlue leader = quorum.getClient().getLeader( + token); + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + getRemoteRepository(leader).prepareUpdate( + updateStr).evaluate(); + log.warn("COMPLETED TRANSACTION " + n); + + Thread.sleep(transactionDelay); + } + // done. + return null; + } + }; + final FutureTask<Void> load = new FutureTask<Void>(task); + + executorService.submit(load); + + // Now create a Callable for the final followes to repeatedly query against the current commit point + final Callable<Void> query = new Callable<Void>() { + public Void call() throws Exception { + int queryCount = 0; + SimpleDateFormat df = new SimpleDateFormat("hh:mm:ss,SSS"); + while (!load.isDone()) { + + final StringBuilder sb = new StringBuilder(); + sb.append("SELECT (COUNT(*) AS ?count) WHERE { ?s ?p ?o }\n"); + + final String query = sb.toString(); + + // final RemoteRepository follower = getRemoteRepository(services.serverA); // try with Leader to see difference! 6537 queries (less than for follower) + final RemoteRepository follower = getRemoteRepository(services.serverC); // 10109 queries for 2000 transact ons + + // Verify quorum is still valid. + quorum.assertQuorum(token); + + follower.prepareTupleQuery(query).evaluate(); + + // add date time format to support comparison with HA logs + log.info(df.format(new Date()) + " - completed query: " + ++queryCount); + } + // done. + return null; + } + }; + + final FutureTask<Void> queries = new FutureTask<Void>(query); + + executorService.submit(queries); + + // Now wait for query completion! + queries.get(); + + assertTrue(load.isDone()); + + } finally { + destroyAll(); + } + + } + + /** * Tests that halog files are removed after fully met on rebuild * face * @throws Exception @@ -1800,7 +1918,7 @@ // start concurrent task loads that continue until fully met final FutureTask<Void> ft = new FutureTask<Void>(new LargeLoadTask( - token)); + token, true)); executorService.submit(ft); @@ -1814,6 +1932,9 @@ shutdownC(); awaitPipeline(new HAGlue[] {startup.serverA, startup.serverB}); + // And no longer joined. + awaitJoined(new HAGlue[] {startup.serverA, startup.serverB}); + // token must remain unchanged to indicate same quorum assertEquals(token, awaitMetQuorum()); @@ -1823,16 +1944,43 @@ // C comes back at the end of the pipeline. awaitPipeline(new HAGlue[] {startup.serverA, startup.serverB, serverC2}); + // And Joins. + // awaitJoined(new HAGlue[] {startup.serverA, startup.serverB, serverC2}); + // Await fully met quorum *before* LOAD is done. - assertTrue(awaitFullyMetDuringLOAD(token, ft)); + assertTrue(awaitFullyMetDuringLOAD2(token, ft)); // Verify fully met. assertTrue(quorum.isQuorumFullyMet(token)); + /* // Await LOAD, but with a timeout. ft.get(longLoadTimeoutMillis, TimeUnit.MILLISECONDS); + // no delay needed since commit2Phase should ensure stores all synced + try { + assertDigestsEquals(new HAGlue[] { startup.serverA, startup.serverB, serverC2 }); + } catch (final AssertionFailedError afe) { + shutdownA(); + shutdownB(); + shutdownC(); + throw afe; + } + */ } + + /** + * Stress test disabled for CI + */ + public void _test_stress() throws Exception { + for (int i = 0; i < 20; i++) { + try { + testABC_LiveLoadRemainsMet_restart_C_fullyMetDuringLOAD(); + } finally { + destroyAll(); + } + } + } /** * Start A+B+C in strict sequence. Wait until the quorum fully meets. Start @@ -1947,7 +2095,7 @@ // start concurrent task loads that continue until fully met final FutureTask<Void> ft = new FutureTask<Void>(new LargeLoadTask( - token)); + token, true)); executorService.submit(ft); @@ -1979,6 +2127,7 @@ // Await LOAD, but with a timeout. ft.get(longLoadTimeoutMillis, TimeUnit.MILLISECONDS); + // assertDigestsEquals(new HAGlue[] { startup.serverA, serverB2, startup.serverC }); } /** @@ -2025,7 +2174,7 @@ awaitPipeline(new HAGlue[] {startup.serverA, startup.serverB, serverC2}); // wait for the quorum to fully meet during the LOAD. - assertTrue(awaitFullyMetDuringLOAD(token, ft)); + assertTrue(awaitFullyMetDuringLOAD2(token, ft)); // Double checked assertion. Should always be true per loop above. assertTrue(quorum.isQuorumFullyMet(token)); @@ -2051,7 +2200,7 @@ awaitPipeline(new HAGlue[] { startup.serverA, serverC2, serverB2 }); // Await fully met quorum *before* LOAD is done. - assertTrue(awaitFullyMetDuringLOAD(token, ft)); + assertTrue(awaitFullyMetDuringLOAD2(token, ft)); // Verify fully met. assertTrue(quorum.isQuorumFullyMet(token)); @@ -2115,7 +2264,7 @@ awaitPipeline(new HAGlue[] { startup.serverA, startup.serverC, serverB2 }); // Await fully met quorum *before* LOAD is done. - assertTrue(awaitFullyMetDuringLOAD(token, ft)); + assertTrue(awaitFullyMetDuringLOAD2(token, ft)); // Verify fully met. assertTrue(quorum.isQuorumFullyMet(token)); @@ -2143,7 +2292,7 @@ awaitPipeline(new HAGlue[] { startup.serverA, serverB2, serverC2 }); // Await fully met quorum *before* LOAD is done. - assertTrue(awaitFullyMetDuringLOAD(token, ft)); + assertTrue(awaitFullyMetDuringLOAD2(token, ft)); // Verify fully met. assertTrue(quorum.isQuorumFullyMet(token)); This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-20 15:42:54
|
Revision: 7065 http://bigdata.svn.sourceforge.net/bigdata/?rev=7065&view=rev Author: thompsonbry Date: 2013-04-20 15:42:43 +0000 (Sat, 20 Apr 2013) Log Message: ----------- AbstractJournal.setQuorumToken(): Modified the code to set the haStatus field as a side-effect and modified the code to detect and appropriately update the haReadyToken and the new haStatus field on a serviceLeave. AbstractJournal.isReadOnly() was modified to use the new haStatus field to decide whether the journal was writable or not. This method is VERY sensitive. It is necessary that a journal be writable if the quorum is not met. Note: The HAJournalServer does not currently have a serviceLeave() event handler. The HA CI test suite (including the new tests and modifications to the existing tests) is 100% green with this commit. However, if I add the serviceLeave() event handler, then it causes 11 test failures - all of which are related to the tests that force a service leave while the quorum is met but do not force a quorum break (live load remains met and friends). This needs to be looked at more closely. New HA tests were written for the HA TXS: - done. Request /status during concurrent load. I have observed deadlock for this on an HA3 installation when a hardware failure caused a pipeline leave. The deadlock was eventually cured - see https://sourceforge.net/apps/trac/bigdata/ticket/665. The test does NOT replicate the deadlock. Looking at the stack traces, it looked like we were doing too much work to decide whether the store is readOnly. This was happening in a tight loop for each triple that was parsed by LOAD. I've modified the isReadOnly() code path as described in this commit message to make it non-blocking. This should make the /status page requests unable to deadlock as observed in that ticket. - done: HAGlue.getHAStatus() - This is a new HAGlue method and we need to write a test for it. It is the RMI equivalent of the "GET /status?HA" HTTP request. - done. TEST: Verify reads and writes are rejected unless the quorum is met. - done. Verify writes are rejected by followers. - done. Verify read-only tx queries against leader and followers. (We have existing tests for this). @see https://sourceforge.net/apps/trac/bigdata/ticket/623 (HA TXS) Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/AbstractJournal.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -2235,16 +2235,54 @@ final long token = this.quorumToken; + /* + * This code path is too expensive and has been observed to deadlock. + * Turn this into a non-blocking code path through correct maintenance + * of the haReadyToken and the haStatus fields. + */ +// if (token != Quorum.NO_QUORUM) { +// +// // Quorum exists, are we the leader for that token? +// final boolean isLeader = quorum.getClient().isLeader(token); +// +// // read-only unless this is the leader. +// return !isLeader; +// +// } + + /* + * This code path is completely non-blocking. It relies on volatile + * writes on [quorumToken] and [haStatus]. + * + * The rule is read-only if there is a met quorum unless this is the + * leader and read/write if there is no quorum or if the quorum is not + * met. + */ if (token != Quorum.NO_QUORUM) { - // Quorum exists, are we the leader for that token? - final boolean isLeader = quorum.getClient().isLeader(token); + switch (haStatus) { + case Leader: // read/write + return false; + case Follower: // read-only + return true; + case NotReady: + /* + * This case is considered "read-only" locally, but not + * available for reads by the HA layer (REST API, SPARQL, etc). + */ + return true; + default: + throw new AssertionError(); + } - // read-only unless this is the leader. - return !isLeader; - } + /* + * Note: Default for HA permits read/write access when the quorum is not + * met. This allows us to make local changes when setting up the + * service, doing resync, etc. + */ + return false; } @@ -2795,16 +2833,15 @@ * @return The timestamp assigned to the commit record -or- 0L if there were * no data to commit. */ + // Note: Overridden by StoreManager (DataService). protected long commitNow(final long commitTime) { + + final WriteLock lock = _fieldReadWriteLock.writeLock(); - final WriteLock lock = _fieldReadWriteLock.writeLock(); + lock.lock(); - lock.lock(); - - final IRootBlockView old = _rootBlock; - - try { - + try { + assertOpen(); final long beginNanos = System.nanoTime(); @@ -2857,10 +2894,6 @@ if ((_bufferStrategy instanceof IHABufferStrategy) && quorum != null && quorum.isHighlyAvailable()) { - final long newCommitCounter = old.getCommitCounter() + 1; - - final ICommitRecord commitRecord = new CommitRecord(commitTime, - newCommitCounter, rootAddrs); try { /** * CRITICAL SECTION. We need obtain a distributed consensus @@ -2920,6 +2953,8 @@ * but good root blocks can be found elsewhere in the file. */ + final IRootBlockView old = _rootBlock; + final long newCommitCounter = old.getCommitCounter() + 1; final ICommitRecord commitRecord = new CommitRecord(commitTime, @@ -3210,24 +3245,14 @@ return commitTime; - } catch (Throwable t) { - if (_bufferStrategy instanceof IHABufferStrategy) { - log.warn("BufferStrategy reset from root block after commit failure", t); - - ((IHABufferStrategy) _bufferStrategy).resetFromHARootBlock(old); - } else { - log.error("BufferStrategy does not support recovery from commit failure: " + _bufferStrategy); - } - - throw new RuntimeException(t); // wrap and rethrow - } finally { + } finally { - lock.unlock(); + lock.unlock(); - } + } + + } - } - // /** // * (debug only) For the {@link RWStrategy}, scans the // * {@link #historicalIndexCache} and verifies that there are no checkpoint @@ -4179,8 +4204,8 @@ if (isHistoryGone(commitTime)) { - if (log.isTraceEnabled()) - log.trace("Removing entry from cache: " + name); + if (log.isTraceEnabled()) + log.trace("Removing entry from cache: " + name); /* * No longer visible. @@ -4515,11 +4540,16 @@ .loadFromCheckpoint(this, checkpointAddr, true/* readOnly */); if (log.isTraceEnabled()) - log.trace("Adding checkpoint to historical index at " + checkpointAddr); + log.trace("Adding checkpoint to historical index at " + + checkpointAddr); } else { + if (log.isTraceEnabled()) - log.trace("Found historical index at " + checkpointAddr + ", historicalIndexCache.size(): " + historicalIndexCache.size()); + log.trace("Found historical index at " + checkpointAddr + + ", historicalIndexCache.size(): " + + historicalIndexCache.size()); + } // Note: putIfAbsent is used to make concurrent requests atomic. @@ -4968,6 +4998,8 @@ */ final long oldValue = quorumToken; + final long oldReady = haReadyToken; + final HAStatusEnum oldStatus = haStatus; if (haLog.isInfoEnabled()) haLog.info("oldValue=" + oldValue + ", newToken=" + newValue); @@ -4985,6 +5017,7 @@ final boolean didBreak; final boolean didMeet; final boolean didJoinMetQuorum; + final boolean didLeaveMetQuorum; if (newValue == Quorum.NO_QUORUM && oldValue != Quorum.NO_QUORUM) { @@ -4996,9 +5029,10 @@ this.quorumToken = newValue; - didBreak = true; + didBreak = true; // quorum break. didMeet = false; didJoinMetQuorum = false; + didLeaveMetQuorum = haReadyToken != Quorum.NO_QUORUM; // if service was joined with met quorum, then it just left the met quorum. } else if (newValue != Quorum.NO_QUORUM && oldValue == Quorum.NO_QUORUM) { @@ -5009,12 +5043,15 @@ */ didBreak = false; - didMeet = true; + didMeet = true; // quorum meet. didJoinMetQuorum = false; + didLeaveMetQuorum = false; - } else if (newValue != Quorum.NO_QUORUM - && haReadyToken == Quorum.NO_QUORUM && localService != null - && localService.isJoinedMember(newValue)) { + } else if (newValue != Quorum.NO_QUORUM // quorum exists + && haReadyToken == Quorum.NO_QUORUM // service was not joined with met quorum. + && localService != null // + && localService.isJoinedMember(newValue) // service is now joined with met quorum. + ) { /* * This service is joining a quorum that is already met. @@ -5022,13 +5059,31 @@ didBreak = false; didMeet = false; - didJoinMetQuorum = true; + didJoinMetQuorum = true; // service joined with met quorum. + didLeaveMetQuorum = false; + } else if (newValue != Quorum.NO_QUORUM // quorum exists + && haReadyToken != Quorum.NO_QUORUM // service was joined with met quorum + && localService != null // + && !localService.isJoinedMember(newValue) // service is no longer joined with met quorum. + ) { + + /* + * This service is leaving a quorum that is already met (but this is + * not a quorum break since the new token is not NO_QUORUM). + */ + + didBreak = false; + didMeet = false; + didJoinMetQuorum = false; + didLeaveMetQuorum = true; // service left met quorum. quorum stil met. + } else { didBreak = false; didMeet = false; didJoinMetQuorum = false; + didLeaveMetQuorum = false; } @@ -5045,7 +5100,7 @@ try { - if (didBreak) { + if (didBreak || didLeaveMetQuorum) { /* * We also need to discard any active read/write tx since there @@ -5061,8 +5116,14 @@ ((AbstractTransactionService) getLocalTransactionManager() .getTransactionService()).abortAllTx(); - // local abort (no quorum, so 2-phase abort not required). - doLocalAbort(); // FIXME HA : local abort on quorum break -or- service leave? + /* + * Local abort (no quorum, so 2-phase abort not required). + * + * FIXME HA : local abort on quorum break -or- service leave? + * + * FIXME HA : Abort the unisolated connection? + */ + doLocalAbort(); /* * Note: We can not re-cast our vote until our last vote is @@ -5073,8 +5134,12 @@ localCommitCounter = -1; isLeader = isFollower = false; + quorumToken = newValue; // volatile write. + haReadyToken = Quorum.NO_QUORUM; // volatile write. - + + haStatus = HAStatusEnum.NotReady; // volatile write. + haReadyCondition.signalAll(); // signal ALL. } else if (didMeet || didJoinMetQuorum) { @@ -5087,7 +5152,7 @@ localCommitCounter = _rootBlock.getCommitCounter(); - if (localService.isFollower(quorumToken)) { + if (localService.isFollower(newValue)) { isLeader = false; isFollower = true; @@ -5100,8 +5165,7 @@ */ // Remote interface for the quorum leader. - final HAGlue leader = localService - .getLeader(quorumToken); + final HAGlue leader = localService.getLeader(newValue); log.info("Fetching root block from leader."); final IRootBlockView leaderRB; @@ -5139,7 +5203,7 @@ // ready as follower. tmp = newValue; - } else if (localService.isLeader(quorumToken)) { + } else if (localService.isLeader(newValue)) { isLeader = true; isFollower = false; @@ -5157,10 +5221,23 @@ } + /* + * Note: These volatile writes need to occur before we do the + * local abort since the readOnly versus readWrite state of the + * journal is decided based on the [haStatus]. + */ + + this.haReadyToken = tmp; // volatile write. + + // volatile write. + this.haStatus = isLeader ? HAStatusEnum.Leader + : isFollower ? HAStatusEnum.Follower + : HAStatusEnum.NotReady; + if (!installedRBs) { /* - * If we install the RBs, the a local abort was already + * If we install the RBs, then a local abort was already * done. Otherwise we need to do one now (this covers the * case when setQuorumToken() is called on the leader as * well as cases where the service is either not a follower @@ -5172,8 +5249,6 @@ } - this.haReadyToken = tmp; // volatile write. - haReadyCondition.signalAll(); // signal ALL. } else { @@ -5188,6 +5263,18 @@ } + if (haLog.isInfoEnabled()) + haLog.info("qorumToken(" + oldValue + " => " + newValue + + "), haReadyToken(" + oldReady + " => " + haReadyToken // + + "), haStatus(" + oldStatus + " => " + haStatus // + + "), isLeader=" + isLeader// + + ", isFollower=" + isFollower// + + ", didMeet=" + didMeet// + + ", didBreak=" + didBreak// + + ", didJoin=" + didJoinMetQuorum// + + ", didLeave=" + didLeaveMetQuorum// + ); + if (isLeader || isFollower) { localService.didMeet(newValue, localCommitCounter, isLeader); @@ -5197,6 +5284,10 @@ } private final Condition haReadyCondition = _fieldReadWriteLock.writeLock().newCondition(); private volatile long haReadyToken = Quorum.NO_QUORUM; + /** + * Updated with the {@link #haReadyToken}. + */ + private volatile HAStatusEnum haStatus = HAStatusEnum.NotReady; /** * Await the service being ready to partitipate in an HA quorum. The @@ -5674,51 +5765,53 @@ return null; } + + return haStatus; - final QuorumService<HAGlue> quorumService = quorum.getClient(); +// final QuorumService<HAGlue> quorumService = quorum.getClient(); +// +// // check, but do not wait. +// final long haReadyToken = AbstractJournal.this.getHAReady(); +// +// final HAStatusEnum status; +// +// if (haReadyToken == Quorum.NO_QUORUM) { +// +// // Quorum is not met (as percieved by the HAJournal). +// status = HAStatusEnum.NotReady; +// +// } else { +// +// if (quorumService.isLeader(haReadyToken)) { +// +// // Service is leader. +// status = HAStatusEnum.Leader; +// +// } else if (quorumService.isFollower(haReadyToken)) { +// +// // Service is follower. +// status = HAStatusEnum.Follower; +// +// } else { +// +// /* +// * awaitHAReady() should only return successfully (and hence +// * haReadyToken should only be a valid token) if the service was +// * elected as either a leader or a follower. However, it is +// * possible for the service to have concurrently left the met +// * quorum, in which case the if/then/else pattern will fall +// * through to this code path. +// */ +// +// // Quorum is not met (as percieved by the HAJournal). +// status = HAStatusEnum.NotReady; +// +// } +// +// } +// +// return status; - // check, but do not wait. - final long haReadyToken = AbstractJournal.this.getHAReady(); - - final HAStatusEnum status; - - if (haReadyToken == Quorum.NO_QUORUM) { - - // Quorum is not met (as percieved by the HAJournal). - status = HAStatusEnum.NotReady; - - } else { - - if (quorumService.isLeader(haReadyToken)) { - - // Service is leader. - status = HAStatusEnum.Leader; - - } else if (quorumService.isFollower(haReadyToken)) { - - // Service is follower. - status = HAStatusEnum.Follower; - - } else { - - /* - * awaitHAReady() should only return successfully (and hence - * haReadyToken should only be a valid token) if the service was - * elected as either a leader or a follower. However, it is - * possible for the service to have concurrently left the met - * quorum, in which case the if/then/else pattern will fall - * through to this code path. - */ - - // Quorum is not met (as percieved by the HAJournal). - status = HAStatusEnum.NotReady; - - } - - } - - return status; - } @Override Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -1139,10 +1139,8 @@ private class QuorumMeetTask implements Callable<Void> { private final long token; -// private final UUID leaderId; public QuorumMeetTask(final long token, final UUID leaderId) { this.token = token; -// this.leaderId = leaderId; } public Void call() throws Exception { journal.setQuorumToken(token); @@ -1225,9 +1223,10 @@ * Set token. Journal will notice that it is no longer * "HA Ready" * - * FIXME AbstractJournal.setQuorumToken() must detect case where - * it transitions from a met quorum through a service leave and - * clears its haReady token. + * Note: AbstractJournal.setQuorumToken() will detect + * case where it transitions from a met quorum through + * a service leave and will clear its haReady token and + * update its haStatus field appropriately. */ journal.setQuorumToken(getQuorum().token()); try { @@ -1235,7 +1234,7 @@ } catch (IOException e) { haLog.error(e, e); } - enterRunState(new SeekConsensusTask()); + enterRunState(new SeekConsensusTask()); // TODO Versus ERROR state? return null; } } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3JournalServerTestCase.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -87,6 +87,7 @@ import com.bigdata.quorum.QuorumClient; import com.bigdata.quorum.QuorumException; import com.bigdata.quorum.zk.ZKQuorumImpl; +import com.bigdata.rdf.sail.webapp.client.HttpException; import com.bigdata.service.jini.JiniClientConfig; import com.bigdata.service.jini.RemoteDestroyAdmin; import com.bigdata.util.InnerCause; @@ -2086,8 +2087,56 @@ getRemoteRepository(leader).prepareUpdate(updateStr).evaluate(); } + + /** + * Verify that an attempt to read on the specified service is disallowed. + */ + protected void assertWriteRejected(final HAGlue haGlue) throws IOException, + Exception { + final StringBuilder sb = new StringBuilder(); + sb.append("DROP ALL;\n"); + sb.append("PREFIX dc: <http://purl.org/dc/elements/1.1/>\n"); + sb.append("INSERT DATA {\n"); + sb.append(" <http://example/book1> dc:title \"A new book\" ;\n"); + sb.append(" dc:creator \"A.N.Other\" .\n"); + sb.append("}\n"); + + final String updateStr = sb.toString(); + + try { + + getRemoteRepository(haGlue).prepareUpdate(updateStr).evaluate(); + + } catch (HttpException ex) { + + assertEquals("statusCode", 405, ex.getStatusCode()); + + } + + } + /** + * Verify that an attempt to read on the specified service is disallowed. + */ + protected void assertReadRejected(final HAGlue haGlue) throws IOException, + Exception { + + final String queryStr = "SELECT (COUNT(*) as ?count) {?s ?p ?o}"; + + try { + + getRemoteRepository(haGlue).prepareTupleQuery(queryStr).evaluate(); + + } catch (HttpException ex) { + + assertEquals("statusCode", 405, ex.getStatusCode()); + + } + + } + + /** * Task loads a large data set. */ protected class LargeLoadTask implements Callable<Void> { Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHAJournalServerTestCase.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -249,6 +249,24 @@ } /** + * Check the HAStatusEnum for the service. + */ + protected void awaitHAStatus(final HAGlue haGlue, + final HAStatusEnum expected) throws IOException { + + assertCondition(new Runnable() { + public void run() { + try { + assertEquals(expected, haGlue.getHAStatus()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }, 5, TimeUnit.SECONDS); + + } + + /** * Check the NSS API by issuing a "GET .../status" request. * <p> * Note: The NSS server will not allow reads or writes until the Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA2JournalServer.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -118,6 +118,10 @@ // Verify that service self-reports role via the REST API. assertEquals(HAStatusEnum.Follower, getNSSHAStatus(serverB)); + // Check RMI API. + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); + // Await initial commit point (KB create). awaitCommitCounter(1L, serverA, serverB); @@ -158,6 +162,9 @@ */ assertHALogDigestsEquals(1L/* firstCommitCounter */, lastCommitCounter1, new HAGlue[] { serverA, serverB }); + + // Verify can not write on follower. + assertWriteRejected(serverB); } @@ -183,6 +190,10 @@ // Await [B] up and running as follower. assertEquals(HAStatusEnum.Follower, awaitNSSAndHAReady(serverB)); + + // Verify self-reporting by RMI in their respective roles. + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); // Verify binary equality on the journal files. assertDigestsEquals(new HAGlue[] { serverA, serverB }); @@ -232,6 +243,9 @@ // The leader MAY have changed (since the quorum broke). final HAGlue leader2 = quorum.getClient().getLeader(token2); + // Verify leader self-reports in new role. + awaitHAStatus(leader2, HAStatusEnum.Leader); + // final UUID leaderId2 = leader2.getServiceId(); // // assertFalse(leaderId1.equals(leaderId2)); @@ -282,6 +296,10 @@ // Await [B] up and running as follower. assertEquals(HAStatusEnum.Follower, awaitNSSAndHAReady(serverB)); + // Verify self-reports in role. + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); + // Verify binary equality on the journal files. assertDigestsEquals(new HAGlue[] { serverA, serverB }); @@ -321,6 +339,9 @@ // // assertFalse(leaderId1.equals(leaderId2)); + // Verify leader self-reports in new role. + awaitHAStatus(leader2, HAStatusEnum.Leader); + /* * Verify we can read on the KB on both nodes. * Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3JournalServer.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -27,19 +27,18 @@ package com.bigdata.journal.jini.ha; import java.io.File; -import java.io.IOException; import java.text.SimpleDateFormat; import java.util.Date; import java.util.UUID; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; +import java.util.concurrent.Executors; import java.util.concurrent.FutureTask; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; -import junit.framework.AssertionFailedError; - import net.jini.config.Configuration; import com.bigdata.ha.HAGlue; @@ -47,7 +46,6 @@ import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.msg.HARootBlockRequest; import com.bigdata.journal.AbstractJournal; -import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; import com.bigdata.rdf.sail.webapp.client.RemoteRepository; @@ -131,6 +129,9 @@ * commit. Therefore, the HALog files should exist on all nodes. */ + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); + // Current commit point. final long lastCommitCounter = 1L; @@ -147,12 +148,19 @@ // Verify binary equality of (A,B) journals. assertDigestsEquals(new HAGlue[] { serverA, serverB }); + // Verify can not write on follower. + assertWriteRejected(serverB); + // Start 3rd service. final HAGlue serverC = startC(); // Wait until the quorum is fully met. The token should not change. assertEquals(token, awaitFullyMetQuorum()); + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); + awaitHAStatus(serverC, HAStatusEnum.Follower); + // The commit counter has not changed. assertEquals( lastCommitCounter, @@ -188,6 +196,10 @@ assertHALogNotFound(0L/* firstCommitCounter */, lastCommitCounter, new HAGlue[] { serverA, serverB, serverC }); + // Verify can not write on followers. + assertWriteRejected(serverB); + assertWriteRejected(serverC); + } /** @@ -443,6 +455,9 @@ // Wait until the quorum is fully met. After RESYNC assertEquals(token, awaitFullyMetQuorum()); + // Await C as Follower. + awaitHAStatus(serverC, HAStatusEnum.Follower); + // HALog files now exist on ALL services, on original commitCounter! assertHALogDigestsEquals(1L/* firstCommitCounter */, lastCommitCounter2, new HAGlue[] { serverA, serverB, serverC }); @@ -2302,17 +2317,185 @@ } -// public void testStress_LiveLoadRemainsMet() throws Exception { -// for (int i = 1; i <= 20; i++) { -// try { -//// testABC_LiveLoadRemainsMet_restart_B_fullyMetDuringLOAD_restartC_fullyMetDuringLOAD(); -// testABC_LiveLoadRemainsMet_restart_C_fullyMetDuringLOAD(); -// } catch (Throwable e) { -// fail("Run " + i, e); -// } finally { -// destroyAll(); -// } -// } -// } + /** + * Variant of {@link #testABC_LargeLoad()} that issues concurrent HA Status + * requests. This is a regression test for a deadlock observed when a status + * request was issued during a BSBM 100M load on an HA3 cluster. + */ + public void testABC_LargeLoad_concurrentStatusRequests() throws Exception { + + final HAGlue serverA = startA(); + final HAGlue serverB = startB(); + + // wait for a quorum met. + final long token = awaitMetQuorum(); + // Current commit point. + final long lastCommitCounter = 1L; + + // Verify 1st commit point is visible on A + B. + awaitCommitCounter(lastCommitCounter, new HAGlue[] { serverA, serverB }); + + // start C. + final HAGlue serverC = startC(); + + // wait for a fully met quorum. + assertEquals(token, awaitFullyMetQuorum()); + + /* + * LOAD data on leader. + */ + final FutureTask<Void> ft = new FutureTask<Void>(new LargeLoadTask( + token, true/* reallyLargeLoad */)); + + final AtomicLong nsuccess = new AtomicLong(); + + // All concurrent requests. + final ScheduledExecutorService scheduledExecutor = Executors + .newScheduledThreadPool(3/* corePoolSize */); + + ScheduledFuture<?> scheduledFuture = null; + + try { + + // Start LOAD. + executorService.submit(ft); + + // Start STATUS requests. Fixed Rate, so can be concurrent. + scheduledFuture = scheduledExecutor.scheduleAtFixedRate(new Runnable() { + public void run() { + try { + doNSSStatusRequest(serverA); + nsuccess.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + }, 50L/* initialDelay */, 20L/* period */, TimeUnit.MILLISECONDS); + + // Await LOAD, but with a timeout. + ft.get(longLoadTimeoutMillis, TimeUnit.MILLISECONDS); + + if (scheduledFuture.isDone()) { + /* + * Note: We are not expecting this to complete during the load. + */ + scheduledFuture.get(); + fail("Not expecting scheduledFuture to be done: nsuccess=" + + nsuccess); + } + + } finally { + + ft.cancel(true/* mayInterruptIfRunning */); + + if (scheduledFuture != null) + scheduledFuture.cancel(true/* mayInterruptIfRunning */); + + scheduledExecutor.shutdownNow(); + + } + + } + + /** + * Verify that the {@link HAStatusEnum} is updated coorectly as services are + * fail over, when their roles change, and when the quorum meets and breaks. + */ + public void testQuorumABC_HAStatusUpdatesWithFailovers() throws Exception { + + // Start 2 services. + HAGlue serverA = startA(); + HAGlue serverB = startB(); + + // Wait for a quorum meet. + final long token = quorum.awaitQuorum(awaitQuorumTimeout, + TimeUnit.MILLISECONDS); + + awaitHAStatus(serverA, HAStatusEnum.Leader); + awaitHAStatus(serverB, HAStatusEnum.Follower); + + // Start 3rd service. + HAGlue serverC = startC(); + + // Wait until the quorum is fully met. The token should not change. + assertEquals(token, awaitFullyMetQuorum()); + + awaitHAStatus(serverB, HAStatusEnum.Follower); + + // Simple transaction. + simpleTransaction(); + + /* + * Shutdown A. This causes a quorum break. B will become the new leader. + */ + shutdownA(); + + final long token2 = quorum.awaitQuorum(awaitQuorumTimeout * 2, + TimeUnit.MILLISECONDS); + + // New token. + assertEquals(token2, token + 1); + + /* + * Figure out the new leader. + */ + final HAGlue leader = quorum.getClient().getLeader(token2); + final HAGlue follower1; + if (leader == serverB) { + follower1 = serverC; + } else { + follower1 = serverB; + } + + // Self-report in the correct roles. + awaitHAStatus(leader, HAStatusEnum.Leader); + awaitHAStatus(follower1, HAStatusEnum.Follower); + + // Restart A. + serverA = startA(); + + // Self-report in the correct roles. + awaitHAStatus(leader, HAStatusEnum.Leader); + awaitHAStatus(follower1, HAStatusEnum.Follower); + awaitHAStatus(serverA, HAStatusEnum.Follower); + + /* + * Shutdown follower1 (either B or C). Quorum remains met. + */ + if (follower1 == serverC) + shutdownC(); + else if (follower1 == serverB) + shutdownB(); + else + throw new AssertionError(); + + // Correct roles. + awaitHAStatus(leader, HAStatusEnum.Leader); + awaitHAStatus(serverA, HAStatusEnum.Follower); + + /* + * Shutdown A. Quorum breaks. + */ + + shutdownA(); + + // Correct roles. + awaitHAStatus(leader, HAStatusEnum.NotReady); + + } + +// public void testStress_LiveLoadRemainsMet() throws Exception { +// for (int i = 1; i <= 20; i++) { +// try { +//// testABC_LiveLoadRemainsMet_restart_B_fullyMetDuringLOAD_restartC_fullyMetDuringLOAD(); +// testABC_LiveLoadRemainsMet_restart_C_fullyMetDuringLOAD(); +// } catch (Throwable e) { +// fail("Run " + i, e); +// } finally { +// destroyAll(); +// } +// } +//} + } Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-04-20 15:30:18 UTC (rev 7064) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServer.java 2013-04-20 15:42:43 UTC (rev 7065) @@ -99,9 +99,18 @@ // Verify can access the REST API "status" page. doNSSStatusRequest(serverA); + // Verify self-reports as NotReady. + awaitHAStatus(serverA, HAStatusEnum.NotReady); + // Verify that service self-reports as NotReady via the REST API. assertEquals(HAStatusEnum.NotReady, getNSSHAStatus(serverA)); - + + // Verify can not read on service. + assertReadRejected(serverA); + + // Verify can not write on service. + assertWriteRejected(serverA); + assertTrue(getHAJournalFileA().exists()); assertTrue(getHALogDirA().exists()); assertTrue(getSnapshotDirA().exists()); @@ -134,6 +143,15 @@ doNSSStatusRequest(serverB); + // Verify self-reports as NotReady. + awaitHAStatus(serverB, HAStatusEnum.NotReady); + + // Verify can not read on service. + assertReadRejected(serverB); + + // Verify can not write on service. + assertWriteRejected(serverB); + assertTrue(getHAJournalFileB().exists()); assertTrue(getHALogDirB().exists()); assertTrue(getSnapshotDirB().exists()); @@ -162,6 +180,15 @@ doNSSStatusRequest(serverC); + // Verify self-reports as NotReady. + awaitHAStatus(serverC, HAStatusEnum.NotReady); + + // Verify can not read on service. + assertReadRejected(serverC); + + // Verify can not write on service. + assertWriteRejected(serverC); + assertTrue(getHAJournalFileC().exists()); assertTrue(getHALogDirC().exists()); assertTrue(getSnapshotDirC().exists()); @@ -270,6 +297,9 @@ doNSSStatusRequest(serverA); + // Verify self-reports as NotReady. + awaitHAStatus(serverA, HAStatusEnum.NotReady); + digestA = serverA.computeDigest( new HADigestRequest(null/* storeId */)).getDigest(); @@ -302,6 +332,9 @@ // Verify NSS is running. doNSSStatusRequest(serverA); + // Verify self-reports as NotReady. + awaitHAStatus(serverA, HAStatusEnum.NotReady); + /* * Verify no changes in digest on restart? */ This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |
From: <tho...@us...> - 2013-04-25 16:23:19
|
Revision: 7078 http://bigdata.svn.sourceforge.net/bigdata/?rev=7078&view=rev Author: thompsonbry Date: 2013-04-25 16:23:07 +0000 (Thu, 25 Apr 2013) Log Message: ----------- @r7077. {{{ merge https://bigdata.svn.sourceforge.net/svnroot/bigdata/branches/BIGDATA_RELEASE_1_2_0 /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE --- Merging r7048 through r7077 into /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata/src/java/com/bigdata/search/CountIndexTask.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata/src/java/com/bigdata/search/AbstractIndexTask.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata/src/java/com/bigdata/search/Hit.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata/src/java/com/bigdata/search/ReadIndexTask.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/CreateKBTask.java --- Merging r7048 through r7077 into /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/NanoSparqlServer.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFServletContextListener.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ConfigParams.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/QueryServlet.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/RESTServlet.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/UpdateServlet.java C /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java U /Users/bryan/Documents/workspace/BIGDATA_READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFContext.java Merge complete. ===== File Statistics: ===== Updated: 13 ==== Conflict Statistics: ===== File conflicts: 1 }}} The StatusServlet now has some conflicts. For future reference, the following have been added or modified in the READ_CACHE branch to support HA: {{{ /** * Request a snapshot of the journal (HA only). The snapshot will be written * into the configured directory on the server. If a snapshot is already * being taken then this is a NOP. */ static final String SNAPSHOT = "snapshot"; /** * Request to generate the digest for the journals, HALog files, and * snapshot files. This is only a debugging tool. In particular, the digests * on the journal are only valid if there are no concurrent writes on the * journal and the journal has been through either a commit or an abort * protocol. */ static final String DIGESTS = "digests"; /** * Special HA status request designed for clients that poll to determine the * status of an HAJournalServer. This option is exclusive of other * parameters. */ static final String HA = "HA"; }}} {{{ protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) throws IOException { if (req.getParameter(HA) != null && getIndexManager() instanceof AbstractJournal && ((AbstractJournal) getIndexManager()).isHighlyAvailable()) { new HAStatusServletUtil(getIndexManager()).doHAStatus(req, resp); return; } }}} {{{ if (getIndexManager() instanceof AbstractJournal && ((AbstractJournal) getIndexManager()) .isHighlyAvailable()) { new HAStatusServletUtil(getIndexManager()). doGet(req, resp, current); } }}} The rest of the changes have originated in the development branch. There is now a substantial delta for this class in the development branch to provide support for listing and canceling SPARQL UPDATE requests. Revision Links: -------------- http://bigdata.svn.sourceforge.net/bigdata/?rev=7077&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7048&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7077&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7048&view=rev http://bigdata.svn.sourceforge.net/bigdata/?rev=7077&view=rev Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/AbstractIndexTask.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/CountIndexTask.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hit.java branches/READ_CACHE/bigdata/src/java/com/bigdata/search/ReadIndexTask.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/CreateKBTask.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFContext.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/BigdataRDFServletContextListener.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ConfigParams.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/NanoSparqlServer.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/QueryServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/RESTServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/StatusServlet.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/UpdateServlet.java Property Changed: ---------------- branches/READ_CACHE/ branches/READ_CACHE/bigdata/lib/jetty/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba/ branches/READ_CACHE/bigdata/src/java/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/joinGraph/ branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/util/ branches/READ_CACHE/bigdata/src/test/com/bigdata/jsr166/ branches/READ_CACHE/bigdata/src/test/com/bigdata/util/httpd/ branches/READ_CACHE/bigdata-compatibility/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/attr/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/disco/ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/util/config/ branches/READ_CACHE/bigdata-perf/ branches/READ_CACHE/bigdata-perf/btc/ branches/READ_CACHE/bigdata-perf/btc/src/resources/ branches/READ_CACHE/bigdata-perf/lubm/ branches/READ_CACHE/bigdata-perf/uniprot/ branches/READ_CACHE/bigdata-perf/uniprot/src/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/changesets/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/error/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/rdf/util/ branches/READ_CACHE/bigdata-rdf/src/samples/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/bop/rdf/aggregate/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/internal/ branches/READ_CACHE/bigdata-rdf/src/test/com/bigdata/rdf/relation/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/changesets/ branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/bench/ branches/READ_CACHE/bigdata-sails/src/test/com/bigdata/rdf/sail/webapp/ branches/READ_CACHE/dsi-utils/ branches/READ_CACHE/dsi-utils/LEGAL/ branches/READ_CACHE/dsi-utils/lib/ branches/READ_CACHE/dsi-utils/src/ branches/READ_CACHE/dsi-utils/src/java/ branches/READ_CACHE/dsi-utils/src/java/it/ branches/READ_CACHE/dsi-utils/src/java/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/ branches/READ_CACHE/dsi-utils/src/test/it/unimi/dsi/ branches/READ_CACHE/lgpl-utils/src/java/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/lgpl-utils/src/test/it/unimi/dsi/fastutil/bytes/custom/ branches/READ_CACHE/osgi/ branches/READ_CACHE/src/resources/bin/config/ Property changes on: branches/READ_CACHE ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-7048 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE:6769-6785 /branches/BIGDATA_RELEASE_1_2_0:6766-7077 /branches/BTREE_BUFFER_BRANCH:2004-2045 /branches/DEV_BRANCH_27_OCT_2009:2270-2546,2548-2782 /branches/INT64_BRANCH:4486-4522 /branches/JOURNAL_HA_BRANCH:2596-4066 /branches/LARGE_LITERALS_REFACTOR:4175-4387 /branches/LEXICON_REFACTOR_BRANCH:2633-3304 /branches/QUADS_QUERY_BRANCH:4525-4531,4550-4584,4586-4609,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH:4814-4836 /branches/bugfix-btm:2594-3237 /branches/dev-btm:2574-2730 /branches/fko:3150-3194 /trunk:3392-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata/lib/jetty ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-7048 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/lib/jetty:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/lib/jetty:6766-7077 /branches/INT64_BRANCH/bigdata/lib/jetty:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/lib/jetty:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/lib/jetty:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/lib/jetty:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/aggregate ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/aggregate:6766-7077 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/aggregate:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/aggregate:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/joinGraph ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/joinGraph:6766-7077 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/joinGraph:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/bop/util ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/bop/util:6766-7077 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/bop/util:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/htree/raba ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/htree/raba:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/htree/raba:6766-7077 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/htree/raba:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/htree/raba:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/java/com/bigdata/jsr166 ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/jsr166:6766-7048 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/jsr166:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/java/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/java/com/bigdata/jsr166:6766-7077 /branches/INT64_BRANCH/bigdata/src/java/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/java/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/java/com/bigdata/jsr166:4814-4836 Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/AbstractIndexTask.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/AbstractIndexTask.java 2013-04-25 15:59:02 UTC (rev 7077) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/AbstractIndexTask.java 2013-04-25 16:23:07 UTC (rev 7078) @@ -17,6 +17,8 @@ final private static Logger log = Logger.getLogger(AbstractIndexTask.class); protected final String queryTerm; + protected final int queryTermNdx; + protected final int numQueryTerms; protected final double queryTermWeight; protected final byte[] fromKey; protected final byte[] toKey; @@ -27,6 +29,10 @@ * * @param termText * The term text for the search term. + * @param termNdx + * The index of this term within the overall search. + * @param numTerms + * The overall number of search terms. * @param prefixMatch * When <code>true</code> any term having <i>termText</i> as a * prefix will be matched. Otherwise the term must be an exact @@ -36,8 +42,10 @@ * @param searchEngine * The search engine. */ - public AbstractIndexTask(final String termText, final boolean prefixMatch, - final double queryTermWeight, final FullTextIndex<V> searchEngine) { + public AbstractIndexTask(final String termText, + final int termNdx, final int numTerms, + final boolean prefixMatch, final double queryTermWeight, + final FullTextIndex<V> searchEngine) { if (termText == null) throw new IllegalArgumentException(); @@ -46,40 +54,82 @@ throw new IllegalArgumentException(); this.queryTerm = termText; + + this.queryTermNdx = termNdx; + + this.numQueryTerms = numTerms; this.queryTermWeight = queryTermWeight; final IKeyBuilder keyBuilder = searchEngine.getIndex() .getIndexMetadata().getKeyBuilder(); - {// = recordBuilder.getFromKey(keyBuilder, termText); - keyBuilder.reset(); - keyBuilder - .appendText(termText, true/* unicode */, false/* successor */); - fromKey = keyBuilder.getKey(); +// {// = recordBuilder.getFromKey(keyBuilder, termText); +// keyBuilder.reset(); +// keyBuilder +// .appendText(termText, true/* unicode */, false/* successor */); +// fromKey = keyBuilder.getKey(); +// } +// +// if (prefixMatch) { +// /* +// * Accepts anything starting with the search term. E.g., given +// * "bro", it will match "broom" and "brown" but not "break". +// * +// * Note: This uses the successor of the Unicode sort key, so it will +// * scan all keys starting with that prefix until the sucessor of +// * that prefix. +// */ +// keyBuilder.reset(); +// keyBuilder.appendText(termText, true/* unicode */, true/*successor*/); +// toKey = keyBuilder.getKey(); +// } else { +// /* +// * Accepts only those entries that exactly match the search term. +// * +// * Note: This uses the fixed length successor of the fromKey. That +// * gives us a key-range scan which only access keys having the same +// * Unicode sort key. +// */ +// toKey = SuccessorUtil.successor(fromKey.clone()); +// } + + /* + * Changed this to lop off the last three bytes (the pad plus + * run-length) for prefix search. Adding this three byte suffix to the + * prefix was causing problems for searches whose prefix ended with a + * numeric less than 7, because this codes to a byte less than the pad + * byte. + * + * TODO We eventually need to change the pad byte to code to zero, but + * this will break binary compatibility. + */ + + { // from key + keyBuilder.reset(); + keyBuilder + .appendText(termText, true/* unicode */, false/* successor */); + final byte[] tmp = keyBuilder.getKey(); + + if (prefixMatch) { + fromKey = new byte[tmp.length-3]; + System.arraycopy(tmp, 0, fromKey, 0, fromKey.length); + } + else + {// = recordBuilder.getFromKey(keyBuilder, termText); + fromKey = tmp; + } } - if (prefixMatch) { - /* - * Accepts anything starting with the search term. E.g., given - * "bro", it will match "broom" and "brown" but not "break". - * - * Note: This uses the successor of the Unicode sort key, so it will - * scan all keys starting with that prefix until the sucessor of - * that prefix. - */ - keyBuilder.reset(); - keyBuilder.appendText(termText, true/* unicode */, true/*successor*/); - toKey = keyBuilder.getKey(); - } else { - /* - * Accepts only those entries that exactly match the search term. - * - * Note: This uses the fixed length successor of the fromKey. That - * gives us a key-range scan which only access keys having the same - * Unicode sort key. - */ - toKey = SuccessorUtil.successor(fromKey.clone()); + { // to key + /* + * Accepts only those entries that exactly match the search term. + * + * Note: This uses the fixed length successor of the fromKey. That + * gives us a key-range scan which only access keys having the same + * Unicode sort key. + */ + toKey = SuccessorUtil.successor(fromKey.clone()); } } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/CountIndexTask.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/CountIndexTask.java 2013-04-25 15:59:02 UTC (rev 7077) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/CountIndexTask.java 2013-04-25 16:23:07 UTC (rev 7078) @@ -41,6 +41,10 @@ * * @param termText * The term text for the search term. + * @param termNdx + * The index of this term within the overall search. + * @param numTerms + * The overall number of search terms. * @param prefixMatch * When <code>true</code> any term having <i>termText</i> as a * prefix will be matched. Otherwise the term must be an exact @@ -50,10 +54,11 @@ * @param searchEngine * The search engine. */ - public CountIndexTask(final String termText, final boolean prefixMatch, + public CountIndexTask(final String termText, final int termNdx, final int numTerms, + final boolean prefixMatch, final double queryTermWeight, final FullTextIndex<V> searchEngine) { - super(termText, prefixMatch, queryTermWeight, searchEngine); + super(termText, termNdx, numTerms, prefixMatch, queryTermWeight, searchEngine); if (log.isDebugEnabled()) log.debug("termText=[" + termText + "], prefixMatch=" + prefixMatch Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java 2013-04-25 15:59:02 UTC (rev 7077) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/FullTextIndex.java 2013-04-25 16:23:07 UTC (rev 7078) @@ -1008,6 +1008,7 @@ + ", minRank=" + minRank + ", maxRank=" + maxRank + ", matchAllTerms=" + matchAllTerms + + ", prefixMatch=" + prefixMatch + ", timeout=" + timeout + ", unit=" + unit); if (timeout == 0L) { @@ -1085,7 +1086,7 @@ final ITermMetadata md = e.getValue(); - final CountIndexTask<V> task1 = new CountIndexTask<V>(termText, prefixMatch, md + final CountIndexTask<V> task1 = new CountIndexTask<V>(termText, 0, 1, prefixMatch, md .getLocalTermWeight(), this); hits = new SingleTokenHitCollector<V>(task1); @@ -1095,13 +1096,14 @@ final List<CountIndexTask<V>> tasks = new ArrayList<CountIndexTask<V>>( qdata.distinctTermCount()); + int i = 0; for (Map.Entry<String, ITermMetadata> e : qdata.terms.entrySet()) { final String termText = e.getKey(); final ITermMetadata md = e.getValue(); - tasks.add(new CountIndexTask<V>(termText, prefixMatch, md + tasks.add(new CountIndexTask<V>(termText, i++, qdata.terms.size(), prefixMatch, md .getLocalTermWeight(), this)); } @@ -1116,14 +1118,15 @@ final List<Callable<Object>> tasks = new ArrayList<Callable<Object>>( qdata.distinctTermCount()); + int i = 0; for (Map.Entry<String, ITermMetadata> e : qdata.terms.entrySet()) { final String termText = e.getKey(); final ITermMetadata md = e.getValue(); - tasks.add(new ReadIndexTask<V>(termText, prefixMatch, md - .getLocalTermWeight(), this, hits)); + tasks.add(new ReadIndexTask<V>(termText, i++, qdata.terms.size(), + prefixMatch, md.getLocalTermWeight(), this, hits)); } @@ -1160,7 +1163,7 @@ if (a.length == 0) { - log.warn("No hits: languageCode=[" + languageCode + "], query=[" + log.info("No hits: languageCode=[" + languageCode + "], query=[" + query + "]"); cache.put(cacheKey, a); @@ -1178,8 +1181,10 @@ final int nterms = qdata.terms.size(); - if (log.isInfoEnabled()) + if (log.isInfoEnabled()) { log.info("matchAll=true, nterms=" + nterms); + log.info("size before: " + a.length); + } final Hit<V>[] tmp = new Hit[a.length]; @@ -1192,6 +1197,10 @@ } + if (log.isDebugEnabled()) { + log.debug(i); + } + if (i < a.length) { a = new Hit[i]; Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hit.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hit.java 2013-04-25 15:59:02 UTC (rev 7077) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/Hit.java 2013-04-25 16:23:07 UTC (rev 7078) @@ -1,9 +1,13 @@ package com.bigdata.search; +import it.unimi.dsi.bits.LongArrayBitVector; + import java.util.concurrent.ConcurrentHashMap; import org.apache.log4j.Logger; +import test.it.unimi.dsi.bits.LongArrayBitVectorTest; + /** * Metadata about a search result. * @@ -18,8 +22,12 @@ /** note: defaults to an illegal value. */ private V docId = null; - /** #of terms reporting. */ - private int nterms; +// /** #of terms reporting. */ +// private int nterms; + + /** Array of whether each search term appears or does not appear in the hit. **/ +// private LongArrayBitVector searchTerms; + private final boolean[] searchTerms; /** Net cosine for the reporting terms. */ private double cosine; @@ -35,8 +43,11 @@ * * @see ReadIndexTask2 */ - Hit() { + Hit(final int numSearchTerms) { +// this.searchTerms = LongArrayBitVector.ofLength(numSearchTerms); + this.searchTerms = new boolean[numSearchTerms]; + } synchronized void setDocId(final V docId) { @@ -54,11 +65,26 @@ } +// synchronized void setNumSearchTerms(final int numSearchTerms) { +// +//// this.searchTerms = LongArrayBitVector.ofLength(numSearchTerms); +// this.searchTerms = new boolean[numSearchTerms]; +// +// } + /** * The #of terms for which a hit was reported for this document. */ synchronized public int getTermCount() { +// if (searchTerms.size() == 0) + if (searchTerms.length == 0) + return 0; + + int nterms = 0; + for (boolean b : searchTerms) + if (b) nterms++; + return nterms; } @@ -84,13 +110,16 @@ /** * Adds another component to the cosine. */ - public void add(final String term, final double weight) { + public void add(final int termNdx, final double weight) { synchronized (this) { cosine += weight; - nterms++; +// nterms++; + +// searchTerms.set(termNdx, true); + searchTerms[termNdx] = true; } @@ -105,7 +134,7 @@ public String toString() { - return "Hit{docId"+docId+",nterms="+nterms+",cosine="+cosine+"}"; + return "Hit{docId"+docId+",nterms="+getTermCount()+",cosine="+cosine+"}"; } Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/search/ReadIndexTask.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/search/ReadIndexTask.java 2013-04-25 15:59:02 UTC (rev 7077) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/search/ReadIndexTask.java 2013-04-25 16:23:07 UTC (rev 7078) @@ -48,7 +48,7 @@ * inserted, the {@link Hit#setDocId(long) docId} is set on the {@link Hit} * and a new instance is assigned to {@link #tmp}. */ - private Hit<V> tmp = new Hit<V>(); + private Hit<V> tmp;// = new Hit<V>(); /** * Setup a task that will perform a range scan for entries matching the @@ -56,6 +56,10 @@ * * @param termText * The term text for the search term. + * @param termNdx + * The index of this term within the overall search. + * @param numTerms + * The overall number of search terms. * @param prefixMatch * When <code>true</code> any term having <i>termText</i> as a * prefix will be matched. Otherwise the term must be an exact @@ -67,11 +71,12 @@ * @param hits * The map where the hits are being aggregated. */ - public ReadIndexTask(final String termText, final boolean prefixMatch, - final double queryTermWeight, final FullTextIndex<V> searchEngine, - final IHitCollector<V> hits) { + public ReadIndexTask(final String termText, + final int termNdx, final int numTerms, + final boolean prefixMatch, final double queryTermWeight, + final FullTextIndex<V> searchEngine, final IHitCollector<V> hits) { - super(termText, prefixMatch, queryTermWeight, searchEngine); + super(termText, termNdx, numTerms, prefixMatch, queryTermWeight, searchEngine); if (hits == null) throw new IllegalArgumentException(); @@ -93,6 +98,8 @@ itr = searchEngine.getIndex() .rangeIterator(fromKey, toKey, 0/* capacity */, IRangeQuery.KEYS | IRangeQuery.VALS, null/* filter */); + + tmp = new Hit<V>(numTerms); } @@ -175,13 +182,13 @@ if (oldValue == null) { hit = tmp; hit.setDocId(docId); - tmp = new Hit<V>(); + tmp = new Hit<V>(numQueryTerms); } else { hit = oldValue; } } - hit.add( queryTerm, queryTermWeight * termWeight ); + hit.add( queryTermNdx, queryTermWeight * termWeight ); nhits++; Property changes on: branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/joinGraph ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/joinGraph:6766-7048 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/joinGraph:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/joinGraph:6766-7077 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/joinGraph:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/bop/joinGraph:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/test/com/bigdata/bop/util ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/util:6766-7048 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/bop/util:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/bop/util:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/bop/util:6766-7077 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/bop/util:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/bop/util:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/bop/util:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/bop/util:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/test/com/bigdata/jsr166 ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/jsr166:6766-7048 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/jsr166:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/jsr166:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/jsr166:6766-7077 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/jsr166:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/jsr166:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/jsr166:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/jsr166:4814-4836 Property changes on: branches/READ_CACHE/bigdata/src/test/com/bigdata/util/httpd ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/util/httpd:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/util/httpd:6766-7048 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/util/httpd:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata/src/test/com/bigdata/util/httpd:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata/src/test/com/bigdata/util/httpd:6766-7077 /branches/INT64_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata/src/test/com/bigdata/util/httpd:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata/src/test/com/bigdata/util/httpd:4814-4836 Property changes on: branches/READ_CACHE/bigdata-compatibility ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-compatibility:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-compatibility:6766-7048 /branches/INT64_BRANCH/bigdata-compatibility:4486-4522 /branches/LARGE_LITERALS_REFACTOR/bigdata-compatibility:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-compatibility:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-compatibility:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-compatibility:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-compatibility:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-compatibility:6766-7077 /branches/INT64_BRANCH/bigdata-compatibility:4486-4522 /branches/LARGE_LITERALS_REFACTOR/bigdata-compatibility:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-compatibility:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-compatibility:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-compatibility:4814-4836 Property changes on: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/attr ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/attr:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/attr:6766-7048 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/attr:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/attr:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/attr:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/attr:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/attr:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/attr:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/attr:2981-3282,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/attr:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/attr:6766-7077 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/attr:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/attr:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/attr:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/attr:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/attr:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/attr:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/attr:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/attr:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/attr:2981-3282,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/disco ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/disco:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/disco:6766-7048 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/disco:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/disco:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/disco:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/disco:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/disco:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/disco:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/disco:2981-3282,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/disco:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/disco:6766-7077 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/disco:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/disco:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/disco:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/disco:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/disco:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/disco:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/disco:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/disco:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/disco:2981-3282,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/util/config ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/util/config:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/util/config:6766-7048 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/util/config:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/util/config:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/util/config:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/util/config:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/util/config:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/util/config:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/util/config:2981-3282,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-jini/src/java/com/bigdata/util/config:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-jini/src/java/com/bigdata/util/config:6766-7077 /branches/BTREE_BUFFER_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-jini/src/java/com/bigdata/util/config:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-jini/src/java/com/bigdata/util/config:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-jini/src/java/com/bigdata/util/config:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-jini/src/java/com/bigdata/util/config:4814-4836 /branches/bugfix-btm/bigdata-jini/src/java/com/bigdata/util/config:2594-3237 /branches/dev-btm/bigdata-jini/src/java/com/bigdata/util/config:2574-2730 /branches/fko/bigdata-jini/src/java/com/bigdata/util/config:3150-3194 /trunk/bigdata-jini/src/java/com/bigdata/util/config:2981-3282,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-perf ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf:6766-7048 /branches/BTREE_BUFFER_BRANCH/bigdata-perf:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-perf:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-perf:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-perf:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-perf:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf:4814-4836 /branches/bugfix-btm/bigdata-perf:2594-3237 /branches/dev-btm/bigdata-perf:2574-2730 /branches/fko/bigdata-perf:3150-3194 /trunk/bigdata-perf:2981-3043,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf:6766-7077 /branches/BTREE_BUFFER_BRANCH/bigdata-perf:2004-2045 /branches/DEV_BRANCH_27_OCT_2009/bigdata-perf:2270-2546,2548-2782 /branches/INT64_BRANCH/bigdata-perf:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf:4175-4387 /branches/LEXICON_REFACTOR_BRANCH/bigdata-perf:2633-3304 /branches/QUADS_QUERY_BRANCH/bigdata-perf:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf:4814-4836 /branches/bugfix-btm/bigdata-perf:2594-3237 /branches/dev-btm/bigdata-perf:2574-2730 /branches/fko/bigdata-perf:3150-3194 /trunk/bigdata-perf:2981-3043,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-perf/btc ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/btc:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/btc:6766-7048 /branches/INT64_BRANCH/bigdata-perf/btc:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/btc:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/btc:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/btc:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/btc:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/btc:4814-4836 /trunk/bigdata-perf/btc:2981-3043,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/btc:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/btc:6766-7077 /branches/INT64_BRANCH/bigdata-perf/btc:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/btc:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/btc:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/btc:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/btc:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/btc:4814-4836 /trunk/bigdata-perf/btc:2981-3043,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-perf/btc/src/resources ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/btc/src/resources:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/btc/src/resources:6766-7048 /branches/INT64_BRANCH/bigdata-perf/btc/src/resources:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/btc/src/resources:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/btc/src/resources:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/btc/src/resources:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/btc/src/resources:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/btc/src/resources:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/btc/src/resources:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/btc/src/resources:6766-7077 /branches/INT64_BRANCH/bigdata-perf/btc/src/resources:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/btc/src/resources:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/btc/src/resources:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/btc/src/resources:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/btc/src/resources:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/btc/src/resources:4814-4836 Property changes on: branches/READ_CACHE/bigdata-perf/lubm ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/lubm:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/lubm:6766-7048 /branches/INT64_BRANCH/bigdata-perf/lubm:4486-4522 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/lubm:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/lubm:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/lubm:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/lubm:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/lubm:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/lubm:6766-7077 /branches/INT64_BRANCH/bigdata-perf/lubm:4486-4522 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/lubm:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/lubm:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/lubm:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/lubm:4814-4836 Property changes on: branches/READ_CACHE/bigdata-perf/uniprot ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/uniprot:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/uniprot:6766-7048 /branches/INT64_BRANCH/bigdata-perf/uniprot:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/uniprot:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/uniprot:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/uniprot:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/uniprot:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/uniprot:4814-4836 /trunk/bigdata-perf/uniprot:2981-3043,3368-3437,3656-4061 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/uniprot:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/uniprot:6766-7077 /branches/INT64_BRANCH/bigdata-perf/uniprot:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/uniprot:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/uniprot:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/uniprot:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/uniprot:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/uniprot:4814-4836 /trunk/bigdata-perf/uniprot:2981-3043,3368-3437,3656-4061 Property changes on: branches/READ_CACHE/bigdata-perf/uniprot/src ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/uniprot/src:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/uniprot/src:6766-7048 /branches/INT64_BRANCH/bigdata-perf/uniprot/src:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/uniprot/src:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/uniprot/src:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/uniprot/src:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/uniprot/src:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/uniprot/src:4814-4836 + /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-perf/uniprot/src:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-perf/uniprot/src:6766-7077 /branches/INT64_BRANCH/bigdata-perf/uniprot/src:4486-4522 /branches/JOURNAL_HA_BRANCH/bigdata-perf/uniprot/src:2596-4066 /branches/LARGE_LITERALS_REFACTOR/bigdata-perf/uniprot/src:4175-4387 /branches/QUADS_QUERY_BRANCH/bigdata-perf/uniprot/src:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-4792,4794-4796,4798-4801 /branches/RWSTORE_1_1_0_DEBUG/bigdata-perf/uniprot/src:5896-5935 /branches/TIDS_PLUS_BLOBS_BRANCH/bigdata-perf/uniprot/src:4814-4836 Property changes on: branches/READ_CACHE/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate ___________________________________________________________________ Modified: svn:mergeinfo - /branches/BIGDATA_OPENRDF_2_6_9_UPDATE/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate:6769-6785 /branches/BIGDATA_RELEASE_1_2_0/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate:6766-7048 /branches/INT64_BRANCH/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate:4486-4522 /branches/QUADS_QUERY_BRANCH/bigdata-rdf/src/java/com/bigdata/bop/rdf/aggregate:4525-4531,4533-4548,4550-4584,4586-4609,4611-4632,4634-4643,4646-4672,4674-4685,4687-4693,4697-4735,4737-4782,4784-479... [truncated message content] |
From: <tho...@us...> - 2013-04-25 17:07:02
|
Revision: 7079 http://bigdata.svn.sourceforge.net/bigdata/?rev=7079&view=rev Author: thompsonbry Date: 2013-04-25 17:06:50 +0000 (Thu, 25 Apr 2013) Log Message: ----------- Checkpoint on refactor to support heirarchical directory structures for snapshots and HALog files. This is being done primarily for the HALog files. Since there is one file per commit point, there can be quite a few of these files. - Modified the SnapshotIndex (was CommitTimeIndex) to use the UnisolatedReadWriteIndex for thread safety and wrote a direct test suite for the SnapshotIndex. Added the sizeOnDisk to a new ISnapshotRecord interface whose instances are the values in the SnapshotIndex. Added a CommitCounterUtility class to ecapsulate some of the common routines for generating and parsing these file names. Updated the test suite where we had hard-coded some of the snapshot file names. Modified the SnapshotManager to use a recursive scan. - Extracted the recursiveDelete() methods from HAJournalServer, HAJournal, and SnapshotManager. I have currently placed this on CommitCounterUtility, but maybe a better location can be found. I added a boolean flag indicating whether the method would throw an IOException if a visited file could not be removed so we can handle the two use cases (manditory cleanup throws an exception while best effort cleanup does not). - HAJournalServer.deleteSnapshots(token,earliestRestorableCommitTime) was refactored to use the SnapshotIndex rather than scanning the file system for better efficiency and scalability. The method was moved into the SnapshotManager. - I have not yet modified modified the SnapshotManager to actually generate recursive directory structures for file names. That would be done by changing the CommitCounterUtility. I will do that once I get the HALog stuff refactored as well. All HA tests are green. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/IHALogReader.java branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotManager.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/AbstractHA3BackupTestCase.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3RestorePolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHA3SnapshotPolicy2.java branches/READ_CACHE/bigdata-sails/src/java/com/bigdata/rdf/sail/webapp/HAStatusServletUtil.java Added Paths: ----------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitCounterUtility.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotIndex.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestSnapshotIndex.java Removed Paths: ------------- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/IHALogReader.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/IHALogReader.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/halog/IHALogReader.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -23,6 +23,8 @@ */ package com.bigdata.ha.halog; +import java.io.File; +import java.io.FileFilter; import java.io.IOException; import java.nio.ByteBuffer; import java.security.DigestException; @@ -44,7 +46,30 @@ */ public static final String HA_LOG_EXT = ".ha-log"; - /** + /** + * A {@link FileFilter} that visits all files ending with the + * {@link #HA_LOG_EXT} and the names of all direct child directories. This + * {@link FileFilter} may be used to establish recursive scans of the HALog + * directory. + */ + static public final FileFilter HALOG_FILTER = new FileFilter() { + + @Override + public boolean accept(File f) { + + if (f.isDirectory()) { + + return true; + + } + + return f.getName().endsWith(HA_LOG_EXT); + + } + + }; + + /** * Closes the Reader. * * @throws IOException Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/journal/Journal.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -716,7 +716,7 @@ final private AtomicReference<BarrierState> barrierRef = new AtomicReference<BarrierState>(); @Override - public void executeWithBarrierLock(final Runnable r) { + public void runWithBarrierLock(final Runnable r) { barrierLock.lock(); try { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/service/AbstractHATransactionService.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -58,6 +58,6 @@ /** * Used to make a serviceJoin() MUTEX with the consensus protocol. */ - abstract public void executeWithBarrierLock(Runnable r); + abstract public void runWithBarrierLock(Runnable r); } Added: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitCounterUtility.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitCounterUtility.java (rev 0) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitCounterUtility.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -0,0 +1,198 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.journal.jini.ha; + +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.util.Formatter; + +import org.apache.log4j.Logger; + +/** + * Utility class for operations on files that are named using a commit counter. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public class CommitCounterUtility { + + private static final Logger log = Logger + .getLogger(CommitCounterUtility.class); + + /** + * Return the name of the {@link File} associated with the commitCounter. + * + * @param dir + * The directory spanning all such files. + * @param commitCounter + * The commit counter for the current root block on the journal. + * @param ext + * The filename extension. + * @return The name of the corresponding snapshot file. + */ + public static File getCommitCounterFile(final File dir, + final long commitCounter, final String ext) { + + /* + * Format the name of the file. + * + * Note: The commit counter in the file name should be zero filled to 20 + * digits so we have the files in lexical order in the file system (for + * convenience). + */ + final String file; + { + + final StringBuilder sb = new StringBuilder(); + + final Formatter f = new Formatter(sb); + + f.format("%020d" + ext, commitCounter); + f.flush(); + f.close(); + + file = sb.toString(); + + } + + return new File(dir, file); + + } + + /** + * Parse out the commitCounter from the file name. + * + * @param name + * The file name + * @param ext + * The expected file extension. + * + * @return The commit counter from the file name. + * + * @throws IllegalArgumentException + * if either argument is <code>null</code> + * @throws NumberFormatException + * if the file name can not be interpreted as a commit counter. + */ + public static long parseCommitCounterFile(final String name, + final String ext) throws NumberFormatException { + + if (name == null) + throw new IllegalArgumentException(); + + if (ext == null) + throw new IllegalArgumentException(); + + // Strip off the filename extension. + final int len = name.length() - ext.length(); + + final String fileBaseName = name.substring(0, len); + + // Closing commitCounter for snapshot file. + final long commitCounter = Long.parseLong(fileBaseName); + + return commitCounter; + + } + + /** + * Return the basename of the file (strip off the extension). + * + * @param name + * The file name. + * @param ext + * The extension. + * + * @return The base name of the file without the extension. + */ + public static String getBaseName(final String name, final String ext) { + + final String basename = name.substring(0, name.length() - ext.length()); + + return basename; + + } + + /** + * Recursively removes any files and subdirectories and then removes the + * file (or directory) itself. Only files recognized by + * {@link #getFileFilter()} will be deleted. + * <p> + * Note: A dedicated version of this method exists here to thrown an + * {@link IOException} if we can not delete a file. This is deliberate. It + * is thrown to prevent a REBUILD from proceeding unless we can clear out + * the old snapshot and HALog files. + * + * @param errorIfDeleteFails + * When <code>true</code> and {@link IOException} is thrown if a + * file matching the filter or an empty directory matching the + * filter can not be removed. When <code>false</code>, that event + * is logged @ WARN instead. + * @param f + * A file or directory. + * @param fileFilter + * A filter matching the files and directories to be visited and + * removed. If directories are matched, then they will be removed + * iff they are empty. A depth first visitation is used, so the + * files and sub-directories will be cleared before we attempt to + * remove the parent directory. + * @throws IOException + * if any file or non-empty directory can not be deleted (iff + * <i>errorIfDeleteFails</i> is <code>true</code>). + */ + public static void recursiveDelete(final boolean errorIfDeleteFails, + final File f, final FileFilter fileFilter) throws IOException { + + if (f.isDirectory()) { + + final File[] children = f.listFiles(fileFilter); + + for (int i = 0; i < children.length; i++) { + + recursiveDelete(errorIfDeleteFails, children[i], fileFilter); + + } + + } + + if (log.isInfoEnabled()) + log.info("Removing: " + f); + + if (f.exists() && !f.delete()) { + if (f.isDirectory() && f.list().length != 0) { + // Ignore non-empty directory. + return; + } + final String msg = "Could not remove file: " + f; + if (errorIfDeleteFails) { + // Complete if we can not delete a file. + throw new IOException(msg); + } else { + log.warn(msg); + } + } + + } + +} Deleted: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -1,548 +0,0 @@ -/** - -Copyright (C) SYSTAP, LLC 2006-2007. 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 -*/ -package com.bigdata.journal.jini.ha; - -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.nio.ByteBuffer; -import java.util.UUID; - -import com.bigdata.btree.BTree; -import com.bigdata.btree.BytesUtil; -import com.bigdata.btree.Checkpoint; -import com.bigdata.btree.DefaultTupleSerializer; -import com.bigdata.btree.IRangeQuery; -import com.bigdata.btree.ITuple; -import com.bigdata.btree.ITupleIterator; -import com.bigdata.btree.IndexMetadata; -import com.bigdata.btree.keys.ASCIIKeyBuilderFactory; -import com.bigdata.btree.keys.IKeyBuilder; -import com.bigdata.btree.keys.IKeyBuilderFactory; -import com.bigdata.btree.keys.KeyBuilder; -import com.bigdata.journal.ICommitRecord; -import com.bigdata.journal.IRootBlockView; -import com.bigdata.journal.RootBlockView; -import com.bigdata.rawstore.Bytes; -import com.bigdata.rawstore.IRawStore; -import com.bigdata.util.ChecksumUtility; - -/** - * {@link BTree} mapping <em>commitTime</em> (long integers) to - * {@link IRootBlockView} records. - * <p> - * Note: Access to this object MUST be synchronized. - * <p> - * Note: This is used as a transient data structure that is populated from the - * file system by the {@link HAJournalServer}. A separate instance is maintained - * for the HALog files and the snapshot files. - */ -public class CommitTimeIndex extends BTree { - - /** - * Instance used to encode the timestamp into the key. - */ - final private IKeyBuilder keyBuilder = new KeyBuilder(Bytes.SIZEOF_LONG); - - /** - * Create a transient instance. - * - * @return The new instance. - */ - static public CommitTimeIndex createTransient() { - - final IndexMetadata metadata = new IndexMetadata(UUID.randomUUID()); - - metadata.setBTreeClassName(CommitTimeIndex.class.getName()); - - metadata.setTupleSerializer(new TupleSerializer( - new ASCIIKeyBuilderFactory(Bytes.SIZEOF_LONG))); - - return (CommitTimeIndex) BTree.createTransient(/*store, */metadata); - - } - - /** - * Load from the store. - * - * @param store - * The backing store. - * @param checkpoint - * The {@link Checkpoint} record. - * @param metadata - * The metadata record for the index. - */ - public CommitTimeIndex(final IRawStore store, final Checkpoint checkpoint, - final IndexMetadata metadata, final boolean readOnly) { - - super(store, checkpoint, metadata, readOnly); - - } - - /** - * Encodes the commit time into a key. - * - * @param commitTime - * The commit time. - * - * @return The corresponding key. - */ - private byte[] getKey(final long commitTime) { - - return keyBuilder.reset().append(commitTime).getKey(); - - } - - /** - * Return the {@link IRootBlock} identifying the journal having the largest - * commitTime that is less than or equal to the given timestamp. This is - * used primarily to locate the commit record that will serve as the ground - * state for a transaction having <i>timestamp</i> as its start time. In - * this context the LTE search identifies the most recent commit state that - * not later than the start time of the transaction. - * - * @param timestamp - * The given timestamp. - * - * @return The description of the relevant journal resource -or- - * <code>null</code> iff there are no journals in the index that - * satisify the probe. - * - * @throws IllegalArgumentException - * if <i>timestamp</i> is less than or equals to ZERO (0L). - */ - synchronized public IRootBlockView find(final long timestamp) { - - if (timestamp <= 0L) - throw new IllegalArgumentException(); - - // find (first less than or equal to). - final long index = findIndexOf(timestamp); - - if(index == -1) { - - // No match. - return null; - - } - - return valueAtIndex(index); - - } - - /** - * Retrieve the entry from the index. - */ - private IRootBlockView valueAtIndex(final long index) { - - final byte[] val = super.valueAt(index); - - assert val != null : "Entry has null value: index=" + index; - - final IRootBlockView entry = new RootBlockView(false/* rootBlock0 */, - ByteBuffer.wrap(val), ChecksumUtility.getCHK()); - - return entry; - - } - - /** - * Return the {@link IRootBlockView} identifying the first snapshot whose - * <em>commitTime</em> is strictly greater than the timestamp. - * - * @param timestamp - * The timestamp. A value of ZERO (0) may be used to find the - * first snapshot. - * - * @return The root block of that snapshot -or- <code>null</code> if there - * is no snapshot whose timestamp is strictly greater than - * <i>timestamp</i>. - */ - synchronized public IRootBlockView findNext(final long timestamp) { - - /* - * Note: can also be written using rangeIterator().next(). - */ - - if (timestamp < 0L) - throw new IllegalArgumentException(); - - // find first strictly greater than. - final long index = findIndexOf(timestamp) + 1; - - if (index == nentries) { - - // No match. - - return null; - - } - - return valueAtIndex(index); - - } - - /** - * Find the index of the {@link ICommitRecord} having the largest timestamp - * that is less than or equal to the given timestamp. - * - * @return The index of the {@link ICommitRecord} having the largest - * timestamp that is less than or equal to the given timestamp -or- - * <code>-1</code> iff there are no {@link ICommitRecord}s - * defined. - */ - synchronized public long findIndexOf(final long timestamp) { - - long pos = super.indexOf(getKey(timestamp)); - - if (pos < 0) { - - /* - * the key lies between the entries in the index, or possible before - * the first entry in the index. [pos] represents the insert - * position. we convert it to an entry index and subtract one to get - * the index of the first commit record less than the given - * timestamp. - */ - - pos = -(pos+1); - - if (pos == 0) { - - // No entry is less than or equal to this timestamp. - return -1; - - } - - pos--; - - return pos; - - } else { - - /* - * exact hit on an entry. - */ - - return pos; - - } - - } - - /** - * Add an entry under the commitTime associated with the - * {@link IRootBlockView} record. - * - * @param rootBlock - * The {@link IRootBlockView} record. - * - * @exception IllegalArgumentException - * if <i>commitTime</i> is <code>0L</code>. - * @exception IllegalArgumentException - * if <i>rootBLock</i> is <code>null</code>. - * @exception IllegalArgumentException - * if there is already an entry registered under for the - * given timestamp. - */ - synchronized public void add(final IRootBlockView rootBlock) { - - if (rootBlock == null) - throw new IllegalArgumentException(); - - final long createTime = rootBlock.getLastCommitTime(); - - if (createTime == 0L) - throw new IllegalArgumentException(); - - final byte[] key = getKey(createTime); - - if(super.contains(key)) { - - throw new IllegalArgumentException("entry exists: timestamp=" - + createTime); - - } - - // add a serialized entry to the persistent index. - super.insert(key, BytesUtil.getBytes(rootBlock.asReadOnlyBuffer())); - - } - - /** - * Find and return the {@link IRootBlockView} for the oldest snapshot (if - * any). - * - * @return That {@link IRootBlockView} -or- <code>null</code> if there are - * no snapshots. - */ - public IRootBlockView getOldestSnapshot() { - - synchronized (this) { - - if (getEntryCount() == 0L) { - - // Empty index. - return null; - - } - - // Lookup first tuple in index. - @SuppressWarnings("unchecked") - final ITuple<IRootBlockView> t = valueAt(0L, getLookupTuple()); - - final IRootBlockView rb = t.getObject(); - - return rb; - - } - - } - - /** - * Find the {@link IRootBlockView} for the most recent snapshot (if any). - * - * @return That {@link IRootBlockView} -or- <code>null</code> if there are - * no snapshots. - */ - public IRootBlockView getNewestSnapshot() { - - /* - * Note: This could also be written using valueAt(nentries). - */ - synchronized (this) { - - @SuppressWarnings("unchecked") - final ITupleIterator<IRootBlockView> itr = rangeIterator( - null/* fromKey */, null/* toKey */, 1/* capacity */, - IRangeQuery.DEFAULT | IRangeQuery.REVERSE/* flags */, null/* filter */); - - if (!itr.hasNext()) { - - return null; - - } - - final ITuple<IRootBlockView> t = itr.next(); - - final IRootBlockView rb = t.getObject(); - - return rb; - - } - - } - - /** - * Find the oldest snapshot whose commit counter is LTE the specified commit - * counter. - * - * @return The {@link IRootBlockView} for that snapshot -or- - * <code>null</code> if there is no such snapshot. - * - * @throws IllegalArgumentException - * if <code>commitCounter LT ZERO (0)</code> - */ - public IRootBlockView findByCommitCounter(final long commitCounter) { - - if (commitCounter < 0L) - throw new IllegalArgumentException(); - - synchronized (this) { - - // Reverse scan. - @SuppressWarnings("unchecked") - final ITupleIterator<IRootBlockView> itr = rangeIterator( - null/* fromKey */, null/* toKey */, 0/* capacity */, - IRangeQuery.DEFAULT | IRangeQuery.REVERSE/* flags */, null/* filter */); - - while (itr.hasNext()) { - - final ITuple<IRootBlockView> t = itr.next(); - - final IRootBlockView rb = t.getObject(); - - if (rb.getCommitCounter() <= commitCounter) { - - // First snapshot LTE that commit counter. - return rb; - - } - - } - - return null; - - } - - } - - /** - * Return the snapshot that is associated with the specified ordinal index - * (origin ZERO) counting backwards from the most recent snapshot (0) - * towards the earliest snapshot (nsnapshots-1). - * <p> - * Note: The effective index is given by <code>(entryCount-1)-index</code>. - * If the effective index is LT ZERO (0) then there is no such snapshot and - * this method will return <code>null</code>. - * - * @param index - * The index. - * - * @return The {@link IRootBlockView} for that snapshot -or- - * <code>null</code> if there is no such snapshot. - * - * @throws IllegalArgumentException - * if <code>index LT ZERO (0)</code> - */ - public IRootBlockView getSnapshotByReverseIndex(final int index) { - - if (index < 0) - throw new IllegalArgumentException(); - - synchronized (this) { - - final long entryCount = getEntryCount(); - - if (entryCount > Integer.MAX_VALUE) - throw new AssertionError(); - - final int effectiveIndex = ((int) entryCount - 1) - index; - - if (effectiveIndex < 0) { - - // No such snapshot. - return null; - - } - - @SuppressWarnings("unchecked") - final ITuple<IRootBlockView> t = valueAt(effectiveIndex, - getLookupTuple()); - - final IRootBlockView rb = t.getObject(); - - return rb; - - } - - } - - /** - * Encapsulates key and value formation. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id: JournalIndex.java 5892 2012-01-27 13:22:11Z thompsonbry $ - */ - static protected class TupleSerializer extends - DefaultTupleSerializer<Long, IRootBlockView> { - - /** - * - */ - private static final long serialVersionUID = -2851852959439807542L; - - /** - * De-serialization ctor. - */ - public TupleSerializer() { - - super(); - - } - - /** - * Ctor when creating a new instance. - * - * @param keyBuilderFactory - */ - public TupleSerializer(final IKeyBuilderFactory keyBuilderFactory) { - - super(keyBuilderFactory); - - } - - /** - * Decodes the key as a commit time. - */ - @Override - public Long deserializeKey(final ITuple tuple) { - - return KeyBuilder - .decodeLong(tuple.getKeyBuffer().array(), 0/* offset */); - - } - - /** - * De-serializes an object from the {@link ITuple#getValue() value} stored - * in the tuple (ignores the key stored in the tuple). - */ - public IRootBlockView deserialize(final ITuple tuple) { - - if (tuple == null) - throw new IllegalArgumentException(); - - return (IRootBlockView) new RootBlockView(false/* rootBlock0 */, - ByteBuffer.wrap(tuple.getValue()), ChecksumUtility.getCHK()); - - } - - /** - * The initial version (no additional persistent state). - */ - private final static transient byte VERSION0 = 0; - - /** - * The current version. - */ - private final static transient byte VERSION = VERSION0; - - public void readExternal(final ObjectInput in) throws IOException, - ClassNotFoundException { - - super.readExternal(in); - - final byte version = in.readByte(); - - switch (version) { - case VERSION0: - break; - default: - throw new UnsupportedOperationException("Unknown version: " - + version); - } - - } - - public void writeExternal(final ObjectOutput out) throws IOException { - - super.writeExternal(out); - - out.writeByte(VERSION); - - } - - } - -} Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/DefaultRestorePolicy.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -28,6 +28,7 @@ import org.apache.log4j.Logger; import com.bigdata.journal.IRootBlockView; +import com.bigdata.journal.jini.ha.SnapshotIndex.ISnapshotRecord; /** * The default restore policy. This policy supports three different criteria for @@ -233,7 +234,8 @@ final long then = now - minRestoreAgeMillis; // The root block for the snapshot with a commitTime LTE [then]. - final IRootBlockView rootBlock = jnl.getSnapshotManager().find(then); + final ISnapshotRecord sr = jnl.getSnapshotManager().find(then); + final IRootBlockView rootBlock = sr == null ? null : sr.getRootBlock(); if (rootBlock == null) { @@ -258,17 +260,17 @@ private long getEarliestRestorableCommitCounterBySnapshots( final HAJournal jnl, final long commitCounterOnJournal) { - final IRootBlockView rootBlock = jnl.getSnapshotManager() + final ISnapshotRecord r = jnl.getSnapshotManager() .getSnapshotByReverseIndex(minSnapshots - 1); - if (rootBlock == null) { + if (r == null) { // There are fewer than minSnapshots snapshots. return 0L; } - return rootBlock.getCommitCounter(); + return r.getRootBlock().getCommitCounter(); } @@ -292,16 +294,16 @@ } // Find the oldest snapshot LTE that commitCounter. - final IRootBlockView rootBlock = jnl.getSnapshotManager() - .findByCommitCounter(desiredCommitCounter); + final ISnapshotRecord r = jnl.getSnapshotManager().findByCommitCounter( + desiredCommitCounter); - if (rootBlock == null) { + if (r == null) { return commitCounterOnJournal; } - return rootBlock.getCommitCounter(); + return r.getRootBlock().getCommitCounter(); } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -326,7 +326,9 @@ if (!haLogDir.exists()) { // Create the directory. - haLogDir.mkdirs(); + if (!haLogDir.mkdirs()) + throw new IOException("Could not create directory: " + + haLogDir); } @@ -519,52 +521,33 @@ }); recursiveDelete(getSnapshotManager().getSnapshotDir(), - new FileFilter() { - - @Override - public boolean accept(File f) { + SnapshotManager.SNAPSHOT_FILTER); - if (f.isDirectory()) - return true; - - return f.getName().endsWith(SnapshotManager.SNAPSHOT_EXT); - } - - }); - } /** * Recursively removes any files and subdirectories and then removes the - * file (or directory) itself. Only files recognized by - * {@link #getFileFilter()} will be deleted. + * file (or directory) itself. Only files recognized by filter will be + * deleted. * * @param f * A file or directory. + * @param fileFilter + * The filter. */ - private void recursiveDelete(final File f,final FileFilter fileFilter) { + private void recursiveDelete(final File f, final FileFilter fileFilter) { - if (f.isDirectory()) { - - final File[] children = f.listFiles(fileFilter); - - for (int i = 0; i < children.length; i++) { - - recursiveDelete(children[i], fileFilter); - - } - + try { + CommitCounterUtility.recursiveDelete(false/* errorIfDeleteFails */, f, + fileFilter); + } catch (IOException e) { + /* + * Note: IOException is not thrown here since + * errorIfDeleteFails:=false. + */ + throw new RuntimeException(e); } - if (log.isInfoEnabled()) - log.info("Removing: " + f); - - if (f.exists() && !f.delete()) { - - log.warn("Could not remove: " + f); - - } - } /** @@ -645,7 +628,7 @@ // The commit counter of the desired closing root block. final long commitCounter = msg.getCommitCounter(); - final File logFile = new File(haLogDir, + final File logFile = new File(getHALogDir(), HALogWriter.getHALogFileName(commitCounter)); if (!logFile.exists()) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -24,6 +24,7 @@ package com.bigdata.journal.jini.ha; import java.io.File; +import java.io.FileFilter; import java.io.FileNotFoundException; import java.io.FilenameFilter; import java.io.IOException; @@ -2864,7 +2865,7 @@ * TODO What happens if we are blocked here? */ ((AbstractHATransactionService) journal.getTransactionService()) - .executeWithBarrierLock(new Runnable() { + .runWithBarrierLock(new Runnable() { public void run() { @@ -3099,8 +3100,9 @@ // Delete snapshots, returning commit counter of the oldest // retained snapshot. - final long earliestRetainedSnapshotLastCommitCounter = deleteSnapshots( - token, earliestRestorableCommitPoint); + final long earliestRetainedSnapshotLastCommitCounter = journal + .getSnapshotManager().deleteSnapshots(token, + earliestRestorableCommitPoint); // Delete HALogs not retained by that snapshot. deleteHALogs(token, earliestRetainedSnapshotLastCommitCounter); @@ -3113,158 +3115,154 @@ } - /** - * Delete snapshots that are no longer required. - * <p> - * Note: If ZERO (0) is passed into this method, then no snapshots will - * be deleted. This is because the first possible commit counter is ONE - * (1). - * - * @param earliestRestorableCommitPoint - * The earliest commit point that we need to be able to - * restore from local backups. - * - * @return The commitCounter of the earliest retained snapshot. - */ - private long deleteSnapshots(final long token, - final long earliestRestorableCommitPoint) { - /* - * List the snapshot files for this service. - */ - final File[] files; - // #of snapshot files found. Set during scan. - final AtomicLong nfound = new AtomicLong(); - // Set to the commit counter of the earliest retained snapshot. - final AtomicLong earliestRetainedSnapshotCommitCounter = new AtomicLong(Long.MAX_VALUE); - final SnapshotManager snapshotManager = journal - .getSnapshotManager(); - { - - final File snapshotDir = snapshotManager.getSnapshotDir(); - - files = snapshotDir.listFiles(new FilenameFilter() { - - /** - * Return <code>true</code> iff the file is an snapshot file - * that should be deleted. - * - * @param name - * The name of that file (encodes the - * commitCounter). - */ - @Override - public boolean accept(final File dir, final String name) { - - if (!name.endsWith(SnapshotManager.SNAPSHOT_EXT)) { - // Not an snapshot file. - return false; - } - - // Strip off the filename extension. - final int len = name.length() - - SnapshotManager.SNAPSHOT_EXT.length(); - final String fileBaseName = name.substring(0, len); - - // Closing commitCounter for snapshot file. - final long commitCounter = Long.parseLong(fileBaseName); - - // Count all snapshot files. - nfound.incrementAndGet(); - - // true iff we will delete this snapshot. - final boolean deleteFile = commitCounter < earliestRestorableCommitPoint; - - if (haLog.isInfoEnabled()) - log.info("snapshotFile=" - + name// - + ", deleteFile=" - + deleteFile// - + ", commitCounter=" - + commitCounter// - + ", earliestRestoreableCommitPoint=" - + earliestRestorableCommitPoint); - - if (!deleteFile - && commitCounter < earliestRetainedSnapshotCommitCounter - .get()) { - - /* - * Update the earliest retained snapshot. - */ - - earliestRetainedSnapshotCommitCounter - .set(commitCounter); - - } - - return deleteFile; - - } - }); - - } - - int ndeleted = 0; - long totalBytes = 0L; - - /* - * If people specify NoSnapshotPolicy then backup is in their hands. - * HALogs will not be retained beyond a fully met commit unless - * there is a snapshot against which they can be applied.. - */ - -// if (files.length == 0) { +// /** +// * Delete snapshots that are no longer required. +// * <p> +// * Note: If ZERO (0) is passed into this method, then no snapshots will +// * be deleted. This is because the first possible commit counter is ONE +// * (1). +// * +// * @param earliestRestorableCommitPoint +// * The earliest commit point that we need to be able to +// * restore from local backups. +// * +// * @return The commitCounter of the earliest retained snapshot. +// */ +// private long deleteSnapshots(final long token, +// final long earliestRestorableCommitPoint) { +// /* +// * List the snapshot files for this service. +// */ +// final File[] files; +// // #of snapshot files found. Set during scan. +// final AtomicLong nfound = new AtomicLong(); +// // Set to the commit counter of the earliest retained snapshot. +// final AtomicLong earliestRetainedSnapshotCommitCounter = new AtomicLong(Long.MAX_VALUE); +// final SnapshotManager snapshotManager = journal +// .getSnapshotManager(); +// { // -// /* -// * Note: If there are no snapshots then we MUST retain ALL HALog -// * files. -// */ -// earliestRetainedSnapshotCommitCounter.set(0L); -// -// } else { - - for (File file : files) { - - // #of bytes in that file. - final long len = file.length(); - - if (!getQuorum().isQuorumFullyMet(token)) { - /* - * Halt operation. - * - * Note: This is not an error, but we can not remove - * snapshots or HALogs if this invariant is violated. - */ - break; - } - - if (!snapshotManager.removeSnapshot(file)) { - - haLog.warn("COULD NOT DELETE FILE: " + file); - - continue; - - } - - ndeleted++; - - totalBytes += len; - - } - +// final File snapshotDir = snapshotManager.getSnapshotDir(); +// +// files = snapshotDir.listFiles(new FilenameFilter() { +// +// /** +// * Return <code>true</code> iff the file is an snapshot file +// * that should be deleted. +// * +// * @param name +// * The name of that file (encodes the +// * commitCounter). +// */ +// @Override +// public boolean accept(final File dir, final String name) { +// +// if (!name.endsWith(SnapshotManager.SNAPSHOT_EXT)) { +// // Not an snapshot file. +// return false; +// } +// +// // Closing commitCounter for snapshot file. +// final long commitCounter = SnapshotManager +// .parseCommitCounterFile(name); +// +// // Count all snapshot files. +// nfound.incrementAndGet(); +// +// // true iff we will delete this snapshot. +// final boolean deleteFile = commitCounter < earliestRestorableCommitPoint; +// +// if (haLog.isInfoEnabled()) +// log.info("snapshotFile=" +// + name// +// + ", deleteFile=" +// + deleteFile// +// + ", commitCounter=" +// + commitCounter// +// + ", earliestRestoreableCommitPoint=" +// + earliestRestorableCommitPoint); +// +// if (!deleteFile +// && commitCounter < earliestRetainedSnapshotCommitCounter +// .get()) { +// +// /* +// * Update the earliest retained snapshot. +// */ +// +// earliestRetainedSnapshotCommitCounter +// .set(commitCounter); +// +// } +// +// return deleteFile; +// +// } +// }); +// // } - - if (haLog.isInfoEnabled()) - haLog.info("PURGED SNAPSHOTS: nfound=" + nfound + ", ndeleted=" - + ndeleted + ", totalBytes=" + totalBytes - + ", earliestRestorableCommitPoint=" - + earliestRestorableCommitPoint - + ", earliestRetainedSnapshotCommitCounter=" - + earliestRetainedSnapshotCommitCounter.get()); - - return earliestRetainedSnapshotCommitCounter.get(); - - } +// +// int ndeleted = 0; +// long totalBytes = 0L; +// +// /* +// * If people specify NoSnapshotPolicy then backup is in their hands. +// * HALogs will not be retained beyond a fully met commit unless +// * there is a snapshot against which they can be applied.. +// */ +// +//// if (files.length == 0) { +//// +//// /* +//// * Note: If there are no snapshots then we MUST retain ALL HALog +//// * files. +//// */ +//// earliestRetainedSnapshotCommitCounter.set(0L); +//// +//// } else { +// +// for (File file : files) { +// +// // #of bytes in that file. +// final long len = file.length(); +// +// if (!getQuorum().isQuorumFullyMet(token)) { +// /* +// * Halt operation. +// * +// * Note: This is not an error, but we can not remove +// * snapshots or HALogs if this invariant is violated. +// */ +// break; +// } +// +// if (!snapshotManager.removeSnapshot(file)) { +// +// haLog.warn("COULD NOT DELETE FILE: " + file); +// +// continue; +// +// } +// +// ndeleted++; +// +// totalBytes += len; +// +// } +// +//// } +// +// if (haLog.isInfoEnabled()) +// haLog.info("PURGED SNAPSHOTS: nfound=" + nfound + ", ndeleted=" +// + ndeleted + ", totalBytes=" + totalBytes +// + ", earliestRestorableCommitPoint=" +// + earliestRestorableCommitPoint +// + ", earliestRetainedSnapshotCommitCounter=" +// + earliestRetainedSnapshotCommitCounter.get()); +// +// return earliestRetainedSnapshotCommitCounter.get(); +// +// } /** * Delete HALogs that are no longer required. @@ -3415,42 +3413,45 @@ haLog.warn("Destroying local backups."); // Delete all snapshots. - { - - final File snapshotDir = journal.getSnapshotManager() - .getSnapshotDir(); - - final File[] files = snapshotDir - .listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, - final String name) { - return name - .endsWith(SnapshotManager.SNAPSHOT_EXT); - } - }); - for (File file : files) { - if (!file.delete()) - throw new IOException("COULD NOT DELETE FILE: " - + file); - } + journal.getSnapshotManager().deleteAllSnapshots(); - } - // Delete all HALogs (except the current one). - { + deleteAllHALogsExceptCurrent(); + + } finally { - final File currentLogFile = journal.getHALogWriter() - .getFile(); + logLock.unlock(); - final String currentLogFileName = currentLogFile == null ? null - : currentLogFile.getName(); + } + + } - final File logDir = journal.getHALogDir(); + /** + * Delete all HALog files (except the current one). + * + * @throws IOException + */ + private void deleteAllHALogsExceptCurrent() throws IOException { + + final File currentLogFile = journal.getHALogWriter() + .getFile(); - final File[] files = logDir.listFiles(new FilenameFilter() { + final String currentLogFileName = currentLogFile == null ? null + : currentLogFile.getName(); + + final File logDir = journal.getHALogDir(); + + CommitCounterUtility.recursiveDelete(true/* errorIfDeleteFails */, + logDir, new FileFilter() { + @Override - public boolean accept(final File dir, final String name) { + public boolean accept(final File f) { + + if (f.isDirectory()) + return true; + + final String name = f.getName(); + // filter out the current log file if (currentLogFile != null && name.equals(currentLogFileName)) { @@ -3459,22 +3460,12 @@ */ return false; } + return name.endsWith(IHALogReader.HA_LOG_EXT); + } - }); - for (File file : files) { - if (!file.delete()) - throw new IOException("COULD NOT DELETE FILE: " - + file); - } - - } - - } finally { - logLock.unlock(); - - } + }); } @@ -3609,6 +3600,10 @@ NSSConfigurationOptions.PORT, Integer.TYPE, NSSConfigurationOptions.DEFAULT_PORT); +// final String servletContextListenerClass = (String) config.getEntry(COMPONENT, +// NSSConfigurationOptions.SERVLET_CONTEXT_LISTENER_CLASS, String.class, +// NSSConfigurationOptions.DEFAULT_SERVLET_CONTEXT_LISTENER_CLASS); + log.warn("Starting NSS: port=" + port); final Map<String, String> initParams = new LinkedHashMap<String, String>(); @@ -3622,6 +3617,9 @@ // Note: Create will be handled by the QuorumListener (above). initParams.put(ConfigParams.CREATE, Boolean.toString(create)); +// initParams.put(ConfigParams.SERVLET_CONTEXT_LISTENER_CLASS, +// servletContextListenerClass); + } if (jettyServer != null && jettyServer.isRunning()) { Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-04-25 16:23:07 UTC (rev 7078) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HARestore.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -457,13 +457,11 @@ // source is the snapshot. final File in = journalFile; - final String basename = journalFile.getName().substring( - 0, - journalFile.getName().length() - - SnapshotManager.SNAPSHOT_EXT.length()); + final long commitCounter = SnapshotManager + .parseCommitCounterFile(journalFile.getName()); // temporary file in the same directory as the snapshot. - final File out = File.createTempFile(basename + "-", + final File out = File.createTempFile("" + commitCounter + "-", Journal.Options.JNL, journalFile.getAbsoluteFile() .getParentFile()); Copied: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotIndex.java (from rev 7048, branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/CommitTimeIndex.java) =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotIndex.java (rev 0) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/SnapshotIndex.java 2013-04-25 17:06:50 UTC (rev 7079) @@ -0,0 +1,862 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 +*/ +package com.bigdata.journal.jini.ha; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; + +import com.bigdata.btree.BTree; +import com.bigdata.btree.BytesUtil; +import com.bigdata.btree.DefaultTupleSerializer; +import com.bigdata.btree.DelegateIndex; +import com.bigdata.btree.ILinearList; +import com.bigdata.btree.IRangeQuery; +import com.bigdata.btree.ITuple; +import com.bigdata.btree.ITupleIterator; +import com.bigdata.btree.IndexMetadata; +import com.bigdata.btree.Tuple; +import com.bigdata.btree.UnisolatedReadWriteIndex; +import com.bigdata.btree.keys.ASCIIKeyBuilderFactory; +import com.bigdata.btree.keys.IKeyBuilderFactory; +import com.bigdata.btree.keys.KeyBuilder; +import com.bigdata.journal.IRootBlockView; +import com.bigdata.journal.RootBlockView; +import com.bigdata.rawstore.Bytes; +import com.bigdata.util.ChecksumUtility; + +/** + * {@link BTree} mapping <em>commitTime</em> (long integers) to + * {@link ISnapshotRecord} records. + * <p> + * This object is thread-safe for concurrent readers and writers. + * <p> + * Note: This is used as a transient data structure that is populated from the + * file system by the {@link HAJournalServer}. + */ +public class SnapshotIndex extends DelegateIndex implements ILinearList { + + /** + * The underlying index. Access to this is NOT thread safe unless you take + * the appropriate lock on the {@link #readWriteLock}. + */ + private final BTree btree; + + /** + * The {@link ReadWriteLock} used by the {@link UnisolatedReadWriteIndex} to + * make operations on the underlying {@link #btree} thread-safe. + */ + private final ReadWriteLock readWriteLock; + + @SuppressWarnings("unchecked") + private Tuple<ISnapshotRecord> getLookupTuple() { + + return btree.getLookupTuple(); + + } + +// /** +// * Instance used to encode the timestamp into the key. +// */ +// final private IKeyBuilder keyBuilder = new KeyBuilder(Bytes.SIZEOF_LONG); + + /** + * Create a transient instance. + * + * @return The new instance. + */ + static public SnapshotIndex createTransient() { + + final IndexMetadata metadata = new IndexMetadata(UUID.randomUUID()); + +// metadata.setBTreeClassName(SnapshotIndex.class.getName()); + + metadata.setTupleSerializer(new TupleSerializer( + new ASCIIKeyBuilderFactory(Bytes.SIZEOF_LONG))); + + final BTree ndx = BTree.createTransient(/*store, */metadata); + + return new SnapshotIndex(ndx); + + } + + private SnapshotIndex(final BTree ndx) { + + // Wrap B+Tree for read/write thread safety. + super(new UnisolatedReadWriteIndex(ndx)); + + this.btree = ndx; + +// this.delegate = new UnisolatedReadWriteIndex(ndx); + + // Save reference to lock for extended synchronization patterns. + this.readWriteLock = UnisolatedReadWriteIndex.getReadWriteLock(ndx); + + } + +// /** +// * Load from the store. +// * +// * @param store +// * The backing store. +// * @param checkpoint +// * The {@link Checkpoint} record. +// * @param metadata +// * The metadata record for the index. +// */ +// public SnapshotIndex(final IRawStore store, final Checkpoint checkpoint, +// final IndexMetadata metadata, final boolean readOnly) { +// +// super(store, checkpoint, metadata, readOnly); +// +// } + + /** + * Encodes the commit time into a key. + * + * @param commitTime + * The commit time. + * + * @return The corresponding key. + */ + public byte[] getKey(final long commitTime) { + + return getIndexMetadata().g... [truncated message content] |
From: <tho...@us...> - 2013-04-30 13:04:21
|
Revision: 7092 http://bigdata.svn.sourceforge.net/bigdata/?rev=7092&view=rev Author: thompsonbry Date: 2013-04-30 13:04:12 +0000 (Tue, 30 Apr 2013) Log Message: ----------- Added a Configuration option to HAJournalServer to allow the HAJournal to be subclassed. This is being done to support writing unit tests that cause failures in the 2-phase commit protocol. Wrote a unit test which verifies that the HAJournal may be subclassed, including the ability to publish new RMI methods that are only visible to the test suite. Pushed down the Administrable RMI implementation onto HAJournal.HAGlueService and removed the use of the delegation pattern to wrap the HAGlueService with the Administrable methods. HAGlueDelegate is no longer used and has been deprecated. Modified Paths: -------------- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java Added Paths: ----------- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/MyHAJournal.java branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServerOverride.java Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-29 20:15:51 UTC (rev 7091) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/ha/HAGlueDelegate.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -69,6 +69,8 @@ * Delegation pattern. * * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + * + * @deprecated No longer used. */ public class HAGlueDelegate implements HAGlue { Modified: branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java =================================================================== --- branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java 2013-04-29 20:15:51 UTC (rev 7091) +++ branches/READ_CACHE/bigdata/src/java/com/bigdata/rwstore/IRWStrategy.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -38,7 +38,11 @@ /** * Called post commit to dispose any transient commit state retained to - * support reset/rollback + * support reset/rollback. + * <p> + * Note: It is the responsibility of the commit protocol layers to wind up + * calling {@link IBufferStrategy#abort()} if there is a failure during the + * commit protocol. */ public void postCommit(); Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-29 20:15:51 UTC (rev 7091) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournal.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -31,6 +31,7 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.rmi.Remote; +import java.rmi.RemoteException; import java.rmi.server.ExportException; import java.security.DigestException; import java.security.MessageDigest; @@ -62,6 +63,7 @@ import com.bigdata.ha.HAGlue; import com.bigdata.ha.QuorumService; import com.bigdata.ha.QuorumServiceBase; +import com.bigdata.ha.RunState; import com.bigdata.ha.halog.HALogReader; import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.halog.IHALogReader; @@ -97,11 +99,15 @@ import com.bigdata.journal.Journal; import com.bigdata.journal.WriteExecutorService; import com.bigdata.journal.jini.ha.HAJournalServer.HAQuorumService; +import com.bigdata.journal.jini.ha.HAJournalServer.NSSConfigurationOptions; +import com.bigdata.journal.jini.ha.HAJournalServer.RunStateEnum; import com.bigdata.quorum.AsynchronousQuorumCloseException; import com.bigdata.quorum.Quorum; import com.bigdata.quorum.zk.ZKQuorumImpl; import com.bigdata.service.AbstractTransactionService; import com.bigdata.service.jini.JiniClient; +import com.bigdata.service.jini.RemoteAdministrable; +import com.bigdata.service.jini.RemoteDestroyAdmin; import com.bigdata.service.proxy.ClientFuture; import com.bigdata.service.proxy.RemoteFuture; import com.bigdata.service.proxy.RemoteFutureImpl; @@ -170,6 +176,11 @@ } /** + * The {@link HAJournalServer} instance that is managing this {@link HAJournal}. + */ + private final HAJournalServer server; + + /** * @see HAJournalServer.ConfigurationOptions#WRITE_PIPELINE_ADDR */ private final InetSocketAddress writePipelineAddr; @@ -229,6 +240,16 @@ } /** + * The {@link HAJournalServer} instance that is managing this + * {@link HAJournal}. + */ + protected HAJournalServer getHAJournalServer() { + + return server; + + } + + /** * {@inheritDoc} * <p> * Overridden to strengthen the return type. @@ -241,12 +262,15 @@ } /** - * + * @param server + * The {@link HAJournalServer} instance. * @param config + * The {@link Configuration} object. * @param quorum + * The {@link Quorum} implementation. * * @throws ConfigurationException - * @throws IOException + * @throws IOException */ public HAJournal(final HAJournalServer server, final Configuration config, final Quorum<HAGlue, QuorumService<HAGlue>> quorum) @@ -278,6 +302,8 @@ super(checkProperties(properties), quorum); + this.server = server; + { // The address at which this service exposes its write pipeline. @@ -398,6 +424,10 @@ } + /* + * Note: The HAJournal and HAGlueService MAY be subclassed. Therefore, do + * not perform any initialization in this factory method. + */ @Override protected HAGlue newHAGlue(final UUID serviceId) { @@ -425,11 +455,15 @@ * observes a quorum break or similar event. This is just being * proactive. * - * TODO Lift into HAJournalServer.quorumBreak() handler? - * * FIXME This will not be called if the quorum remains met but the * local service leaves the quorum. However, we should still cancel - * a running snapshot if that occurs. + * a running snapshot if that occurs (if we add a serviceLeave() + * handle then this will fix that). + * + * FIXME We MUST ensure that the snapshot is terminated based on the + * clear of the haReadyToken. Even if the quorumToken does not + * change, the haReadyToken can be cleared if a serviceLeave() is + * performed by this service. */ final Future<IHASnapshotResponse> ft = getSnapshotManager() @@ -605,7 +639,8 @@ /** * Extended implementation supports RMI. */ - protected class HAGlueService extends BasicHA { + protected class HAGlueService extends BasicHA implements + RemoteAdministrable, RemoteDestroyAdmin { protected HAGlueService(final UUID serviceId) { @@ -1442,7 +1477,7 @@ * @return The {@link Exporter}. */ protected Exporter getExporter(final boolean enableDGC) { - + // TODO This should be based on the Configuration object (EXPORTER field). See AbstractServer. return new BasicJeriExporter(TcpServerEndpoint .getInstance(0/* port */), invocationLayerFactory, enableDGC, false/* keepAlive */); @@ -1523,6 +1558,122 @@ } + /* + * + */ + + /** + * Returns an object that implements whatever administration interfaces + * are appropriate for the particular service. + * + * @return an object that implements whatever administration interfaces + * are appropriate for the particular service. + */ + public Object getAdmin() throws RemoteException { + + if (log.isInfoEnabled()) + log.info("serviceID=" + server.getServiceID()); + + return server.proxy; + + } + + /* + * DestroyAdmin + */ + + @Override + public void destroy() { + + server.runShutdown(true/* destroy */); + + } + + @Override + public void shutdown() { + + server.runShutdown(false/* destroy */); + + } + + @Override + public void shutdownNow() { + + server.runShutdown(false/* destroy */); + + } + + /* + * Misc. + */ + + @Override + public int getNSSPort() { + + final String COMPONENT = NSSConfigurationOptions.COMPONENT; + + try { + + final Integer port = (Integer) server.config.getEntry( + COMPONENT, NSSConfigurationOptions.PORT, Integer.TYPE, + NSSConfigurationOptions.DEFAULT_PORT); + + return port; + + } catch (ConfigurationException e) { + + throw new RuntimeException(e); + + } + + } + + @Override + public RunState getRunState() { + + return server.getRunState(); + + } + + /** + * Return this quorum member, appropriately cast. + * + * @return The quorum member -or- <code>null</code> if the quorum is not + * running. + */ + protected HAQuorumService<HAGlue, HAJournal> getQuorumService() { + + // This quorum member. + @SuppressWarnings("unchecked") + final HAQuorumService<HAGlue, HAJournal> quorumService = (HAQuorumService<HAGlue, HAJournal>) getQuorum() + .getClient(); + + return quorumService; + + } + + @Override + public String getExtendedRunState() { + + final HAJournalServer server = getHAJournalServer(); + + // This quorum member. + final HAQuorumService<HAGlue, HAJournal> quorumService = getQuorumService(); + + final RunStateEnum innerRunState = (quorumService == null ? null + : quorumService.getRunStateEnum()); + + final HAJournal journal = HAJournal.this; + + final String innerRunStateStr = (innerRunState == null ? "N/A" + : (innerRunState.name() + ((innerRunState == RunStateEnum.Resync && journal != null) ? (" @ " + journal + .getRootBlockView().getCommitCounter()) : ""))); + + return "{server=" + server.getRunState() + ", quorumService=" + + innerRunStateStr + "}"; + + } + } - + } Modified: branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-29 20:15:51 UTC (rev 7091) +++ branches/READ_CACHE/bigdata-jini/src/java/com/bigdata/journal/jini/ha/HAJournalServer.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -28,11 +28,12 @@ import java.io.FileNotFoundException; import java.io.FilenameFilter; import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.net.InetSocketAddress; import java.net.URL; import java.nio.ByteBuffer; import java.rmi.Remote; -import java.rmi.RemoteException; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -54,7 +55,6 @@ import net.jini.core.lookup.ServiceRegistrar; import org.apache.log4j.Logger; -import org.apache.log4j.MDC; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.apache.zookeeper.data.ACL; @@ -62,11 +62,9 @@ import com.bigdata.concurrent.FutureTaskMon; import com.bigdata.ha.HAGlue; -import com.bigdata.ha.HAGlueDelegate; import com.bigdata.ha.HAPipelineGlue; import com.bigdata.ha.QuorumService; import com.bigdata.ha.QuorumServiceBase; -import com.bigdata.ha.RunState; import com.bigdata.ha.halog.HALogWriter; import com.bigdata.ha.halog.IHALogReader; import com.bigdata.ha.msg.HALogRequest; @@ -102,8 +100,6 @@ import com.bigdata.rwstore.RWStore; import com.bigdata.service.AbstractHATransactionService; import com.bigdata.service.jini.FakeLifeCycle; -import com.bigdata.service.jini.RemoteAdministrable; -import com.bigdata.service.jini.RemoteDestroyAdmin; import com.bigdata.util.InnerCause; import com.bigdata.util.concurrent.LatchedExecutor; import com.bigdata.util.concurrent.MonitoredFutureTask; @@ -299,6 +295,13 @@ IRestorePolicy DEFAULT_RESTORE_POLICY = new DefaultRestorePolicy(); + /** + * Permit override of the {@link HAJournal} implementation class. + */ + String HA_JOURNAL_CLASS = "HAJournalClass"; + + String DEFAULT_HA_JOURNAL_CLASS = HAJournal.class.getName(); + } /** @@ -384,7 +387,7 @@ * Enum of the run states. The states are labeled by the goal of the run * state. */ - private enum RunStateEnum { + static enum RunStateEnum { Restore, // apply local HALog files GT current commit point. SeekConsensus, // seek consensus. RunMet, // run while joined with met quorum. @@ -553,7 +556,7 @@ } // The HAJournal. - this.journal = new HAJournal(this, config, quorum); + this.journal = newHAJournal(this, config, quorum); } @@ -564,16 +567,94 @@ // our external interface. haGlueService = journal.newHAGlue(serviceUUID); - // wrap the external interface, exposing administrative functions. - final AdministrableHAGlueService administrableService = new AdministrableHAGlueService( - this, haGlueService); +// // wrap the external interface, exposing administrative functions. +// final AdministrableHAGlueService administrableService = new AdministrableHAGlueService( +// this, haGlueService); +// +// // return that wrapped interface. +// return administrableService; - // return that wrapped interface. - return administrableService; + /* + * Return that object. This will get proxied. If we wrap it with a + * delegation pattern here, then RMI methods on a subclass of + * HAGlueService will not be visible on the exported proxy. + */ + return haGlueService; } /** + * Permit override of the {@link HAJournal} implementation class. + * + * @throws ConfigurationException + */ + private HAJournal newHAJournal(final HAJournalServer server, + final Configuration config, + final Quorum<HAGlue, QuorumService<HAGlue>> quorum) + throws ConfigurationException { + + final String className = (String) config.getEntry( + ConfigurationOptions.COMPONENT, + ConfigurationOptions.HA_JOURNAL_CLASS, String.class, + ConfigurationOptions.DEFAULT_HA_JOURNAL_CLASS); + + try { + @SuppressWarnings("unchecked") + final Class<HAJournal> cls = (Class<HAJournal>) Class + .forName(className); + + if (!HAJournal.class.isAssignableFrom(cls)) { + + throw new ConfigurationException("Invalid option: " + + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + + className + ":: Class does not extend " + + HAJournal.class); + + } + + final Constructor<HAJournal> ctor = cls.getConstructor(new Class[] { + HAJournalServer.class, Configuration.class, Quorum.class }); + + final HAJournal jnl = ctor.newInstance(new Object[] { server, + config, quorum }); + + return jnl; + + } catch (ClassNotFoundException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } catch (NoSuchMethodException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } catch (InstantiationException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } catch (IllegalAccessException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } catch (IllegalArgumentException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } catch (InvocationTargetException e) { + + throw new ConfigurationException( + ConfigurationOptions.HA_JOURNAL_CLASS + "=" + className, e); + + } + + } + + /** * {@inheritDoc} * <p> * Overridden to handle initialization that must wait until the @@ -798,6 +879,15 @@ private final AtomicReference<RunStateEnum> runStateRef = new AtomicReference<RunStateEnum>( null/* none */); + /* + * Exposed to HAJournal.HAGlueService. + */ + protected RunStateEnum getRunStateEnum() { + + return runStateRef.get(); + + } + protected void setRunState(final RunStateEnum runState) { if (runStateRef.get() == RunStateEnum.Shutdown) { @@ -3712,219 +3802,154 @@ } - /** - * Adds jini administration interfaces to the basic {@link HAGlue} interface - * exposed by the {@link HAJournal}. - * - * @see HAJournal.HAGlueService - * - * @author <a href="mailto:tho...@us...">Bryan - * Thompson</a> - */ - public static class AdministrableHAGlueService extends HAGlueDelegate - implements RemoteAdministrable, RemoteDestroyAdmin { - - final protected HAJournalServer server; - - public AdministrableHAGlueService(final HAJournalServer server, - final HAGlue service) { - - super(service); - - this.server = server; - - } - - public Object getAdmin() throws RemoteException { - - if (log.isInfoEnabled()) - log.info("serviceID=" + server.getServiceID()); - - return server.proxy; - - } - -// /** -// * Sets up the {@link MDC} logging context. You should do this on every -// * client facing point of entry and then call -// * {@link #clearLoggingContext()} in a <code>finally</code> clause. You -// * can extend this method to add additional context. -// * <p> -// * This implementation adds the following parameters to the {@link MDC}. -// * <dl> -// * <dt>serviceName</dt> -// * <dd>The serviceName is typically a configuration property for the -// * service. This datum can be injected into log messages using -// * <em>%X{serviceName}</em> in your log4j pattern layout.</dd> -// * <dt>serviceUUID</dt> -// * <dd>The serviceUUID is, in general, assigned asynchronously by the -// * service registrar. Once the serviceUUID becomes available it will be -// * added to the {@link MDC}. This datum can be injected into log -// * messages using <em>%X{serviceUUID}</em> in your log4j pattern layout. -// * </dd> -// * <dt>hostname</dt> -// * <dd>The hostname statically determined. This datum can be injected -// * into log messages using <em>%X{hostname}</em> in your log4j pattern -// * layout.</dd> -// * <dt>clientname -// * <dt> -// * <dd>The hostname or IP address of the client making the request.</dd> -// * </dl> -// * Note: {@link InetAddress#getHostName()} is used. This method makes a -// * one-time best effort attempt to resolve the host name from the -// * {@link InetAddress}. -// */ -// private void setupLoggingContext() { +// /** +// * Adds jini administration interfaces to the basic {@link HAGlue} interface +// * exposed by the {@link HAJournal}. +// * +// * @see HAJournal.HAGlueService +// * +// * @author <a href="mailto:tho...@us...">Bryan +// * Thompson</a> +// */ +// public static class AdministrableHAGlueService extends HAGlueDelegate +// implements RemoteAdministrable, RemoteDestroyAdmin { // -// try { +// final protected HAJournalServer server; // -// // Note: This _is_ a local method call. -// final ServiceID serviceUUID = server.getServiceID(); +// public AdministrableHAGlueService(final HAJournalServer server, +// final HAGlue service) { // -// // Will be null until assigned by the service registrar. +// super(service); // -// if (serviceUUID != null) { -// -// MDC.put("serviceUUID", serviceUUID); -// -// } -// -// MDC.put("serviceName", server.getServiceName()); -// -// MDC.put("hostname", server.getHostName()); -// -// try { -// -// final InetAddress clientAddr = ((ClientHost) ServerContext -// .getServerContextElement(ClientHost.class)) -// .getClientHost(); -// -// MDC.put("clientname", clientAddr.getHostName()); -// -// } catch (ServerNotActiveException e) { -// -// /* -// * This exception gets thrown if the client has made a -// * direct (vs RMI) call so we just ignore it. -// */ -// -// } -// -// } catch (Throwable t) { -// -// /* -// * Ignore. -// */ -// -// } -// +// this.server = server; +// // } - - /** - * Clear the logging context. - */ - protected void clearLoggingContext() { - - MDC.remove("serviceName"); - - MDC.remove("serviceUUID"); - - MDC.remove("hostname"); - - MDC.remove("clientname"); - - } - - /* - * DestroyAdmin - */ - - @Override - public void destroy() { - - server.runShutdown(true/* destroy */); - - } - - @Override - public void shutdown() { - - server.runShutdown(false/* destroy */); - - } - - @Override - public void shutdownNow() { - - server.runShutdown(false/* destroy */); - - } - -// /** -// * Extends the base behavior to return a {@link Name} of the service -// * from the {@link Configuration}. If no name was specified in the -// * {@link Configuration} then the value returned by the base class is -// * returned instead. -// */ -// @Override -// public String getServiceName() { +// +//// /** +//// * Returns an object that implements whatever administration interfaces +//// * are appropriate for the particular service. +//// * +//// * @return an object that implements whatever administration interfaces +//// * are appropriate for the particular service. +//// */ +//// public Object getAdmin() throws RemoteException { +//// +//// if (log.isInfoEnabled()) +//// log.info("serviceID=" + server.getServiceID()); +//// +//// return server.proxy; +//// +//// } +// +//// /** +//// * Sets up the {@link MDC} logging context. You should do this on every +//// * client facing point of entry and then call +//// * {@link #clearLoggingContext()} in a <code>finally</code> clause. You +//// * can extend this method to add additional context. +//// * <p> +//// * This implementation adds the following parameters to the {@link MDC}. +//// * <dl> +//// * <dt>serviceName</dt> +//// * <dd>The serviceName is typically a configuration property for the +//// * service. This datum can be injected into log messages using +//// * <em>%X{serviceName}</em> in your log4j pattern layout.</dd> +//// * <dt>serviceUUID</dt> +//// * <dd>The serviceUUID is, in general, assigned asynchronously by the +//// * service registrar. Once the serviceUUID becomes available it will be +//// * added to the {@link MDC}. This datum can be injected into log +//// * messages using <em>%X{serviceUUID}</em> in your log4j pattern layout. +//// * </dd> +//// * <dt>hostname</dt> +//// * <dd>The hostname statically determined. This datum can be injected +//// * into log messages using <em>%X{hostname}</em> in your log4j pattern +//// * layout.</dd> +//// * <dt>clientname +//// * <dt> +//// * <dd>The hostname or IP address of the client making the request.</dd> +//// * </dl> +//// * Note: {@link InetAddress#getHostName()} is used. This method makes a +//// * one-time best effort attempt to resolve the host name from the +//// * {@link InetAddress}. +//// */ +//// private void setupLoggingContext() { +//// +//// try { +//// +//// // Note: This _is_ a local method call. +//// final ServiceID serviceUUID = server.getServiceID(); +//// +//// // Will be null until assigned by the service registrar. +//// +//// if (serviceUUID != null) { +//// +//// MDC.put("serviceUUID", serviceUUID); +//// +//// } +//// +//// MDC.put("serviceName", server.getServiceName()); +//// +//// MDC.put("hostname", server.getHostName()); +//// +//// try { +//// +//// final InetAddress clientAddr = ((ClientHost) ServerContext +//// .getServerContextElement(ClientHost.class)) +//// .getClientHost(); +//// +//// MDC.put("clientname", clientAddr.getHostName()); +//// +//// } catch (ServerNotActiveException e) { +//// +//// /* +//// * This exception gets thrown if the client has made a +//// * direct (vs RMI) call so we just ignore it. +//// */ +//// +//// } +//// +//// } catch (Throwable t) { +//// +//// /* +//// * Ignore. +//// */ +//// +//// } +//// +//// } +//// +//// /** +//// * Clear the logging context. +//// */ +//// protected void clearLoggingContext() { +//// +//// MDC.remove("serviceName"); +//// +//// MDC.remove("serviceUUID"); +//// +//// MDC.remove("hostname"); +//// +//// MDC.remove("clientname"); +//// +//// } // -// String s = server.getServiceName(); +//// /** +//// * Extends the base behavior to return a {@link Name} of the service +//// * from the {@link Configuration}. If no name was specified in the +//// * {@link Configuration} then the value returned by the base class is +//// * returned instead. +//// */ +//// @Override +//// public String getServiceName() { +//// +//// String s = server.getServiceName(); +//// +//// if (s == null) +//// s = super.getServiceName(); +//// +//// return s; +//// +//// } // -// if (s == null) -// s = super.getServiceName(); -// -// return s; -// -// } +// } - @Override - public int getNSSPort() { - - final String COMPONENT = NSSConfigurationOptions.COMPONENT; - - try { - - final Integer port = (Integer) server.config.getEntry( - COMPONENT, NSSConfigurationOptions.PORT, Integer.TYPE, - NSSConfigurationOptions.DEFAULT_PORT); - - return port; - - } catch (ConfigurationException e) { - - throw new RuntimeException(e); - - } - - } - - @Override - public RunState getRunState() { - - return server.getRunState(); - - } - - @Override - public String getExtendedRunState() { - - final HAQuorumService<HAGlue, HAJournal> quorumService = server.quorumService; - - final RunStateEnum innerRunState = (quorumService == null ? null - : quorumService.runStateRef.get()); - - final HAJournal journal = server.journal; - - final String innerRunStateStr = (innerRunState == null ? "N/A" - : (innerRunState.name() + ((innerRunState == RunStateEnum.Resync && journal != null) ? (" @ " + journal - .getRootBlockView().getCommitCounter()) : ""))); - - return "{server=" + server.getRunState() + ", quorumService=" - + innerRunStateStr + "}"; - - } - - } - } Added: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/MyHAJournal.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/MyHAJournal.java (rev 0) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/MyHAJournal.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -0,0 +1,101 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 Oct 31, 2012 + */ +package com.bigdata.journal.jini.ha; + +import java.io.IOException; +import java.rmi.Remote; +import java.util.UUID; + +import org.apache.log4j.Logger; + +import net.jini.config.Configuration; +import net.jini.config.ConfigurationException; + +import com.bigdata.ha.HAGlue; +import com.bigdata.ha.QuorumService; +import com.bigdata.quorum.Quorum; + +/** + * Class extends {@link HAJournal}. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public class MyHAJournal extends HAJournal { + + private static final Logger log = Logger.getLogger(HAJournal.class); + + public MyHAJournal(final HAJournalServer server, + final Configuration config, + final Quorum<HAGlue, QuorumService<HAGlue>> quorum) + throws ConfigurationException, IOException { + + super(server, config, quorum); + + } + + @Override + protected HAGlue newHAGlue(final UUID serviceId) { + +// return super.newHAGlue(serviceId); + return new MyHAGlueService(serviceId); + + } + + /** + * A {@link Remote} interface for new methods published by the service. + */ + public static interface MyHAGlue extends HAGlue { + + public void helloWorld() throws IOException; + + } + + /** + * Class extends the public RMI interface of the {@link HAJournal}. + * <p> + * Note: Any new RMI methods must be (a) declared on an interface; and (b) + * must throw {@link IOException}. + */ + protected class MyHAGlueService extends HAJournal.HAGlueService implements + MyHAGlue { + + protected MyHAGlueService(final UUID serviceId) { + + super(serviceId); + + } + + @Override + public void helloWorld() throws IOException { + + log.warn("Hello world!"); + + } + + } + +} Modified: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java 2013-04-29 20:15:51 UTC (rev 7091) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestAll.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -84,6 +84,9 @@ // Basic tests for a single HAJournalServer (quorum does not meet) suite.addTestSuite(TestHAJournalServer.class); + // Verify ability to override the HAJournal implementation class. + suite.addTestSuite(TestHAJournalServerOverride.class); + // HA2 test suite (k=3, but only 2 services are running). suite.addTestSuite(TestHA2JournalServer.class); Added: branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServerOverride.java =================================================================== --- branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServerOverride.java (rev 0) +++ branches/READ_CACHE/bigdata-jini/src/test/com/bigdata/journal/jini/ha/TestHAJournalServerOverride.java 2013-04-30 13:04:12 UTC (rev 7092) @@ -0,0 +1,126 @@ +/** + +Copyright (C) SYSTAP, LLC 2006-2007. 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 Oct 31, 2012 + */ +package com.bigdata.journal.jini.ha; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import net.jini.config.Configuration; + +import com.bigdata.ha.HAGlue; +import com.bigdata.ha.HAStatusEnum; +import com.bigdata.journal.jini.ha.MyHAJournal.MyHAGlue; + +/** + * Unit test of the ability to override the {@link HAJournal} implementation + * class. + * + * @author <a href="mailto:tho...@us...">Bryan Thompson</a> + */ +public class TestHAJournalServerOverride extends AbstractHA3JournalServerTestCase { + + public TestHAJournalServerOverride() { + } + + public TestHAJournalServerOverride(final String name) { + super(name); + } + + /** + * {@inheritDoc} + * <p> + * Note: This overrides some {@link Configuration} values for the + * {@link HAJournalServer} in order to establish conditions suitable for + * testing the {@link ISnapshotPolicy} and {@link IRestorePolicy}. + */ + @Override + protected String[] getOverrides() { + + return new String[]{ + "com.bigdata.journal.jini.ha.HAJournalServer.restorePolicy=new com.bigdata.journal.jini.ha.DefaultRestorePolicy(0L,1,0)", + "com.bigdata.journal.jini.ha.HAJournalServer.snapshotPolicy=new com.bigdata.journal.jini.ha.NoSnapshotPolicy()", + "com.bigdata.journal.jini.ha.HAJournalServer.HAJournalClass=\"com.bigdata.journal.jini.ha.MyHAJournal\"" + }; + + } + + /** + * One service starts, quorum does not meet (replication factor is 3). This + * also serves to verify the <code>HAJournal-A.config</code> file. + */ + public void testStartA() throws Exception { + + final HAGlue serverA = startA(); + + try { + + quorum.awaitQuorum(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + + fail("Not expecting quorum meet"); + + } catch (TimeoutException ex) { + + // ignore. + + } + + // Service is not met in any role around a quorum. + try { + serverA.awaitHAReady(awaitQuorumTimeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException ex) { + // Ignore expected exception. + } + + // Verify can access the REST API "status" page. + doNSSStatusRequest(serverA); + + // Verify self-reports as NotReady. + awaitHAStatus(serverA, HAStatusEnum.NotReady); + + // Verify that service self-reports as NotReady via the REST API. + assertEquals(HAStatusEnum.NotReady, getNSSHAStatus(serverA)); + + // Verify can not read on service. + assertReadRejected(serverA); + + // Verify can not write on service. + assertWriteRejected(serverA); + + assertTrue(getHAJournalFileA().exists()); + assertTrue(getHALogDirA().exists()); + assertTrue(getSnapshotDirA().exists()); + + // Verify that we can invoke extension methods on the service. + ((MyHAGlue)serverA).helloWorld(); + +// serverA.enterErrorState().get(); +// +// Thread.sleep(10000/*ms*/); + + } + +} This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |