This is an automated email from the ASF dual-hosted git repository. ctubbsii 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 b37b0626b6 fix parameter check and minor variable rename (#3176) b37b0626b6 is described below commit b37b0626b6aa2acd905b4df332d3f3b2c0d5d97b Author: EdColeman <d...@etcoleman.com> AuthorDate: Wed Feb 8 12:42:50 2023 -0500 fix parameter check and minor variable rename (#3176) Co-authored-by: Ed Coleman <edcole...@apache.org> --- .../core/client/ClientSideIteratorScanner.java | 8 +++---- .../accumulo/core/client/IsolatedScanner.java | 8 +++---- .../accumulo/core/client/impl/ScannerIterator.java | 2 +- .../accumulo/core/client/impl/ScannerOptions.java | 26 +++++++++++----------- .../core/client/impl/TabletServerBatchReader.java | 2 +- .../impl/TabletServerBatchReaderIterator.java | 12 +++++----- 6 files changed, 29 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java index ab05a12406..556bc431f1 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java @@ -239,8 +239,8 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner smi = new ScannerTranslatorImpl(scanner, scanner.getSamplerConfiguration()); this.range = scanner.getRange(); this.size = scanner.getBatchSize(); - this.timeOut = scanner.getTimeout(TimeUnit.MILLISECONDS); - this.batchTimeOut = scanner.getTimeout(TimeUnit.MILLISECONDS); + this.retryTimeout = scanner.getTimeout(TimeUnit.MILLISECONDS); + this.batchTimeout = scanner.getTimeout(TimeUnit.MILLISECONDS); this.readaheadThreshold = scanner.getReadaheadThreshold(); SamplerConfiguration samplerConfig = scanner.getSamplerConfiguration(); if (samplerConfig != null) @@ -257,8 +257,8 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner @Override public Iterator<Entry<Key,Value>> iterator() { smi.scanner.setBatchSize(size); - smi.scanner.setTimeout(timeOut, TimeUnit.MILLISECONDS); - smi.scanner.setBatchTimeout(batchTimeOut, TimeUnit.MILLISECONDS); + smi.scanner.setTimeout(retryTimeout, TimeUnit.MILLISECONDS); + smi.scanner.setBatchTimeout(batchTimeout, TimeUnit.MILLISECONDS); smi.scanner.setReadaheadThreshold(readaheadThreshold); if (isolated) smi.scanner.enableIsolation(); diff --git a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java index d4120ee4a0..6cd97de04f 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/IsolatedScanner.java @@ -228,8 +228,8 @@ public class IsolatedScanner extends ScannerOptions implements Scanner { public IsolatedScanner(Scanner scanner, RowBufferFactory bufferFactory) { this.scanner = scanner; this.range = scanner.getRange(); - this.timeOut = scanner.getTimeout(TimeUnit.MILLISECONDS); - this.batchTimeOut = scanner.getBatchTimeout(TimeUnit.MILLISECONDS); + this.retryTimeout = scanner.getTimeout(TimeUnit.MILLISECONDS); + this.batchTimeout = scanner.getBatchTimeout(TimeUnit.MILLISECONDS); this.batchSize = scanner.getBatchSize(); this.readaheadThreshold = scanner.getReadaheadThreshold(); this.bufferFactory = bufferFactory; @@ -237,8 +237,8 @@ public class IsolatedScanner extends ScannerOptions implements Scanner { @Override public Iterator<Entry<Key,Value>> iterator() { - return new RowBufferingIterator(scanner, this, range, timeOut, batchSize, readaheadThreshold, - bufferFactory); + return new RowBufferingIterator(scanner, this, range, retryTimeout, batchSize, + readaheadThreshold, bufferFactory); } @Deprecated diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java index bc80f5503a..3323de01d2 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java @@ -122,7 +122,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> { scanState = new ScanState(context, tableId, authorizations, new Range(range), options.fetchedColumns, size, options.serverSideIteratorList, options.serverSideIteratorOptions, isolated, readaheadThreshold, - options.getSamplerConfiguration(), options.batchTimeOut, options.classLoaderContext); + options.getSamplerConfiguration(), options.batchTimeout, options.classLoaderContext); // If we want to start readahead immediately, don't wait for hasNext to be called if (0l == readaheadThreshold) { diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java index 838d003c9c..e95764b15a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java @@ -49,9 +49,9 @@ public class ScannerOptions implements ScannerBase { protected SortedSet<Column> fetchedColumns = new TreeSet<>(); - protected long timeOut = Long.MAX_VALUE; + protected long retryTimeout = Long.MAX_VALUE; - protected long batchTimeOut = Long.MAX_VALUE; + protected long batchTimeout = Long.MAX_VALUE; private String regexIterName = null; @@ -178,7 +178,7 @@ public class ScannerOptions implements ScannerBase { dst.serverSideIteratorOptions.put(entry.getKey(), new HashMap<>(entry.getValue())); dst.samplerConfig = src.samplerConfig; - dst.batchTimeOut = src.batchTimeOut; + dst.batchTimeout = src.batchTimeout; } } } @@ -190,19 +190,19 @@ public class ScannerOptions implements ScannerBase { @Override public synchronized void setTimeout(long timeout, TimeUnit timeUnit) { - if (timeOut < 0) { - throw new IllegalArgumentException("TimeOut must be positive : " + timeOut); + if (timeout < 0) { + throw new IllegalArgumentException("retry timeout must be positive : " + timeout); } if (timeout == 0) - this.timeOut = Long.MAX_VALUE; + this.retryTimeout = Long.MAX_VALUE; else - this.timeOut = timeUnit.toMillis(timeout); + this.retryTimeout = timeUnit.toMillis(timeout); } @Override public synchronized long getTimeout(TimeUnit timeunit) { - return timeunit.convert(timeOut, TimeUnit.MILLISECONDS); + return timeunit.convert(retryTimeout, TimeUnit.MILLISECONDS); } @Override @@ -233,19 +233,19 @@ public class ScannerOptions implements ScannerBase { @Override public void setBatchTimeout(long timeout, TimeUnit timeUnit) { - if (timeOut < 0) { - throw new IllegalArgumentException("Batch timeout must be positive : " + timeOut); + if (timeout < 0) { + throw new IllegalArgumentException("Batch timeout must be positive : " + timeout); } if (timeout == 0) { - this.batchTimeOut = Long.MAX_VALUE; + this.batchTimeout = Long.MAX_VALUE; } else { - this.batchTimeOut = timeUnit.toMillis(timeout); + this.batchTimeout = timeUnit.toMillis(timeout); } } @Override public long getBatchTimeout(TimeUnit timeUnit) { - return timeUnit.convert(batchTimeOut, TimeUnit.MILLISECONDS); + return timeUnit.convert(batchTimeout, TimeUnit.MILLISECONDS); } @Override diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java index 12e2ec02ca..f6c0ffd49e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java @@ -117,6 +117,6 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan } return new TabletServerBatchReaderIterator(context, tableId, authorizations, ranges, numThreads, - queryThreadPool, this, timeOut); + queryThreadPool, this, retryTimeout); } } diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java index baf41c10b2..71fe392a4c 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java @@ -98,7 +98,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value private Map<String,TimeoutTracker> timeoutTrackers; private Set<String> timedoutServers; - private final long timeout; + private final long retryTimeout; private TabletLocator locator; @@ -108,7 +108,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value public TabletServerBatchReaderIterator(ClientContext context, String tableId, Authorizations authorizations, ArrayList<Range> ranges, int numThreads, - ExecutorService queryThreadPool, ScannerOptions scannerOptions, long timeout) { + ExecutorService queryThreadPool, ScannerOptions scannerOptions, long retryTimeout) { this.context = context; this.instance = context.getInstance(); @@ -119,12 +119,12 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value this.options = new ScannerOptions(scannerOptions); resultsQueue = new ArrayBlockingQueue<>(numThreads); - this.locator = new TimeoutTabletLocator(timeout, context, tableId); + this.locator = new TimeoutTabletLocator(retryTimeout, context, tableId); timeoutTrackers = Collections .synchronizedMap(new HashMap<String,TabletServerBatchReaderIterator.TimeoutTracker>()); timedoutServers = Collections.synchronizedSet(new HashSet<String>()); - this.timeout = timeout; + this.retryTimeout = retryTimeout; if (options.fetchedColumns.size() > 0) { ArrayList<Range> ranges2 = new ArrayList<>(ranges.size()); @@ -361,7 +361,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value try { TimeoutTracker timeoutTracker = timeoutTrackers.get(tsLocation); if (timeoutTracker == null) { - timeoutTracker = new TimeoutTracker(tsLocation, timedoutServers, timeout); + timeoutTracker = new TimeoutTracker(tsLocation, timedoutServers, retryTimeout); timeoutTrackers.put(tsLocation, timeoutTracker); } doLookup(context, tsLocation, tabletsRanges, tsFailures, unscanned, receiver, columns, @@ -686,7 +686,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value options.serverSideIteratorList, options.serverSideIteratorOptions, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()), waitForWrites, SamplerConfigurationImpl.toThrift(options.getSamplerConfiguration()), - options.batchTimeOut, options.classLoaderContext); + options.batchTimeout, options.classLoaderContext); if (waitForWrites) ThriftScanner.serversWaitedForWrites.get(ttype).add(server.toString());