This is an automated email from the ASF dual-hosted git repository. kturner pushed a commit to branch 1.10 in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/1.10 by this push: new d61ba8ea3a Avoids holding tablet lock while doing split computations (#3249) d61ba8ea3a is described below commit d61ba8ea3a939a51996d46742d4690a39b09522f Author: Keith Turner <ktur...@apache.org> AuthorDate: Fri Mar 24 19:30:24 2023 -0400 Avoids holding tablet lock while doing split computations (#3249) --- .../org/apache/accumulo/tserver/TabletServer.java | 2 +- .../tserver/tablet/MinorCompactionTask.java | 2 +- .../org/apache/accumulo/tserver/tablet/Tablet.java | 245 ++++++++++++++------- 3 files changed, 165 insertions(+), 84 deletions(-) diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index e141de9608..6e29beaae3 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -2081,7 +2081,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable { // if we need to split AND compact, we need a good way // to decide what to do - if (tablet.needsSplit()) { + if (tablet.needsSplit(tablet.getSplitComputations())) { executeSplit(tablet); continue; } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java index 8f922d2084..3d745fc697 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java @@ -105,7 +105,7 @@ class MinorCompactionTask implements Runnable { minorCompaction.data("size", Long.toString(this.stats.getSize())); minorCompaction.stop(); - if (tablet.needsSplit()) { + if (tablet.needsSplit(tablet.getSplitComputations())) { tablet.getTabletServer().executeSplit(tablet); } else { tablet.initiateMajorCompaction(MajorCompactionReason.NORMAL); diff --git 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 index 6c452c4d64..5352b7e91a 100644 --- 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 @@ -45,6 +45,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import org.apache.accumulo.core.Constants; @@ -1629,13 +1630,19 @@ public class Tablet implements TabletCommitter { return true; } - private SplitRowSpec findSplitRow(Collection<FileRef> files) { + private SplitRowSpec findSplitRow(Optional<SplitComputations> splitComputations) { long maxEndRow = tableConfiguration.getMemoryInBytes(Property.TABLE_MAX_END_ROW_SIZE); if (!isSplitPossible()) { return null; } + if (!splitComputations.isPresent()) { + // information needed to compute a split point is out of date or does not exists, try again + // later + return null; + } + // have seen a big row before, do not bother checking unless a minor compaction or map file // import has occurred. if (sawBigRow) { @@ -1649,17 +1656,7 @@ public class Tablet implements TabletCommitter { } } - SortedMap<Double,Key> keys = null; - - try { - // we should make .25 below configurable - keys = FileUtil.findMidPoint(getTabletServer().getFileSystem(), - getTabletServer().getConfiguration(), extent.getPrevEndRow(), extent.getEndRow(), - FileUtil.toPathStrings(files), .25); - } catch (IOException e) { - log.error("Failed to find midpoint " + e.getMessage()); - return null; - } + SortedMap<Double,Key> keys = splitComputations.get().midPoint; if (keys.isEmpty()) { log.info("Cannot split tablet " + extent + ", files contain no data for tablet."); @@ -1673,85 +1670,75 @@ public class Tablet implements TabletCommitter { } // check to see if one row takes up most of the tablet, in which case we can not split - try { - - Text lastRow; - if (extent.getEndRow() == null) { - Key lastKey = (Key) FileUtil.findLastKey(getTabletServer().getFileSystem(), - getTabletServer().getConfiguration(), files); - lastRow = lastKey.getRow(); - } else { - lastRow = extent.getEndRow(); - } - - // We expect to get a midPoint for this set of files. If we don't get one, we have a problem. - final Key mid = keys.get(.5); - if (mid == null) { - throw new IllegalStateException("Could not determine midpoint for files on " + extent); - } + Text lastRow; + if (extent.getEndRow() == null) { + lastRow = splitComputations.get().lastRowForDefaultTablet; + } else { + lastRow = extent.getEndRow(); + } - // check to see that the midPoint is not equal to the end key - if (mid.compareRow(lastRow) == 0) { - if (keys.firstKey() < .5) { - Key candidate = keys.get(keys.firstKey()); - if (candidate.getLength() > maxEndRow) { - log.warn("Cannot split tablet " + extent - + ", selected split point too long. Length : " + candidate.getLength()); + // We expect to get a midPoint for this set of files. If we don't get one, we have a problem. + final Key mid = keys.get(.5); + if (mid == null) { + throw new IllegalStateException("Could not determine midpoint for files on " + extent); + } - sawBigRow = true; - timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; - timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; + // check to see that the midPoint is not equal to the end key + if (mid.compareRow(lastRow) == 0) { + if (keys.firstKey() < .5) { + Key candidate = keys.get(keys.firstKey()); + if (candidate.getLength() > maxEndRow) { + log.warn("Cannot split tablet " + extent + ", selected split point too long. Length : " + + candidate.getLength()); - return null; - } - if (candidate.compareRow(lastRow) != 0) { - // we should use this ratio in split size estimations - if (log.isTraceEnabled()) - log.trace( - String.format("Splitting at %6.2f instead of .5, row at .5 is same as end row%n", - keys.firstKey())); - return new SplitRowSpec(keys.firstKey(), candidate.getRow()); - } + sawBigRow = true; + timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; + timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; + return null; + } + if (candidate.compareRow(lastRow) != 0) { + // we should use this ratio in split size estimations + if (log.isTraceEnabled()) + log.trace( + String.format("Splitting at %6.2f instead of .5, row at .5 is same as end row%n", + keys.firstKey())); + return new SplitRowSpec(keys.firstKey(), candidate.getRow()); } - log.warn("Cannot split tablet " + extent + " it contains a big row : " + lastRow); + } - sawBigRow = true; - timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; - timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; + log.warn("Cannot split tablet " + extent + " it contains a big row : " + lastRow); - return null; - } + sawBigRow = true; + timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; + timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; - Text text = mid.getRow(); - SortedMap<Double,Key> firstHalf = keys.headMap(.5); - if (firstHalf.size() > 0) { - Text beforeMid = firstHalf.get(firstHalf.lastKey()).getRow(); - Text shorter = new Text(); - int trunc = longestCommonLength(text, beforeMid); - shorter.set(text.getBytes(), 0, Math.min(text.getLength(), trunc + 1)); - text = shorter; - } + return null; + } - if (text.getLength() > maxEndRow) { - log.warn("Cannot split tablet " + extent + ", selected split point too long. Length : " - + text.getLength()); + Text text = mid.getRow(); + SortedMap<Double,Key> firstHalf = keys.headMap(.5); + if (firstHalf.size() > 0) { + Text beforeMid = firstHalf.get(firstHalf.lastKey()).getRow(); + Text shorter = new Text(); + int trunc = longestCommonLength(text, beforeMid); + shorter.set(text.getBytes(), 0, Math.min(text.getLength(), trunc + 1)); + text = shorter; + } - sawBigRow = true; - timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; - timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; + if (text.getLength() > maxEndRow) { + log.warn("Cannot split tablet " + extent + ", selected split point too long. Length : " + + text.getLength()); - return null; - } + sawBigRow = true; + timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime; + timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime; - return new SplitRowSpec(.5, text); - } catch (IOException e) { - // don't split now, but check again later - log.error("Failed to find lastkey " + e.getMessage()); return null; } + return new SplitRowSpec(.5, text); } private static int longestCommonLength(Text text, Text beforeMid) { @@ -1816,14 +1803,99 @@ public class Tablet implements TabletCommitter { return result; } + // encapsulates results of computations needed to make determinations about splits + private static class SplitComputations { + final Set<FileRef> inputFiles; + + // cached result of calling FileUtil.findMidpoint + final SortedMap<Double,Key> midPoint; + + // the last row seen in the files, only set for the default tablet + final Text lastRowForDefaultTablet; + + private SplitComputations(Set<FileRef> inputFiles, SortedMap<Double,Key> midPoint, + Text lastRowForDefaultTablet) { + this.inputFiles = inputFiles; + this.midPoint = midPoint; + this.lastRowForDefaultTablet = lastRowForDefaultTablet; + } + } + + private AtomicReference<SplitComputations> lastSplitComputation = new AtomicReference<>(); + private Lock splitComputationLock = new ReentrantLock(); + + /** + * Computes split point information from files when a tablets set of files changes. Do not call + * this method when holding the tablet lock. + */ + @SuppressModernizer + public Optional<SplitComputations> getSplitComputations() { + + if (!isSplitPossible() || isClosing() || isClosed()) { + // do not want to bother doing any computations when a split is not possible + return Optional.absent(); + } + + Set<FileRef> files = getDatafileManager().getFiles(); + SplitComputations lastComputation = lastSplitComputation.get(); + if (lastComputation != null && lastComputation.inputFiles.equals(files)) { + // the last computation is still relevant + return Optional.of(lastComputation); + } + + if (Thread.holdsLock(this)) { + log.warn( + "Thread holding tablet lock is doing split computation, this is unexpected and needs " + + "investigation. Please open an Accumulo issue with the stack trace because this can " + + "cause performance problems for scans.", + new RuntimeException()); + } + + SplitComputations newComputation; + + // Only want one thread doing this computation at time for a tablet. + if (splitComputationLock.tryLock()) { + try { + SortedMap<Double, + Key> midpoint = FileUtil.findMidPoint(getTabletServer().getFileSystem(), + getTabletServer().getConfiguration(), extent.getPrevEndRow(), extent.getEndRow(), + FileUtil.toPathStrings(files), .25); + + Text lastRow = null; + + if (extent.getEndRow() == null) { + Key lastKey = (Key) FileUtil.findLastKey(getTabletServer().getFileSystem(), + getTabletServer().getConfiguration(), files); + lastRow = lastKey.getRow(); + } + + newComputation = new SplitComputations(files, midpoint, lastRow); + } catch (IOException e) { + lastSplitComputation = null; + log.error("Failed to compute split information from files " + e.getMessage()); + return Optional.absent(); + } finally { + splitComputationLock.unlock(); + } + + lastSplitComputation.set(newComputation); + + return Optional.of(newComputation); + } else { + // some other thread seems to be working on split, let the other thread work on it + return Optional.absent(); + } + } + /** * Returns true if this tablet needs to be split - * */ - public synchronized boolean needsSplit() { - if (isClosing() || isClosed()) + public synchronized boolean needsSplit(Optional<SplitComputations> splitComputations) { + if (!splitComputations.isPresent() || isClosing() || isClosed()) { return false; - return isSplitPossible(); + } + + return findSplitRow(splitComputations) != null; } // BEGIN PRIVATE METHODS RELATED TO MAJOR COMPACTION @@ -2157,12 +2229,15 @@ public class Tablet implements TabletCommitter { timer.incrementStatusMajor(); + // call this outside of tablet lock because it opens files + Optional<SplitComputations> splitComputations = getSplitComputations(); + synchronized (this) { // check that compaction is still needed - defer to splitting majorCompactionQueued.remove(reason); if (isClosing() || isClosed() || !needsMajorCompaction(reason) || isMajorCompactionRunning() - || needsSplit()) { + || needsSplit(splitComputations)) { return null; } @@ -2292,6 +2367,12 @@ public class Tablet implements TabletCommitter { throw new RuntimeException(msg); } + Optional<SplitComputations> splitComputations = null; + if (sp == null) { + // call this outside of sync block + splitComputations = getSplitComputations(); + } + try { initiateClose(true, false, false); } catch (IllegalStateException ise) { @@ -2316,7 +2397,7 @@ public class Tablet implements TabletCommitter { // choose a split point SplitRowSpec splitPoint; if (sp == null) - splitPoint = findSplitRow(getDatafileManager().getFiles()); + splitPoint = findSplitRow(splitComputations); else { Text tsp = new Text(sp); splitPoint = @@ -2491,7 +2572,7 @@ public class Tablet implements TabletCommitter { getDatafileManager().importMapFiles(tid, entries, setTime); lastMapFileImportTime = System.currentTimeMillis(); - if (needsSplit()) { + if (isSplitPossible()) { getTabletServer().executeSplit(this); } else { initiateMajorCompaction(MajorCompactionReason.NORMAL);