accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject svn commit: r1351844 - in /accumulo/trunk/server/src/main/java/org/apache/accumulo/server: master/Master.java tabletserver/log/LogSorter.java
Date Tue, 19 Jun 2012 20:41:04 GMT
Author: kturner
Date: Tue Jun 19 20:41:03 2012
New Revision: 1351844

URL: http://svn.apache.org/viewvc?rev=1351844&view=rev
Log:
ACCUMULO-649 fixed a few log recovery issues

Modified:
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java
    accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java?rev=1351844&r1=1351843&r2=1351844&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java (original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/master/Master.java Tue
Jun 19 20:41:03 2012
@@ -2072,6 +2072,12 @@ public class Master implements LiveTServ
       @Override
       public void process(WatchedEvent event) {
         nextEvent.event("Noticed recovery changes", event.getType());
+        try {
+          // watcher only fires once, add it back
+          ZooReaderWriter.getInstance().getChildren(zroot + Constants.ZRECOVERY, this);
+        } catch (Exception e) {
+          log.error("Failed to add log recovery watcher back", e);
+        }
       }
     });
     

Modified: accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java
URL: http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java?rev=1351844&r1=1351843&r2=1351844&view=diff
==============================================================================
--- accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java
(original)
+++ accumulo/trunk/server/src/main/java/org/apache/accumulo/server/tabletserver/log/LogSorter.java
Tue Jun 19 20:41:03 2012
@@ -113,7 +113,7 @@ public class LogSorter {
           }
         }
         fs.create(new Path(destPath, "finished")).close();
-        log.info("Log copy/sort of " + name + " complete");
+        log.debug("Log copy/sort of " + name + " complete");
       } catch (Throwable t) {
         try {
           fs.create(new Path(destPath, "failed")).close();
@@ -201,7 +201,7 @@ public class LogSorter {
           case NodeChildrenChanged:
             if (event.getPath().equals(path))
               try {
-                attemptRecoveries(zoo, serverName, path, zoo.getChildren(path));
+                attemptRecoveries(zoo, serverName, path, zoo.getChildren(path, this));
               } catch (KeeperException e) {
                 log.error("Unable to get recovery information", e);
               } catch (InterruptedException e) {
@@ -237,10 +237,14 @@ public class LogSorter {
     }, r.nextInt(1000), 60 * 1000);
   }
   
-  private void attemptRecoveries(final ZooReaderWriter zoo, final String serverName, String
path, List<String> children) {
+  private void attemptRecoveries(final ZooReaderWriter zoo, final String serverName, final
String path, List<String> children) {
     if (children.size() == 0)
       return;
-    log.info("Zookeeper references " + children.size() + " recoveries, attempting locks");
+    
+    if (threadPool.getQueue().size() > 1)
+      return;
+
+    log.debug("Zookeeper references " + children.size() + " recoveries, attempting locks");
     Random random = new Random();
     Collections.shuffle(children, random);
     try {
@@ -258,9 +262,9 @@ public class LogSorter {
           if (threadPool.getQueue().size() > 1) {
             lock.unlock();
             log.debug("got the lock, but thread pool is busy; released the lock on " + child);
-            continue;
+            break;
           }
-          log.info("got lock for " + child);
+          log.debug("got lock for " + child);
           byte[] contents = zoo.getData(childPath, null);
           String destination = Constants.getRecoveryDir(conf) + "/" + child;
           startSort(new String(contents), destination, new LogSortNotifier() {
@@ -272,10 +276,17 @@ public class LogSorter {
               } catch (Exception e) {
                 log.error("Error received when trying to delete recovery entry in zookeeper
" + childPath);
               }
+              try {
+                attemptRecoveries(zoo, serverName, path, zoo.getChildren(path));
+              } catch (KeeperException e) {
+                log.error("Unable to get recovery information", e);
+              } catch (InterruptedException e) {
+                log.info("Interrupted getting recovery information", e);
+              }
             }
           });
         } else {
-          log.info("failed to get the lock " + child);
+          log.debug("failed to get the lock " + child);
         }
       }
     } catch (Throwable t) {



Mime
View raw message