This is an automated email from the ASF dual-hosted git repository.

edcoleman pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new e69456534b CompactableImpl sync fix and message update (#4198)
e69456534b is described below

commit e69456534b12f33178d6372f4905a32f023217f6
Author: EdColeman <d...@etcoleman.com>
AuthorDate: Fri Jan 26 14:30:12 2024 -0500

    CompactableImpl sync fix and message update (#4198)
    
    * remove unnecessary sync, clarify error messages
---
 .../accumulo/tserver/tablet/CompactableImpl.java   | 34 +++++++++++-----------
 1 file changed, 17 insertions(+), 17 deletions(-)

diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
index 73a2efa0e4..89c68c5b55 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactableImpl.java
@@ -106,12 +106,12 @@ public class CompactableImpl implements Compactable {
 
   private final FileManager fileMgr;
 
-  private Set<CompactionJob> runningJobs = new HashSet<>();
+  private final Set<CompactionJob> runningJobs = new HashSet<>();
   private volatile boolean compactionRunning = false;
 
-  private Supplier<Set<CompactionServiceId>> servicesInUse;
+  private final Supplier<Set<CompactionServiceId>> servicesInUse;
 
-  private Set<CompactionServiceId> servicesUsed = new 
ConcurrentSkipListSet<>();
+  private final Set<CompactionServiceId> servicesUsed = new 
ConcurrentSkipListSet<>();
 
   enum ChopSelectionStatus {
     SELECTING, SELECTED, NOT_ACTIVE, MARKING
@@ -126,21 +126,21 @@ public class CompactableImpl implements Compactable {
   private Long compactionId;
   private CompactionConfig compactionConfig;
 
-  private CompactionManager manager;
+  private final CompactionManager manager;
 
   AtomicLong lastSeenCompactionCancelId = new AtomicLong(Long.MIN_VALUE);
 
   private volatile boolean closed = false;
 
-  private Map<ExternalCompactionId,ExternalCompactionInfo> externalCompactions 
=
+  private final Map<ExternalCompactionId,ExternalCompactionInfo> 
externalCompactions =
       new ConcurrentHashMap<>();
 
-  private Set<ExternalCompactionId> externalCompactionsCommitting = new 
HashSet<>();
+  private final Set<ExternalCompactionId> externalCompactionsCommitting = new 
HashSet<>();
 
   // This interface exists for two purposes. First it allows abstraction of 
new and old
   // implementations for user pluggable file selection code. Second it 
facilitates placing code
   // outside of this class.
-  public static interface CompactionHelper {
+  public interface CompactionHelper {
     Set<StoredTabletFile> 
selectFiles(SortedMap<StoredTabletFile,DataFileValue> allFiles);
 
     Set<StoredTabletFile> getFilesToDrop();
@@ -195,17 +195,17 @@ public class CompactableImpl implements Compactable {
     // important to track this in order to know if the last compaction is a 
full compaction and
     // should not propagate deletes.
     private boolean initiallySelectedAll = false;
-    private Set<StoredTabletFile> selectedFiles = new HashSet<>();
+    private final Set<StoredTabletFile> selectedFiles = new HashSet<>();
 
     protected Set<StoredTabletFile> allCompactingFiles = new HashSet<>();
 
     // track files produced by compactions of this tablet, those are 
considered chopped
-    private Set<StoredTabletFile> choppedFiles = new HashSet<>();
+    private final Set<StoredTabletFile> choppedFiles = new HashSet<>();
     private ChopSelectionStatus chopStatus = ChopSelectionStatus.NOT_ACTIVE;
-    private Set<StoredTabletFile> allFilesWhenChopStarted = new HashSet<>();
+    private final Set<StoredTabletFile> allFilesWhenChopStarted = new 
HashSet<>();
 
     private final KeyExtent extent;
-    private Deriver<Duration> selectionExpirationDeriver;
+    private final Deriver<Duration> selectionExpirationDeriver;
 
     public FileManager(KeyExtent extent, Collection<StoredTabletFile> 
extCompactingFiles,
         Optional<SelectedInfo> extSelInfo, Deriver<Duration> 
selectionExpirationDeriver) {
@@ -314,8 +314,8 @@ public class CompactableImpl implements Compactable {
     }
 
     class ChopSelector {
-      private Set<StoredTabletFile> allFiles;
-      private Set<StoredTabletFile> filesToExamine;
+      private final Set<StoredTabletFile> allFiles;
+      private final Set<StoredTabletFile> filesToExamine;
 
       private ChopSelector(Set<StoredTabletFile> allFiles, 
Set<StoredTabletFile> filesToExamine) {
         this.allFiles = allFiles;
@@ -1485,7 +1485,7 @@ public class CompactableImpl implements Compactable {
 
       if (dispatcher == null) {
         log.error(
-            "Failed to dispatch compaction {} kind:{} hints:{}, falling back 
to {} service. Unable to instantiate dispatcher plugin. Check server log.",
+            "Failed to dispatch compaction, no dispatcher. extent:{} kind:{} 
hints:{}, falling back to {} service. Unable to instantiate dispatcher plugin. 
Check server log.",
             getExtent(), kind, debugHints, 
CompactionServicesConfig.DEFAULT_SERVICE);
         return CompactionServicesConfig.DEFAULT_SERVICE;
       }
@@ -1532,7 +1532,8 @@ public class CompactableImpl implements Compactable {
 
       return dispatch.getService();
     } catch (RuntimeException e) {
-      log.error("Failed to dispatch compaction {} kind:{} hints:{}, falling 
back to {} service.",
+      log.error(
+          "Failed to dispatch compaction due to exception. extent:{} kind:{} 
hints:{}, falling back to {} service.",
           getExtent(), kind, debugHints, 
CompactionServicesConfig.DEFAULT_SERVICE, e);
       return CompactionServicesConfig.DEFAULT_SERVICE;
     }
@@ -1556,7 +1557,7 @@ public class CompactableImpl implements Compactable {
    * Interrupts and waits for any running compactions. After this method 
returns, no compactions
    * should be running and none should be able to start.
    */
-  public synchronized void close() {
+  public void close() {
     synchronized (this) {
       if (closed) {
         return;
@@ -1580,7 +1581,6 @@ public class CompactableImpl implements Compactable {
         }
       }
     }
-
     manager.compactableClosed(getExtent(), servicesUsed, 
externalCompactions.keySet());
   }
 }

Reply via email to