cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gdusba...@apache.org
Subject svn commit: r920361 - in /incubator/cassandra/trunk: ./ interface/thrift/gen-java/org/apache/cassandra/thrift/ src/java/org/apache/cassandra/streaming/
Date Mon, 08 Mar 2010 15:42:42 GMT
Author: gdusbabek
Date: Mon Mar  8 15:42:41 2010
New Revision: 920361

URL: http://svn.apache.org/viewvc?rev=920361&view=rev
Log:
merge changes from 0.6

Modified:
    incubator/cassandra/trunk/   (props changed)
    incubator/cassandra/trunk/CHANGES.txt
    incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
  (props changed)
    incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
  (props changed)
    incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java
  (props changed)
    incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java
  (props changed)
    incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
  (props changed)
    incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingService.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java

Propchange: incubator/cassandra/trunk/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,4 +1,4 @@
 /incubator/cassandra/branches/cassandra-0.3:774578-796573
 /incubator/cassandra/branches/cassandra-0.4:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5:888872-915439
-/incubator/cassandra/branches/cassandra-0.6:911237-919603
+/incubator/cassandra/branches/cassandra-0.6:911237-920360

Modified: incubator/cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/CHANGES.txt?rev=920361&r1=920360&r2=920361&view=diff
==============================================================================
--- incubator/cassandra/trunk/CHANGES.txt (original)
+++ incubator/cassandra/trunk/CHANGES.txt Mon Mar  8 15:42:41 2010
@@ -18,6 +18,8 @@
  * Add logging of GC activity (CASSANDRA-813)
  * fix ConcurrentModificationException in commitlog discard (CASSANDRA-853)
  * Fix hardcoded row count in Hadoop RecordReader (CASSANDRA-837)
+ * Add a jmx status to the streaming service and change several DEBUG
+   messages to INFO (CASSANDRA-845)
 
 
 0.6.0-beta1/beta2

Propchange: incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,6 +1,6 @@
 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/Cassandra.java:774578-796573
 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Cassandra.java:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Cassandra.java:888872-903502
-/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:911237-919603
+/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:911237-920360
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/Cassandra.java:749219-768588
 /incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java:749219-904544

Propchange: incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,7 +1,7 @@
 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/column_t.java:774578-792198
 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/Column.java:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/Column.java:888872-903502
-/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:911237-919603
+/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:911237-920360
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/Column.java:749219-794428
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/column_t.java:749219-768588
 /incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/Column.java:749219-904544

Propchange: incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,6 +1,6 @@
 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/InvalidRequestException.java:774578-796573
 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/InvalidRequestException.java:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/InvalidRequestException.java:888872-903502
-/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:911237-919603
+/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:911237-920360
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/InvalidRequestException.java:749219-768588
 /incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/InvalidRequestException.java:749219-904544

Propchange: incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,5 +1,5 @@
 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/NotFoundException.java:774578-796573
 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/NotFoundException.java:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/NotFoundException.java:888872-903502
-/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java:911237-919603
+/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/NotFoundException.java:911237-920360
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/NotFoundException.java:749219-768588

Propchange: incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Mon Mar  8 15:42:41 2010
@@ -1,7 +1,7 @@
 /incubator/cassandra/branches/cassandra-0.3/interface/gen-java/org/apache/cassandra/service/superColumn_t.java:774578-792198
 /incubator/cassandra/branches/cassandra-0.4/interface/gen-java/org/apache/cassandra/service/SuperColumn.java:810145-834239,834349-834350
 /incubator/cassandra/branches/cassandra-0.5/interface/gen-java/org/apache/cassandra/service/SuperColumn.java:888872-903502
-/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java:911237-919603
+/incubator/cassandra/branches/cassandra-0.6/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java:911237-920360
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/SuperColumn.java:749219-794428
 /incubator/cassandra/trunk/interface/gen-java/org/apache/cassandra/service/superColumn_t.java:749219-768588
 /incubator/cassandra/trunk/interface/thrift/gen-java/org/apache/cassandra/thrift/SuperColumn.java:749219-904544

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java?rev=920361&r1=920360&r2=920361&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java Mon Mar
 8 15:42:41 2010
