accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject svn commit: r1297304 - /incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
Date Tue, 06 Mar 2012 00:05:24 GMT
Author: kturner
Date: Tue Mar  6 00:05:23 2012
New Revision: 1297304

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

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

Modified: incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java?rev=1297304&r1=1297303&r2=1297304&view=diff
==============================================================================
--- incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
(original)
+++ incubator/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
Tue Mar  6 00:05:23 2012
@@ -2409,17 +2409,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);
           }
           
           Assignment assignment = new Assignment(extentToOpen, getTabletSession());



Mime
View raw message