Author: slebresne Date: Fri Oct 28 17:22:53 2011 New Revision: 1190461 URL: http://svn.apache.org/viewvc?rev=1190461&view=rev Log: Cleanup usage of StorageService.setMode() patch by slebresne; reviewed by thobbs for CASSANDRA-3388
Modified: cassandra/branches/cassandra-1.0/CHANGES.txt cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/StorageService.java Modified: cassandra/branches/cassandra-1.0/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1190461&r1=1190460&r2=1190461&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/CHANGES.txt (original) +++ cassandra/branches/cassandra-1.0/CHANGES.txt Fri Oct 28 17:22:53 2011 @@ -1,5 +1,6 @@ 1.0.2 * "defragment" rows for name-based queries under STCS (CASSANDRA-2503) + * cleanup usage of StorageService.setMode() (CASANDRA-3388) 1.0.1 Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/StorageService.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/StorageService.java?rev=1190461&r1=1190460&r2=1190461&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/StorageService.java (original) +++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/service/StorageService.java Fri Oct 28 17:22:53 2011 @@ -201,7 +201,9 @@ public class StorageService implements I private boolean isClientMode; private boolean initialized; private volatile boolean joined = false; - private String operationMode; + + private static enum Mode { NORMAL, CLIENT, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED } + private Mode operationMode; private volatile boolean efficientCrossDCWrites; private MigrationManager migrationManager = new MigrationManager(); @@ -224,7 +226,7 @@ public class StorageService implements I SystemTable.updateToken(token); tokenMetadata_.updateNormalToken(token, FBUtilities.getBroadcastAddress()); Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.normal(getLocalToken())); - setMode("Normal", false); + setMode(Mode.NORMAL, false); } public StorageService() @@ -359,7 +361,7 @@ public class StorageService implements I initialized = true; isClientMode = true; logger_.info("Starting up client gossip"); - setMode("Client", false); + setMode(Mode.CLIENT, false); Gossiper.instance.register(this); Gossiper.instance.start((int)(System.currentTimeMillis() / 1000)); // needed for node-ring gathering. MessagingService.instance().listen(FBUtilities.getLocalAddress()); @@ -500,7 +502,7 @@ public class StorageService implements I || DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()) || !Schema.instance.getNonSystemTables().isEmpty())) { - setMode("Joining: waiting for ring and schema information", true); + setMode(Mode.JOINING, "waiting for ring and schema information", true); try { Thread.sleep(delay); @@ -519,7 +521,7 @@ public class StorageService implements I String s = "This node is already a member of the token ring; bootstrap aborted. (If replacing a dead node, remove the old one from the ring first.)"; throw new UnsupportedOperationException(s); } - setMode("Joining: getting bootstrap token", true); + setMode(Mode.JOINING, "getting bootstrap token", true); token = BootStrapper.getBootstrapToken(tokenMetadata_, LoadBroadcaster.instance.getLoadInfo()); } else @@ -539,7 +541,7 @@ public class StorageService implements I current = tokenMetadata_.getEndpoint(token); if (null != current && Gossiper.instance.getEndpointStateForEndpoint(current).getUpdateTimestamp() > (System.currentTimeMillis() - delay)) throw new UnsupportedOperationException("Cannnot replace a token for a Live node... "); - setMode("Joining: Replacing a node with token: " + token, true); + setMode(Mode.JOINING, "Replacing a node with token: " + token, true); } bootstrap(token); @@ -597,17 +599,24 @@ public class StorageService implements I return DatabaseDescriptor.getCompactionThroughputMbPerSec(); } - public void setCompactionThroughputMbPerSec(int value) { + public void setCompactionThroughputMbPerSec(int value) + { DatabaseDescriptor.setCompactionThroughputMbPerSec(value); } - private void setMode(String m, boolean log) + private void setMode(Mode m, boolean log) + { + setMode(m, null, log); + } + + private void setMode(Mode m, String msg, boolean log) { operationMode = m; + String logMsg = msg == null ? m.toString() : String.format("%s: %s", m, msg); if (log) - logger_.info(m); + logger_.info(logMsg); else - logger_.debug(m); + logger_.debug(logMsg); } private void bootstrap(Token token) throws IOException @@ -618,7 +627,7 @@ public class StorageService implements I { // if not an existing token then bootstrap Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.bootstrapping(token)); - setMode("Joining: sleeping " + RING_DELAY + " ms for pending range setup", true); + setMode(Mode.JOINING, "sleeping " + RING_DELAY + " ms for pending range setup", true); try { Thread.sleep(RING_DELAY); @@ -633,7 +642,7 @@ public class StorageService implements I // Dont set any state for the node which is bootstrapping the existing token... tokenMetadata_.updateNormalToken(token, FBUtilities.getBroadcastAddress()); } - setMode("Starting to bootstrap...", true); + setMode(Mode.JOINING, "Starting to bootstrap...", true); new BootStrapper(FBUtilities.getBroadcastAddress(), token, tokenMetadata_).bootstrap(); // handles token update } @@ -1979,7 +1988,7 @@ public class StorageService implements I if (logger_.isDebugEnabled()) logger_.debug("DECOMMISSIONING"); startLeaving(); - setMode("Leaving: sleeping " + RING_DELAY + " ms for pending range setup", true); + setMode(Mode.LEAVING, "sleeping " + RING_DELAY + " ms for pending range setup", true); Thread.sleep(RING_DELAY); Runnable finishLeaving = new Runnable() @@ -1989,7 +1998,7 @@ public class StorageService implements I Gossiper.instance.stop(); MessagingService.instance().shutdown(); StageManager.shutdownNow(); - setMode("Decommissioned", true); + setMode(Mode.DECOMMISSIONED, true); // let op be responsible for killing the process } }; @@ -2028,7 +2037,7 @@ public class StorageService implements I rangesToStream.put(table, rangesMM); } - setMode("Leaving: streaming data to other nodes", true); + setMode(Mode.LEAVING, "streaming data to other nodes", true); CountDownLatch latch = streamRanges(rangesToStream); @@ -2162,7 +2171,7 @@ public class StorageService implements I throw new RuntimeException("Sleep interrupted " + e.getMessage()); } - setMode("Moving: fetching new ranges and streaming old ranges", true); + setMode(Mode.MOVING, "fetching new ranges and streaming old ranges", true); if (logger_.isDebugEnabled()) logger_.debug("[Move->STREAMING] Work Map: " + rangesToStreamByTable); @@ -2366,7 +2375,7 @@ public class StorageService implements I public String getOperationMode() { - return operationMode; + return operationMode.toString(); } public String getDrainProgress() @@ -2383,21 +2392,21 @@ public class StorageService implements I logger_.warn("Cannot drain node (did it already happen?)"); return; } - setMode("Starting drain process", true); + setMode(Mode.DRAINING, "starting drain process", true); optionalTasks.shutdown(); Gossiper.instance.stop(); - setMode("Draining: shutting down MessageService", false); + setMode(Mode.DRAINING, "shutting down MessageService", false); MessagingService.instance().shutdown(); - setMode("Draining: waiting for streaming", false); + setMode(Mode.DRAINING, "waiting for streaming", false); MessagingService.instance().waitForStreaming(); - setMode("Draining: clearing mutation stage", false); + setMode(Mode.DRAINING, "clearing mutation stage", false); mutationStage.shutdown(); mutationStage.awaitTermination(3600, TimeUnit.SECONDS); StorageProxy.instance.verifyNoHintsInProgress(); - setMode("Draining: flushing column families", false); + setMode(Mode.DRAINING, "flushing column families", false); List<ColumnFamilyStore> cfses = new ArrayList<ColumnFamilyStore>(); for (String tableName : Schema.instance.getNonSystemTables()) { @@ -2421,7 +2430,7 @@ public class StorageService implements I if (!tasks.awaitTermination(1, TimeUnit.MINUTES)) logger_.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown"); - setMode("Node is drained", true); + setMode(Mode.DRAINED, true); } // Never ever do this at home. Used by tests.