@@ -31,13 +31,12 @@
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.streaming.StreamInitiateMessage;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.streaming.StreamOutManager;
+
 
 /**
  * This class handles streaming data from one node to another.
@@ -57,6 +56,13 @@
     private static Logger logger = Logger.getLogger(StreamOut.class);
 
     static String TABLE_NAME = "STREAMING-TABLE-NAME";
+    
+    private static void updateStatus(String msg)
+    {
+        StreamingService.instance.setStatus(msg);
+        if (logger.isInfoEnabled() && !StreamingService.NOTHING.equals(msg))
+            logger.info(msg);
+    }
 
     /**
      * Split out files for all tables on disk locally for each range and then stream them
to the target endpoint.
@@ -65,8 +71,7 @@
     {
         assert ranges.size() > 0;
 
-        if (logger.isDebugEnabled())
-            logger.debug("Beginning transfer process to " + target + " for ranges " + StringUtils.join(ranges,
", "));
+        logger.debug("Beginning transfer process to " + target + " for ranges " + StringUtils.join(ranges,
", "));
 
         /*
          * (1) dump all the memtables to disk.
@@ -76,8 +81,7 @@
         try
         {
             Table table = Table.open(tableName);
-            if (logger.isDebugEnabled())
-                logger.debug("Flushing memtables ...");
+            updateStatus("Flushing memtables for " + tableName + "...");
             for (Future f : table.flush())
             {
                 try
@@ -93,8 +97,7 @@
                     throw new RuntimeException(e);
                 }
             }
-            if (logger.isDebugEnabled())
-                logger.debug("Performing anticompaction ...");
+            updateStatus("Performing anticompaction ...");
             /* Get the list of files that need to be streamed */
             transferSSTables(target, table.forceAntiCompaction(ranges, target), tableName);
// SSTR GC deletes the file when done
         }
@@ -102,6 +105,10 @@
         {
             throw new IOError(e);
         }
+        finally
+        {
+            StreamingService.instance.setStatus(StreamingService.NOTHING);
+        }
         if (callback != null)
             callback.run();
     }
@@ -124,23 +131,21 @@
             }
         }
         if (logger.isDebugEnabled())
-          logger.debug("Stream context metadata " + StringUtils.join(pendingFiles, ", " +
" " + sstables.size() + " sstables."));
-
+            logger.debug("Stream context metadata " + StringUtils.join(pendingFiles, ", "
+ " " + sstables.size() + " sstables."));
         StreamOutManager.get(target).addFilesToStream(pendingFiles);
         StreamInitiateMessage biMessage = new StreamInitiateMessage(pendingFiles);
         Message message = StreamInitiateMessage.makeStreamInitiateMessage(biMessage);
         message.setHeader(StreamOut.TABLE_NAME, table.getBytes());
-        if (logger.isDebugEnabled())
-          logger.debug("Sending a stream initiate message to " + target + " ...");
+        updateStatus("Sending a stream initiate message to " + target + " ...");
         MessagingService.instance.sendOneWay(message, target);
 
         if (pendingFiles.length > 0)
         {
-            logger.info("Waiting for transfer to " + target + " to complete");
+            StreamingService.instance.setStatus("Waiting for transfer to " + target + " to
complete");
             StreamOutManager.get(target).waitForStreamCompletion();
             // todo: it would be good if there were a dafe way to remove the StreamManager
for target.
             // (StreamManager will delete the streamed file on completion.)
-            logger.info("Done with transfer to " + target);
+            updateStatus("Done with transfer to " + target);
         }
     }
 

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingService.java?rev=920361&r1=920360&r2=920361&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingService.java
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingService.java
Mon Mar  8 15:42:41 2010
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.streaming;
 
-import org.apache.cassandra.streaming.StreamingServiceMBean;
 import org.apache.log4j.Logger;
 
 import javax.management.MBeanServer;
@@ -35,6 +34,8 @@
     private static final Logger logger = Logger.getLogger(StreamingService.class);
     public static final String MBEAN_OBJECT_NAME = "org.apache.cassandra.streaming:type=StreamingService";
     public static final StreamingService instance = new StreamingService();
+    static final String NOTHING = "Nothing is happening";
+    private String status = NOTHING;
 
     private StreamingService()
     {
@@ -48,6 +49,17 @@
             throw new RuntimeException(e);
         }
     }
+    
+    public void setStatus(String s)
+    {
+        assert s != null;
+        status = s;
+    }
+
+    public String getStatus()
+    {
+        return status;
+    }
 
     /** hosts receiving outgoing streams. */
     public Set<InetAddress> getStreamDestinations()

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java?rev=920361&r1=920360&r2=920361&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
Mon Mar  8 15:42:41 2010
@@ -36,4 +36,7 @@
 
     /** details about incoming streams */
     public List<String> getIncomingFiles(String host) throws IOException;
+    
+    /** What's currently happening wrt streaming. */
+    public String getStatus();
 }



Mime
View raw message