You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2012/03/06 16:49:19 UTC

svn commit: r1297539 - /incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java

Author: kturner
Date: Tue Mar  6 15:49:19 2012
New Revision: 1297539

URL: http://svn.apache.org/viewvc?rev=1297539&view=rev
Log:
ACCUMULO-444 wait for MinC after log recovery before putting tablet online (merged from 1.4)

Modified:
    incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java

Modified: incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1297539&r1=1297538&r2=1297539&view=diff
==============================================================================
--- incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java (original)
+++ incubator/accumulo/branches/1.3/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java Tue Mar  6 15:49:19 2012
@@ -151,15 +151,15 @@ import org.apache.accumulo.core.util.Col
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.LoggingRunnable;
 import org.apache.accumulo.core.util.ServerServices;
+import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.zookeeper.ZooCache;
 import org.apache.accumulo.core.zookeeper.ZooLock;
-import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.core.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.core.zookeeper.ZooLock.LockWatcher;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -200,8 +200,8 @@ import org.apache.accumulo.server.util.F
 import org.apache.accumulo.server.util.Halt;
 import org.apache.accumulo.server.util.MapCounter;
 import org.apache.accumulo.server.util.MetadataTable;
-import org.apache.accumulo.server.util.TServerUtils;
 import org.apache.accumulo.server.util.MetadataTable.LogEntry;
+import org.apache.accumulo.server.util.TServerUtils;
 import org.apache.accumulo.server.util.TServerUtils.ServerPort;
 import org.apache.accumulo.server.util.time.RelativeTime;
 import org.apache.accumulo.server.util.time.SimpleTimer;
@@ -211,7 +211,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableName;
 import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
@@ -2170,14 +2169,21 @@ public class TabletServer extends Abstra
           
           // this opens the tablet file and fills in the endKey in the extent
           tablet = new Tablet(TabletServer.this, locationToOpen, extentToOpen, trm, tabletsKeyValues);
-          if (!tablet.initiateMinorCompaction() && tablet.getNumEntriesInMemory() > 0) {
-            log.warn("Minor compaction after recovery fails for " + extentToOpen);
-            
-            // it is important to wait for minc in the case that the minor compaction finish
-            // event did not make it to the logs (the file will be in !METADATA, preventing replay of compacted data)...
-            // but do not want a majc to wipe the file out from !METADATA and then have another process failure...
-            // this could cause duplicate data to replay
+          if (tablet.initiateMinorCompaction()) {
+            /*
+             * If a minor compaction starts after a tablet opens, this indicates a log recovery occurred. This recovered data must be minor compacted.
+             * 
+             * There are three reasons to wait for this minor compaction to finish before placing the tablet in online tablets.
+             * 
+             * 1) The log recovery code does not handle data written to the tablet on multiple tablet servers. 2) The log recovery code does not block if memory
+             * is full. Therefore recovering lots of tablets that use a lot of memory could run out of memory. 3) The minor compaction finish event did not make
+             * it to the logs (the file will be in !METADATA, preventing replay of compacted data)... but do not want a majc to wipe the file out from !METADATA
+             * and then have another process failure... this could cause duplicate data to replay
+             */
+
             tablet.waitForMinC();
+          } else if (tablet.getNumEntries() > 0) {
+            log.warn("Minor compaction after recovery fails for " + extentToOpen);
           }
           
           synchronized (openingTablets) {