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

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

commit 18c2254af1f0a16fbb52df604973a8ccd94dafb1
Merge: 3ad9198a86 66f49ef8c3
Author: Keith Turner <ktur...@apache.org>
AuthorDate: Thu Jul 25 15:21:13 2024 -0400

    Merge branch 'main' into elasticity

 .../org/apache/accumulo/tserver/tablet/Tablet.java | 32 +++++++---------------
 1 file changed, 10 insertions(+), 22 deletions(-)

diff --cc 
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 12c8ddc185,4ea148046b..62e4e17b77
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@@ -781,12 -888,7 +781,7 @@@ public class Tablet extends TabletBase 
    @Override
    public void close(boolean saveState) throws IOException {
      initiateClose(saveState);
-     final var lock = lockLogLock();
-     try {
-       completeClose(saveState);
-     } finally {
-       lock.unlock();
-     }
 -    completeClose(saveState, true);
++    completeClose(saveState);
      log.info("Tablet {} closed.", this.extent);
    }
  
@@@ -894,13 -1002,8 +889,8 @@@
  
    private boolean closeCompleting = false;
  
 -  synchronized void completeClose(boolean saveState, boolean completeClose) 
throws IOException {
 +  synchronized void completeClose(boolean saveState) throws IOException {
  
-     // The lockLock must be acquired before the tablet lock. Later in this 
function the log lock may
-     // be acquired during minor compaction. It will fail if the tablet lock 
is held and not the log
-     // lock. Fail sooner here and always, not only in the case when a minor 
compaction is needed.
-     Preconditions.checkState(logLock.isHeldByCurrentThread());
- 
      if (!isClosing() || isCloseComplete() || closeCompleting) {
        throw new IllegalStateException("Bad close state " + closeState + " on 
tablet " + extent);
      }
@@@ -1497,105 -2220,12 +1485,105 @@@
      return timer.getTabletStats();
    }
  
 -  private static String createTabletDirectoryName(ServerContext context, Text 
endRow) {
 -    if (endRow == null) {
 -      return ServerColumnFamily.DEFAULT_TABLET_DIR_NAME;
 -    } else {
 -      UniqueNameAllocator namer = context.getUniqueNameAllocator();
 -      return Constants.GENERATED_TABLET_DIRECTORY_PREFIX + 
namer.getNextName();
 +  public boolean isOnDemand() {
 +    // TODO a change in the tablet availability could refresh online tablets
 +    return getMetadata().getTabletAvailability() == 
TabletAvailability.ONDEMAND;
 +  }
 +
 +  // The purpose of this lock is to prevent race conditions between 
concurrent refresh RPC calls and
 +  // between minor compactions and refresh calls.
 +  private final ReentrantLock refreshLock = new ReentrantLock();
 +
 +  void bringMinorCompactionOnline(ReferencedTabletFile tmpDatafile,
 +      ReferencedTabletFile newDatafile, DataFileValue dfv, CommitSession 
commitSession,
 +      long flushId, MinorCompactionReason mincReason) {
 +    Optional<StoredTabletFile> newFile;
 +    // rename before putting in metadata table, so files in metadata table 
should
 +    // always exist
 +    boolean attemptedRename = false;
 +    VolumeManager vm = getTabletServer().getContext().getVolumeManager();
 +    do {
 +      try {
 +        if (dfv.getNumEntries() == 0) {
 +          log.debug("No data entries so delete temporary file {}", 
tmpDatafile);
 +          vm.deleteRecursively(tmpDatafile.getPath());
 +        } else {
 +          if (!attemptedRename && vm.exists(newDatafile.getPath())) {
 +            log.warn("Target data file already exist {}", newDatafile);
 +            throw new RuntimeException("File unexpectedly exists " + 
newDatafile.getPath());
 +          }
 +          // the following checks for spurious rename failures that succeeded 
but gave an IoE
 +          if (attemptedRename && vm.exists(newDatafile.getPath())
 +              && !vm.exists(tmpDatafile.getPath())) {
 +            // seems like previous rename succeeded, so break
 +            break;
 +          }
 +          attemptedRename = true;
 +          ScanfileManager.rename(vm, tmpDatafile.getPath(), 
newDatafile.getPath());
 +        }
 +        break;
 +      } catch (IOException ioe) {
 +        log.warn("Tablet " + getExtent() + " failed to rename " + newDatafile
 +            + " after MinC, will retry in 60 secs...", ioe);
 +        sleepUninterruptibly(1, TimeUnit.MINUTES);
 +      }
 +    } while (true);
 +
 +    // The refresh lock must be held for the metadata write that adds the new 
file to the tablet.
 +    // This prevents a concurrent refresh operation from pulling in the new 
tablet file before the
 +    // in memory map reference related to the file is deactivated. Scans 
should use one of the in
 +    // memory map or the new file, never both.
-     Preconditions.checkState(!logLock.isHeldByCurrentThread());
++    Preconditions.checkState(!getLogLock().isHeldByCurrentThread());
 +    refreshLock.lock();
 +    try {
 +      // Can not hold tablet lock while acquiring the log lock.
-       final var localLogLock = lockLogLock();
++      getLogLock().lock();
 +      // do not place any code here between lock and try
 +      try {
 +        // The following call pairs with tablet.finishClearingUnusedLogs() 
later in this block. If
 +        // moving where the following method is called, examine it and 
finishClearingUnusedLogs()
 +        // before moving.
 +        Set<LogEntry> unusedWalLogs = beginClearingUnusedLogs();
 +        // 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 
event should be written
 +        // before the following metadata write is made
 +
 +        newFile = updateTabletDataFile(commitSession.getMaxCommittedTime(), 
newDatafile, dfv,
 +            unusedWalLogs, flushId, mincReason);
 +
 +        finishClearingUnusedLogs();
 +      } finally {
-         localLogLock.unlock();
++        getLogLock().unlock();
 +      }
 +
 +      // Without the refresh lock, if a refresh happened here it could make 
the new file written to
 +      // the metadata table above available for scans while the in memory map 
from which the file
 +      // was produced is still available for scans
 +
 +      do {
 +        try {
 +          // the purpose of making this update use the new commit session, 
instead of the old one
 +          // passed in, is because the new one will reference the logs used 
by current memory...
 +          
getTabletServer().minorCompactionFinished(getTabletMemory().getCommitSession(),
 +              commitSession.getWALogSeq() + 2);
 +          break;
 +        } catch (IOException e) {
 +          log.error("Failed to write to write-ahead log " + e.getMessage() + 
" will retry", e);
 +          sleepUninterruptibly(1, TimeUnit.SECONDS);
 +        }
 +      } while (true);
 +
 +      refreshMetadata(RefreshPurpose.MINC_COMPLETION);
 +    } finally {
 +      refreshLock.unlock();
 +    }
 +    TabletLogger.flushed(getExtent(), newFile);
 +
 +    long splitSize = 
getTableConfiguration().getAsBytes(Property.TABLE_SPLIT_THRESHOLD);
 +    if (dfv.getSize() > splitSize) {
 +      log.debug(String.format("Minor Compaction wrote out file larger than 
split threshold."
 +          + " split threshold = %,d  file size = %,d", splitSize, 
dfv.getSize()));
      }
    }
  

Reply via email to