keith-turner commented on code in PR #3392:
URL: https://github.com/apache/accumulo/pull/3392#discussion_r1191515037


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java:
##########
@@ -350,41 +350,43 @@ Optional<StoredTabletFile> 
bringMinorCompactionOnline(TabletFile tmpDatafile,
         logFileOnly.add(unusedWalLog);
       }
     }
-    try {
-      // the order of writing to metadata and walog is important in the face 
of machine/process
-      // failures need to write to metadata before writing to walog, when 
things are done in the
-      // reverse order data could be lost... the minor compaction start even 
should be written
-      // before the following metadata write is made
-      newFile = 
tablet.updateTabletDataFile(commitSession.getMaxCommittedTime(), newDatafile, 
dfv,
-          unusedWalLogs, flushId);
-
-      // Mark that we have data we want to replicate
-      // This WAL could still be in use by other Tablets *from the same 
table*, so we can only mark
-      // that there is data to replicate,
-      // but it is *not* closed. We know it is not closed by the fact that 
this MinC triggered. A
-      // MinC cannot happen unless the
-      // tablet is online and thus these WALs are referenced by that tablet. 
Therefore, the WAL
-      // replication status cannot be 'closed'.
-      if (replicate) {
-        if (log.isDebugEnabled()) {
-          log.debug("Recording that data has been ingested into {} using {}", 
tablet.getExtent(),
-              logFileOnly);
-        }
-        for (String logFile : logFileOnly) {
-          @SuppressWarnings("deprecation")
-          Status status = 
org.apache.accumulo.server.replication.StatusUtil.openWithUnknownLength();
-          ReplicationTableUtil.updateFiles(tablet.getContext(), 
tablet.getExtent(), logFile,
-              status);
-        }
-      }
-    } finally {
-      tablet.finishClearingUnusedLogs();
-    }
 
     // increment start count before metadata update AND updating in memory map 
of files
     metadataUpdateCount.updateAndGet(MetadataUpdateCount::incrementStart);
     // do not place any code here between above stmt and try{}finally
     try {
+      try {
+        // the order of writing to metadata and walog is important in the face 
of machine/process
+        // failures need to write to metadata before writing to walog, when 
things are done in the
+        // reverse order data could be lost... the minor compaction start even 
should be written
+        // before the following metadata write is made
+        newFile = 
tablet.updateTabletDataFile(commitSession.getMaxCommittedTime(), newDatafile, 
dfv,

Review Comment:
   Analyzing the code, it seems like the following log message should trigger
   
   
https://github.com/apache/accumulo/blob/fdabdc5133dd262316bac5d71a832f332831a61a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java#L135
   
   Also it rethrows the exception on the next line, so it seems like that 
should go to the uncaught exception handler in a thread pool.
   
   The exception takes an interesting path from being thrown till the log stmt 
referenced above.  Things catch it and rethrow it and do tracing stuff multiple 
times.  Something also wraps it another exception and adds the extent.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to