Author: jbellis
Date: Mon Apr 20 20:05:20 2009
New Revision: 766840
URL: http://svn.apache.org/viewvc?rev=766840&view=rev
Log:
rename ReadResponseMessage -> ReadResponse
patch by jbellis; reviewed by Eric Evans for #88
Added:
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponse.java
- copied, changed from r766839,
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponseMessage.java
Removed:
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponseMessage.java
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
incubator/cassandra/trunk/src/org/apache/cassandra/service/ConsistencyManager.java
incubator/cassandra/trunk/src/org/apache/cassandra/service/ReadResponseResolver.java
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java
incubator/cassandra/trunk/src/org/apache/cassandra/test/DataImporter.java
Copied: incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponse.java
(from r766839,
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponseMessage.java)
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponse.java?p2=incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponse.java&p1=incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponseMessage.java&r1=766839&r2=766840&rev=766840&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponseMessage.java
(original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadResponse.java Mon
Apr 20 20:05:20 2009
@@ -35,25 +35,25 @@
* The table name is needed so that we can use it to create repairs.
* Author : Avinash Lakshman ( [email protected]) & Prashant Malik (
[email protected] )
*/
-public class ReadResponseMessage implements Serializable
+public class ReadResponse implements Serializable
{
-private static ICompactSerializer<ReadResponseMessage> serializer_;
-
+private static ICompactSerializer<ReadResponse> serializer_;
+
static
{
- serializer_ = new ReadResponseMessageSerializer();
+ serializer_ = new ReadResponseSerializer();
}
- public static ICompactSerializer<ReadResponseMessage> serializer()
+ public static ICompactSerializer<ReadResponse> serializer()
{
return serializer_;
}
- public static Message makeReadResponseMessage(ReadResponseMessage
readResponseMessage) throws IOException
+ public static Message makeReadResponseMessage(ReadResponse
readResponse) throws IOException
{
ByteArrayOutputStream bos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream( bos );
- ReadResponseMessage.serializer().serialize(readResponseMessage, dos);
+ ReadResponse.serializer().serialize(readResponse, dos);
Message message = new
Message(StorageService.getLocalStorageEndPoint(),
MessagingService.responseStage_, MessagingService.responseVerbHandler_, new
Object[]{bos.toByteArray()});
return message;
}
@@ -63,13 +63,13 @@
private byte[] digest_ = new byte[0];
private boolean isDigestQuery_ = false;
- public ReadResponseMessage(String table, byte[] digest )
+ public ReadResponse(String table, byte[] digest )
{
table_ = table;
digest_= digest;
}
- public ReadResponseMessage(String table, Row row)
+ public ReadResponse(String table, Row row)
{
table_ = table;
row_ = row;
@@ -101,9 +101,9 @@
}
}
-class ReadResponseMessageSerializer implements
ICompactSerializer<ReadResponseMessage>
+class ReadResponseSerializer implements ICompactSerializer<ReadResponse>
{
- public void serialize(ReadResponseMessage rm, DataOutputStream dos)
throws IOException
+ public void serialize(ReadResponse rm, DataOutputStream dos) throws
IOException
{
dos.writeUTF(rm.table());
dos.writeInt(rm.digest().length);
@@ -116,7 +116,7 @@
}
}
- public ReadResponseMessage deserialize(DataInputStream dis) throws
IOException
+ public ReadResponse deserialize(DataInputStream dis) throws IOException
{
String table = dis.readUTF();
int digestSize = dis.readInt();
@@ -130,14 +130,14 @@
row = Row.serializer().deserialize(dis);
}
- ReadResponseMessage rmsg = null;
+ ReadResponse rmsg = null;
if( isDigest )
{
- rmsg = new ReadResponseMessage(table, digest);
+ rmsg = new ReadResponse(table, digest);
}
else
{
- rmsg = new ReadResponseMessage(table, row);
+ rmsg = new ReadResponse(table, row);
}
rmsg.setIsDigestQuery(isDigest);
return rmsg;
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java?rev=766840&r1=766839&r2=766840&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
(original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
Mon Apr 20 20:05:20 2009
@@ -94,21 +94,21 @@
}
logger_.info("getRow() TIME: " + (System.currentTimeMillis() -
start) + " ms.");
start = System.currentTimeMillis();
- ReadResponseMessage readResponseMessage = null;
+ ReadResponse readResponse = null;
if(readCommand.isDigestQuery())
{
- readResponseMessage = new
ReadResponseMessage(table.getTableName(), row.digest());
+ readResponse = new ReadResponse(table.getTableName(),
row.digest());
}
else
{
- readResponseMessage = new
ReadResponseMessage(table.getTableName(), row);
+ readResponse = new ReadResponse(table.getTableName(), row);
}
- readResponseMessage.setIsDigestQuery(readCommand.isDigestQuery());
+ readResponse.setIsDigestQuery(readCommand.isDigestQuery());
/* serialize the ReadResponseMessage. */
readCtx.bufOut_.reset();
start = System.currentTimeMillis();
- ReadResponseMessage.serializer().serialize(readResponseMessage,
readCtx.bufOut_);
+ ReadResponse.serializer().serialize(readResponse, readCtx.bufOut_);
logger_.info("serialize TIME: " + (System.currentTimeMillis() -
start) + " ms.");
byte[] bytes = new byte[readCtx.bufOut_.getLength()];
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/service/ConsistencyManager.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/ConsistencyManager.java?rev=766840&r1=766839&r2=766840&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/org/apache/cassandra/service/ConsistencyManager.java
(original)
+++
incubator/cassandra/trunk/src/org/apache/cassandra/service/ConsistencyManager.java
Mon Apr 20 20:05:20 2009
@@ -25,7 +25,7 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.ReadResponseMessage;
+import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.Row;
import org.apache.cassandra.io.DataInputBuffer;
import org.apache.cassandra.net.EndPoint;
@@ -70,7 +70,7 @@
bufIn.reset(body, body.length);
try
{
- ReadResponseMessage result =
ReadResponseMessage.serializer().deserialize(bufIn);
+ ReadResponse result =
ReadResponse.serializer().deserialize(bufIn);
byte[] digest = result.digest();
if( !Arrays.equals(row_.digest(), digest) )
{
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/service/ReadResponseResolver.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/ReadResponseResolver.java?rev=766840&r1=766839&r2=766840&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/org/apache/cassandra/service/ReadResponseResolver.java
(original)
+++
incubator/cassandra/trunk/src/org/apache/cassandra/service/ReadResponseResolver.java
Mon Apr 20 20:05:20 2009
@@ -26,7 +26,7 @@
import java.util.Set;
import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ReadResponseMessage;
+import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.Row;
import org.apache.cassandra.db.RowMutation;
import org.apache.cassandra.db.RowMutationMessage;
@@ -80,7 +80,7 @@
try
{
long start = System.currentTimeMillis();
- ReadResponseMessage result =
ReadResponseMessage.serializer().deserialize(bufIn);
+ ReadResponse result =
ReadResponse.serializer().deserialize(bufIn);
logger_.debug( "Response deserialization time : " +
(System.currentTimeMillis() - start) + " ms.");
if(!result.isDigestQuery())
{
@@ -168,7 +168,7 @@
bufIn.reset(body, body.length);
try
{
- ReadResponseMessage result =
ReadResponseMessage.serializer().deserialize(bufIn);
+ ReadResponse result =
ReadResponse.serializer().deserialize(bufIn);
if(!result.isDigestQuery())
{
isDataPresent = true;
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java?rev=766840&r1=766839&r2=766840&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java
(original)
+++
incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java
Mon Apr 20 20:05:20 2009
@@ -31,7 +31,7 @@
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.ReadResponseMessage;
+import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.Row;
import org.apache.cassandra.db.RowMutation;
import org.apache.cassandra.db.Table;
@@ -197,8 +197,8 @@
byte[] body = (byte[])result[0];
DataInputBuffer bufIn = new DataInputBuffer();
bufIn.reset(body, body.length);
- ReadResponseMessage responseMessage =
ReadResponseMessage.serializer().deserialize(bufIn);
- Row row = responseMessage.row();
+ ReadResponse response =
ReadResponse.serializer().deserialize(bufIn);
+ Row row = response.row();
rows.put(row.key(), row);
}
return rows;
@@ -217,8 +217,8 @@
byte[] body = (byte[])result[0];
DataInputBuffer bufIn = new DataInputBuffer();
bufIn.reset(body, body.length);
- ReadResponseMessage responseMessage =
ReadResponseMessage.serializer().deserialize(bufIn);
- row = responseMessage.row();
+ ReadResponse response =
ReadResponse.serializer().deserialize(bufIn);
+ row = response.row();
}
else
{
Modified:
incubator/cassandra/trunk/src/org/apache/cassandra/test/DataImporter.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/test/DataImporter.java?rev=766840&r1=766839&r2=766840&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/test/DataImporter.java
(original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/test/DataImporter.java
Mon Apr 20 20:05:20 2009
@@ -44,7 +44,7 @@
import org.apache.cassandra.db.ColumnFamily;
import org.apache.cassandra.db.IColumn;
import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.ReadResponseMessage;
+import org.apache.cassandra.db.ReadResponse;
import org.apache.cassandra.db.Row;
import org.apache.cassandra.db.RowMutation;
import org.apache.cassandra.db.RowMutationMessage;
@@ -887,8 +887,8 @@
IAsyncResult iar =
MessagingService.getMessagingInstance().sendRR(
message, to_);
Object[] result = iar.get();
- ReadResponseMessage readResponseMessage =
(ReadResponseMessage) result[0];
- Row row = readResponseMessage.row();
+ ReadResponse readResponse = (ReadResponse) result[0];
+ Row row = readResponse.row();
if (row == null) {
logger_.debug("ERROR No row for this key .....:
" + line);
Thread.sleep(1000/requestsPerSecond_, 1000%requestsPerSecond_);