Author: gdusbabek
Date: Wed May 26 18:56:27 2010
New Revision: 948535

URL: http://svn.apache.org/viewvc?rev=948535&view=rev
Log:
rename CompletedFileStatus to FileStatus. patch by stuhood, reviwed by 
gdusbabek. CASSANDRA-1019

Added:
    cassandra/trunk/src/java/org/apache/cassandra/streaming/FileStatus.java
      - copied, changed from r948533, 
cassandra/trunk/src/java/org/apache/cassandra/streaming/CompletedFileStatus.java
Removed:
    
cassandra/trunk/src/java/org/apache/cassandra/streaming/CompletedFileStatus.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/streaming/IStreamComplete.java
    
cassandra/trunk/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
    
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamCompletionHandler.java
    
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamFinishedVerbHandler.java
    cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInManager.java
    
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java

Copied: cassandra/trunk/src/java/org/apache/cassandra/streaming/FileStatus.java 
(from r948533, 
cassandra/trunk/src/java/org/apache/cassandra/streaming/CompletedFileStatus.java)
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/FileStatus.java?p2=cassandra/trunk/src/java/org/apache/cassandra/streaming/FileStatus.java&p1=cassandra/trunk/src/java/org/apache/cassandra/streaming/CompletedFileStatus.java&r1=948533&r2=948535&rev=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/CompletedFileStatus.java
 (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/FileStatus.java Wed 
May 26 18:56:27 2010
@@ -31,9 +31,9 @@ import org.apache.cassandra.net.Message;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
-class CompletedFileStatus
+class FileStatus
 {
-    private static ICompactSerializer<CompletedFileStatus> serializer_;
+    private static ICompactSerializer<FileStatus> serializer_;
 
     public static enum StreamCompletionAction
     {
@@ -43,10 +43,10 @@ class CompletedFileStatus
 
     static
     {
-        serializer_ = new CompletedFileStatusSerializer();
+        serializer_ = new FileStatusSerializer();
     }
 
-    public static ICompactSerializer<CompletedFileStatus> serializer()
+    public static ICompactSerializer<FileStatus> serializer()
     {
         return serializer_;
     }
@@ -55,7 +55,7 @@ class CompletedFileStatus
     private long expectedBytes_;
     private StreamCompletionAction action_;
 
-    public CompletedFileStatus(String file, long expectedBytes)
+    public FileStatus(String file, long expectedBytes)
     {
         file_ = file;
         expectedBytes_ = expectedBytes;
@@ -86,24 +86,24 @@ class CompletedFileStatus
     {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
         DataOutputStream dos = new DataOutputStream( bos );
-        CompletedFileStatus.serializer().serialize(this, dos);
+        FileStatus.serializer().serialize(this, dos);
         return new Message(FBUtilities.getLocalAddress(), "", 
StorageService.Verb.STREAM_FINISHED, bos.toByteArray());
     }
 
-    private static class CompletedFileStatusSerializer implements 
ICompactSerializer<CompletedFileStatus>
+    private static class FileStatusSerializer implements 
ICompactSerializer<FileStatus>
     {
-        public void serialize(CompletedFileStatus streamStatus, 
DataOutputStream dos) throws IOException
+        public void serialize(FileStatus streamStatus, DataOutputStream dos) 
throws IOException
         {
             dos.writeUTF(streamStatus.getFile());
             dos.writeLong(streamStatus.getExpectedBytes());
             dos.writeInt(streamStatus.getAction().ordinal());
         }
 
-        public CompletedFileStatus deserialize(DataInputStream dis) throws 
IOException
+        public FileStatus deserialize(DataInputStream dis) throws IOException
         {
             String targetFile = dis.readUTF();
             long expectedBytes = dis.readLong();
-            CompletedFileStatus streamStatus = new 
CompletedFileStatus(targetFile, expectedBytes);
+            FileStatus streamStatus = new FileStatus(targetFile, 
expectedBytes);
 
             int ordinal = dis.readInt();
             if ( ordinal == StreamCompletionAction.DELETE.ordinal() )

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IStreamComplete.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/IStreamComplete.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IStreamComplete.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IStreamComplete.java 
Wed May 26 18:56:27 2010
@@ -28,5 +28,5 @@ interface IStreamComplete
      * This callback if registered with the StreamContextManager is 
      * called when the stream from a host is completely handled. 
     */
-    public void onStreamCompletion(InetAddress from, PendingFile pendingFile, 
CompletedFileStatus streamStatus) throws IOException;
+    public void onStreamCompletion(InetAddress from, PendingFile pendingFile, 
FileStatus streamStatus) throws IOException;
 }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
 Wed May 26 18:56:27 2010
@@ -33,7 +33,7 @@ public class IncomingStreamReader
 {
     private static Logger logger = 
LoggerFactory.getLogger(IncomingStreamReader.class);
     private PendingFile pendingFile;
-    private CompletedFileStatus streamStatus;
+    private FileStatus streamStatus;
     private SocketChannel socketChannel;
 
     public IncomingStreamReader(SocketChannel socketChannel)
@@ -68,7 +68,7 @@ public class IncomingStreamReader
         catch (IOException ex)
         {
             /* Ask the source node to re-stream this file. */
-            
streamStatus.setAction(CompletedFileStatus.StreamCompletionAction.STREAM);
+            streamStatus.setAction(FileStatus.StreamCompletionAction.STREAM);
             handleStreamCompletion(remoteAddress.getAddress());
             /* Delete the orphaned file. */
             File file = new File(pendingFile.getFilename());

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamCompletionHandler.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamCompletionHandler.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamCompletionHandler.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamCompletionHandler.java
 Wed May 26 18:56:27 2010
@@ -44,7 +44,7 @@ class StreamCompletionHandler implements
 {
     private static Logger logger = 
LoggerFactory.getLogger(StreamCompletionHandler.class);
 
-    public void onStreamCompletion(InetAddress host, PendingFile pendingFile, 
CompletedFileStatus streamStatus) throws IOException
+    public void onStreamCompletion(InetAddress host, PendingFile pendingFile, 
FileStatus streamStatus) throws IOException
     {
         /* Parse the stream context and the file to the list of SSTables in 
the associated Column Family Store. */
         if (pendingFile.getFilename().contains("-Data.db"))

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamFinishedVerbHandler.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamFinishedVerbHandler.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamFinishedVerbHandler.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamFinishedVerbHandler.java
 Wed May 26 18:56:27 2010
@@ -44,7 +44,7 @@ public class StreamFinishedVerbHandler i
 
         try
         {
-            CompletedFileStatus streamStatus = 
CompletedFileStatus.serializer().deserialize(new DataInputStream(bufIn));
+            FileStatus streamStatus = FileStatus.serializer().deserialize(new 
DataInputStream(bufIn));
 
             switch (streamStatus.getAction())
             {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInManager.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInManager.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInManager.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInManager.java 
Wed May 26 18:56:27 2010
@@ -36,7 +36,7 @@ class StreamInManager
     /* Maintain a stream context per host that is the source of the stream */
     public static final Map<InetAddress, List<PendingFile>> ctxBag_ = new 
Hashtable<InetAddress, List<PendingFile>>();
     /* Maintain in this map the status of the streams that need to be sent 
back to the source */
-    public static final Map<InetAddress, List<CompletedFileStatus>> 
streamStatusBag_ = new Hashtable<InetAddress, List<CompletedFileStatus>>();
+    public static final Map<InetAddress, List<FileStatus>> streamStatusBag_ = 
new Hashtable<InetAddress, List<FileStatus>>();
     /* Maintains a callback handler per endpoint to notify the app that a 
stream from a given endpoint has been handled */
     public static final Map<InetAddress, IStreamComplete> 
streamNotificationHandlers_ = new HashMap<InetAddress, IStreamComplete>();
 
@@ -53,12 +53,12 @@ class StreamInManager
         return pendingFile;
     }
     
-    public synchronized static CompletedFileStatus getStreamStatus(InetAddress 
key)
+    public synchronized static FileStatus getStreamStatus(InetAddress key)
     {
-        List<CompletedFileStatus> status = streamStatusBag_.get(key);
+        List<FileStatus> status = streamStatusBag_.get(key);
         if ( status == null )
             throw new IllegalStateException("Streaming status has not been set 
for " + key);
-        CompletedFileStatus streamStatus = status.remove(0);
+        FileStatus streamStatus = status.remove(0);
         if ( status.isEmpty() )
             streamStatusBag_.remove(key);
         return streamStatus;
@@ -108,7 +108,7 @@ class StreamInManager
         streamNotificationHandlers_.put(key, streamComplete);
     }
     
-    public synchronized static void addStreamContext(InetAddress key, 
PendingFile pendingFile, CompletedFileStatus streamStatus)
+    public synchronized static void addStreamContext(InetAddress key, 
PendingFile pendingFile, FileStatus streamStatus)
     {
         /* Record the stream context */
         List<PendingFile> context = ctxBag_.get(key);
@@ -120,10 +120,10 @@ class StreamInManager
         context.add(pendingFile);
         
         /* Record the stream status for this stream context */
-        List<CompletedFileStatus> status = streamStatusBag_.get(key);
+        List<FileStatus> status = streamStatusBag_.get(key);
         if ( status == null )
         {
-            status = new ArrayList<CompletedFileStatus>();
+            status = new ArrayList<FileStatus>();
             streamStatusBag_.put(key, status);
         }
         status.add( streamStatus );

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java?rev=948535&r1=948534&r2=948535&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
 Wed May 26 18:56:27 2010
@@ -79,8 +79,7 @@ public class StreamInitiateVerbHandler i
                 PendingFile remoteFile = pendingFile.getKey();
                 PendingFile localFile = pendingFile.getValue();
 
-                CompletedFileStatus streamStatus = new 
CompletedFileStatus(remoteFile.getFilename(),
-                                                                           
remoteFile.getExpectedBytes());
+                FileStatus streamStatus = new 
FileStatus(remoteFile.getFilename(), remoteFile.getExpectedBytes());
 
                 if (logger.isDebugEnabled())
                   logger.debug("Preparing to receive stream from " + 
message.getFrom() + ": " + remoteFile + " -> " + localFile);
@@ -129,7 +128,7 @@ public class StreamInitiateVerbHandler i
         return mapping;
     }
 
-    private void addStreamContext(InetAddress host, PendingFile pendingFile, 
CompletedFileStatus streamStatus)
+    private void addStreamContext(InetAddress host, PendingFile pendingFile, 
FileStatus streamStatus)
     {
         if (logger.isDebugEnabled())
           logger.debug("Adding stream context " + pendingFile + " for " + host 
+ " ...");


Reply via email to