Author: jbellis
Date: Thu Apr 30 16:09:38 2009
New Revision: 770307
URL: http://svn.apache.org/viewvc?rev=770307&view=rev
Log:
add string de/serialize code to TokenFactory for ApplicationState's benefit.
patch by jbellis; tested by nk11 and MarkR42 for CASSANDRA-114
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/HttpRequestVerbHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java?rev=770307&r1=770306&r2=770307&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
Thu Apr 30 16:09:38 2009
@@ -99,6 +99,11 @@
}
}
+ public String toString(Token<String> stringToken)
+ {
+ return stringToken.token;
+ }
+
public Token<String> fromString(String string)
{
return new StringToken(string);
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java?rev=770307&r1=770306&r2=770307&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
Thu Apr 30 16:09:38 2009
@@ -92,6 +92,11 @@
return new BigIntegerToken(new BigInteger(bytes));
}
+ public String toString(Token<BigInteger> bigIntegerToken)
+ {
+ return bigIntegerToken.token.toString();
+ }
+
public Token<BigInteger> fromString(String string)
{
return new BigIntegerToken(new BigInteger(string));
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java?rev=770307&r1=770306&r2=770307&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/dht/Token.java Thu
Apr 30 16:09:38 2009
@@ -52,7 +52,8 @@
{
public abstract byte[] toByteArray(Token<T> token);
public abstract Token<T> fromByteArray(byte[] bytes);
- public abstract Token<T> fromString(String string);
+ public abstract String toString(Token<T> token); // serialize as
string, not necessarily human-readable
+ public abstract Token<T> fromString(String string); // deserialize
}
public static class TokenSerializer implements ICompactSerializer<Token>
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/HttpRequestVerbHandler.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/HttpRequestVerbHandler.java?rev=770307&r1=770306&r2=770307&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/HttpRequestVerbHandler.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/HttpRequestVerbHandler.java
Thu Apr 30 16:09:38 2009
@@ -224,7 +224,7 @@
{
HTMLFormatter formatter = new HTMLFormatter();
- formatter.appendLine("Token: " + storageService_.getToken());
+ formatter.appendLine(storageService_.getToken());
RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
formatter.appendLine("Up time (in seconds): " +
(runtimeMxBean.getUptime()/1000));
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=770307&r1=770306&r2=770307&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Thu Apr 30 16:09:38 2009
@@ -468,7 +468,8 @@
Gossiper.instance().start(udpAddr_, storageMetadata_.getGeneration());
/* Make sure this token gets gossiped around. */
tokenMetadata_.update(storageMetadata_.getStorageId(),
StorageService.tcpAddr_);
- Gossiper.instance().addApplicationState(StorageService.nodeId_, new
ApplicationState(storageMetadata_.getStorageId().toString()));
+ ApplicationState state = new
ApplicationState(StorageService.getPartitioner().getTokenFactory().toString(storageMetadata_.getStorageId()));
+ Gossiper.instance().addApplicationState(StorageService.nodeId_, state);
}
public void killMe() throws Throwable
@@ -726,7 +727,8 @@
/* Get the old token. This needs to be removed. */
tokenMetadata_.update(token, StorageService.tcpAddr_);
/* Gossip this new token for the local storage instance */
- Gossiper.instance().addApplicationState(StorageService.nodeId_, new
ApplicationState(token.toString()));
+ ApplicationState state = new
ApplicationState(StorageService.getPartitioner().getTokenFactory().toString(token));
+ Gossiper.instance().addApplicationState(StorageService.nodeId_, state);
}
/*