This is an automated email from the ASF dual-hosted git repository. kturner 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 688e4ed68b Avoids reusing rfile objects when runtime exception happens (#3623) 688e4ed68b is described below commit 688e4ed68b4980eccd77931fee3996987018fe5c Author: Keith Turner <ktur...@apache.org> AuthorDate: Tue Jul 18 11:24:59 2023 -0400 Avoids reusing rfile objects when runtime exception happens (#3623) Rfiles were not reused when IOException happened, but were reused when a runtime exception happened. This changes the code to not reuse when a runtime exception happens. See #3617 --- .../apache/accumulo/tserver/tablet/Scanner.java | 27 ++++++++++++---------- .../org/apache/accumulo/tserver/tablet/Tablet.java | 9 ++++---- .../apache/accumulo/tserver/tablet/TabletBase.java | 9 ++++---- 3 files changed, 25 insertions(+), 20 deletions(-) diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java index 107d36f24a..311492b822 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java @@ -137,7 +137,6 @@ public class Scanner { } sawException = true; - dataSource.close(true); throw ioe; } catch (RuntimeException re) { if (ShutdownUtil.wasCausedByHadoopShutdown(re)) { @@ -148,19 +147,23 @@ public class Scanner { sawException = true; throw re; } finally { - // code in finally block because always want - // to return mapfiles, even when exception is thrown - if (dataSource != null && !scanParams.isIsolated()) { - dataSource.close(false); - } else if (dataSource != null) { - dataSource.detachFileManager(); - } + try { + // code in finally block because always want + // to return mapfiles, even when exception is thrown + if (dataSource != null) { + if (sawException || !scanParams.isIsolated()) { + dataSource.close(sawException); + } else { + dataSource.detachFileManager(); + } + } - if (results != null && results.getResults() != null) { - tablet.updateQueryStats(results.getResults().size(), results.getNumBytes()); + if (results != null && results.getResults() != null) { + tablet.updateQueryStats(results.getResults().size(), results.getNumBytes()); + } + } finally { + scannerSemaphore.release(); } - - scannerSemaphore.release(); } } 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 23cc377bcf..251d9df30b 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 @@ -452,16 +452,17 @@ public class Tablet extends TabletBase { ScanDataSource dataSource = createDataSource(scanParams, false, iFlag); + boolean sawException = false; try { SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource); checker.check(iter); - } catch (IOException ioe) { - dataSource.close(true); - throw ioe; + } catch (IOException | RuntimeException e) { + sawException = true; + throw e; } finally { // code in finally block because always want // to return mapfiles, even when exception is thrown - dataSource.close(false); + dataSource.close(sawException); } } diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java index ce8cddc348..12890106d4 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletBase.java @@ -201,19 +201,20 @@ public abstract class TabletBase { Tablet.LookupResult result = null; + boolean sawException = false; try { SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource); this.lookupCount.incrementAndGet(); this.server.getScanMetrics().incrementLookupCount(1); result = lookup(iter, ranges, results, scanParams, maxResultSize); return result; - } catch (IOException ioe) { - dataSource.close(true); - throw ioe; + } catch (IOException | RuntimeException e) { + sawException = true; + throw e; } finally { // code in finally block because always want // to return mapfiles, even when exception is thrown - dataSource.close(false); + dataSource.close(sawException); synchronized (this) { queryResultCount.addAndGet(results.size());