You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/04/29 13:19:47 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #2665: Eventually Consistent scans / ScanServer feature

dlmarion opened a new pull request, #2665:
URL: https://github.com/apache/accumulo/pull/2665

   This commit builds on the changes added in prior commits
   8be98d6, 50b9267f, and 39bc7a0 to create a new server
   component that implements TabletHostingServer and uses
   the TabletScanClientService Thrift API to serve client
   scan requests on Tablets outside the TabletServer. To
   accomplish this the new server (ScanServer) constructs a
   new type of tablet called a SnapshotTablet which is comprised
   of the files in the metadata table and not the in-memory
   mutations that the TabletServer might contain. The Accumulo
   client has been modified to allow the user to set a flag
   on scans to make them eventually consistent, meaning that
   the user is ok with scanning data that may not be immediately
   consistent with the version of the Tablet that is being hosted
   by the TabletServer.
   
   This feature is optional and experimental.
   
   Closes #2411
   
   
   Co-authored-by: Keith Turner <kt...@apache.org>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939135839


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939180215


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerSelector.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.lang.reflect.Type;
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.data.TabletId;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+/**
+ * The default Accumulo selector for scan servers. This dispatcher will :
+ *
+ * <ul>
+ * <li>Hash each tablet to a per attempt configurable number of scan servers and then randomly
+ * choose one of those scan servers. Using hashing allows different client to select the same scan
+ * servers for a given tablet.</li>
+ * <li>Use a per attempt configurable busy timeout.</li>
+ * </ul>
+ *
+ * <p>
+ * This class accepts a single configuration that has a json value. To configure this class set
+ * {@code scan.server.selector.opts.profiles=<json>} in the accumulo client configuration along with
+ * the config for the class. The following is the default configuration value.
+ * </p>
+ * <p>
+ * {@value DefaultScanServerSelector#PROFILES_DEFAULT}
+ * </p>
+ *
+ * The json is structured as a list of profiles, with each profile having the following fields.
+ *
+ * <ul>
+ * <li><b>isDefault : </b> A boolean that specifies whether this is the default profile. One and
+ * only one profile must set this to true.</li>
+ * <li><b>maxBusyTimeout : </b> The maximum busy timeout to use. The busy timeout from the last
+ * attempt configuration grows exponentially up to this max.</li>
+ * <li><b>scanTypeActivations : </b> A list of scan types that will activate this profile. Scan
+ * types are specified by setting {@code scan_type=<scan_type>} as execution on the scanner. See
+ * {@link org.apache.accumulo.core.client.ScannerBase#setExecutionHints(Map)}</li>
+ * <li><b>group : </b> Scan servers can be started with an optional group. If specified, this option
+ * will limit the scan servers used to those that were started with this group name. If not
+ * specified, the set of scan servers that did not specify a group will be used. Grouping scan
+ * servers supports at least two use cases. First groups can be used to dedicate resources for
+ * certain scans. Second groups can be used to have different hardware/VM types for scans, for
+ * example could have some scans use expensive high memory VMs and others use cheaper burstable VMs.
+ * <li><b>attemptPlans : </b> A list of configuration to use for each scan attempt. Each list object
+ * has the following fields:
+ * <ul>
+ * <li><b>servers : </b> The number of servers to randomly choose from for this attempt.</li>
+ * <li><b>busyTimeout : </b> The busy timeout to use for this attempt.</li>
+ * <li><b>salt : </b> An optional string to append when hashing the tablet. When this is set
+ * differently for attempts it has the potential to cause the set of servers chosen from to be
+ * disjoint. When not set or the same, the servers between attempts will be subsets.</li>
+ * </ul>
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * Below is an example configuration with two profiles, one is the default and the other is used
+ * when the scan execution hint {@code scan_type=slow} is set.
+ * </p>
+ *
+ * <pre>
+ *    [
+ *     {
+ *       "isDefault":true,
+ *       "maxBusyTimeout":"5m",
+ *       "busyTimeoutMultiplier":4,
+ *       "attemptPlans":[
+ *         {"servers":"3", "busyTimeout":"33ms"},
+ *         {"servers":"100%", "busyTimeout":"100ms"}
+ *       ]
+ *     },
+ *     {
+ *       "scanTypeActivations":["slow"],
+ *       "maxBusyTimeout":"20m",
+ *       "busyTimeoutMultiplier":8,
+ *       "group":"lowcost",
+ *       "attemptPlans":[
+ *         {"servers":"1", "busyTimeout":"10s"},
+ *         {"servers":"3", "busyTimeout":"30s","salt":"42"},
+ *         {"servers":"9", "busyTimeout":"60s","salt":"84"}
+ *       ]
+ *     }
+ *    ]
+ * </pre>
+ *
+ * <p>
+ * For the default profile in the example it will start off by choosing randomly from 3 scan servers
+ * based on a hash of the tablet with no salt. For the first attempt it will use a busy timeout of
+ * 33 milliseconds. If the first attempt returns with busy, then it will randomly choose from 100%
+ * or all servers for the second attempt and use a busy timeout of 100ms. For subsequent attempts it
+ * will keep choosing from all servers and start multiplying the busy timeout by 4 until the max
+ * busy timeout of 4 minutes is reached.
+ * </p>
+ *
+ * <p>
+ * For the profile activated by {@code scan_type=slow} it start off by choosing randomly from 1 scan
+ * server based on a hash of the tablet with no salt and a busy timeout of 10s. The second attempt
+ * will choose from 3 scan servers based on a hash of the tablet plus the salt {@literal 42}.
+ * Without the salt, the single scan servers from the first attempt would always be included in the
+ * set of 3. With the salt the single scan server from the first attempt may not be included. The
+ * third attempt will choose a scan server from 9 using the salt {@literal 84} and a busy timeout of
+ * 60s. The different salt means the set of servers that attempts 2 and 3 choose from may be
+ * disjoint. Attempt 4 and greater will continue to choose from the same 9 servers as attempt 3 and
+ * will keep increasing the busy timeout by multiplying 8 until the maximum of 20 minutes is
+ * reached. For this profile it will choose from scan servers in the group {@literal lowcost}.
+ * </p>
+ */
+public class DefaultScanServerSelector implements ScanServerSelector {

Review Comment:
   Addressed in 9ec7f5f. Renamed class to ConfigurableScanServerSelector. I wasn't sure what to call the strategy.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939179808


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java:
##########
@@ -84,7 +85,8 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
         new ScanState(context, tableId, authorizations, new Range(range), options.fetchedColumns,
             size, options.serverSideIteratorList, options.serverSideIteratorOptions, isolated,
             readaheadThreshold, options.getSamplerConfiguration(), options.batchTimeOut,
-            options.classLoaderContext, options.executionHints);
+            options.classLoaderContext, options.executionHints,
+            (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) ? true : false);

Review Comment:
   Addressed in 9ec7f5f



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r940240058


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +392,51 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  public static class ScanServerInfo {
+    public final UUID uuid;
+    public final String group;
+
+    public ScanServerInfo(UUID uuid, String group) {
+      this.uuid = uuid;
+      this.group = group;
+    }
+
+  }

Review Comment:
   Removed class and replaced with Pair in d416900f14ac0c0e88af37aff49cb3b97a6a8173



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1152237400

   I kicked off a full build with ITs and the only ITs that failed are the same ones that are currently failing in `main`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1154733877

   @keith-turner wrote:
   > > It's not the scan execution hints that are modifying the behavior... it's the configured dispatcher. And, the scan hints are still not affecting the data returned... it's the server that it was dispatched to that is doing that.
   > 
   > If scan hints+config can change the behavior of a scanner from immediate to eventual I think this could lead to disaster. Consider something like the Accumulo GC algorithm where its correctness relies on only using scanners with immediate consistency. Consider the following situation.
   > 
   > * Person A writes a scanner that requires immediate consistency and sets a scan hint with intention of changing cache behavior to be opportunistic.
   > * Later Person B changes Accumulo configuration such that it causes the scan hints set by person A to now make the scanner coded by person A be eventually consistent.
   
   I think it's an exaggeration to call this a disaster. Scan hints controlling a specific configured dispatcher's behavior should already be documented in that dispatcher's documentation and stable before users can rely on it for stable behavior. This is not a problem. If we change the semantics of *any* configuration, we can break things users intended with their previous configuration. This situation is no different... scan hints are just configuration for a dispatcher that are semantically constrained to that dispatcher's documented behavior.
   
   Having this as behavior with an explicit API method to configure isn't any different. A configured dispatcher could just ignore that configuration and dispatch to an eventually consistent ScanServer instead of a TabletServer. Hints are just another kind of configuration. Whether that configuration is set by an API with a different name, or set by the API that sets hints, we're in the same situation... we have to trust the dispatcher that the user has configured to do the thing we expect it to, based on whatever configuration is set on the scan task, regardless of how it is set.
   
   The main difference here, is that it already logically makes sense to use scan hints to modify the dispatcher behavior, because that's what that configuration is *for*.
   
   > 
   > If the code in question were the Accumulo GC, this could cause files to be deleted when they should not be. Eventual vs immediate consistency is so important to some algorithms that it should always be explicitly declared per scanner and never be overridden by config that could impact all scanners in an indiscriminate manner without consideration of individual circumstances and per scanner intent.
   
   I think the discussion of the accumulo-gc is a bit of a red herring. That scans the metadata. It is already well documented that all metadata scans are always dispatched to an executor named "meta", and should always be immediately consistent. Even it if it wasn't, though, I don't think by setting the scan configuration via executor hints is substantially different than setting other scan configuration via other APIs. It's all configuration, and the dispatcher's behavior ultimately has to be documented, known, and relied upon in order to get any kind of guarantees about any scan results.
   
   
   @dlmarion wrote:
   > Since immediate/strict consistency is the default, maybe we just need a method to disable it for a specific query instead of specifying the value. For example, `enableEventualConsistency()`, `relaxReadGuarantees()`, `disableConsistentReads()`, `allowStaleReads()`, etc.
   > 
   > I'm also thinking that there should be a table configuration that enables/disables this feature. Currently, an admin can spin up some ScanServers and an application developer can enable eventual consistency, but do we want that on the `metadata` table for example?
   
   I would like to keep configuration simple. I've read so many articles about software complexity killing projects, and I think Accumulo is already in that risky area, where every new complex feature we add, often for niche use cases, adds an obtuse amount of complexity. We already have an overwhelming amount of single-purpose configuration elements that micro-manage so many elements of Accumulo's operations. We have an opportunity here to keep things *simple*. The dispatcher is already one such configurable component. If the dispatcher is responsible for deciding which server to use, and we already have a way to pass configuration to a dispatcher through the scan hints, then I don't see why we need to have additional configuration that add to the bloat. Let's be modular... let's let the configurable dispatcher to the work. We can add this feature without any additional user facing complexity... if we recognize that scan hints are merely dispatcher configuration, and the dispatcher 
 is already a pluggable module, and all of these configurations are already per-table or per-scan.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1158113117

   @keith-turner  - I think your latest commit left `ScanServerIT.testBatchScannerTimeout` in a broken state.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881550685


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java:
##########
@@ -358,7 +380,22 @@ else if (log.isTraceEnabled())
           }
 
           TraceUtil.setException(child2, e, false);
-          sleepMillis = pause(sleepMillis, maxSleepTime);
+          sleepMillis = pause(sleepMillis, maxSleepTime, scanState.runOnScanServer);
+        } catch (ScanServerBusyException e) {
+          error = "Scan failed, scan server was busy " + loc;

Review Comment:
   fix applied in ec7cbad



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ivakegg commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ivakegg commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r863767744


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {

Review Comment:
   I would love some class documentation that gives me an idea what this class is used for.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   Is it possible to have the consistency level set to EVENTUAL but have no scan servers?  I think I see that in that case the rebinToScanServers will delegate the scans to tservers.  But in that case we don't get the advantage of the code in the else statement.  I guess I am wondering whether the ScanServerData concept should be used no matter the consistency level and the rebinToScanServers will take care of delegating to scan servers vs tservers based on the consistency level.



##########
core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import java.util.UUID;
+
+import org.apache.accumulo.core.data.Value;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+public class ScanServerRefTabletFile extends TabletFile {
+
+  private final Value NULL_VALUE = new Value(new byte[0]);
+  private final Text colf;
+  private final Text colq;
+
+  public ScanServerRefTabletFile(String file, String serverAddress, UUID serverLockUUID) {
+    super(new Path(file));
+    this.colf = new Text(serverAddress);
+    this.colq = new Text(serverLockUUID.toString());
+  }
+
+  public ScanServerRefTabletFile(String file, Text colf, Text colq) {
+    super(new Path(file));
+    this.colf = colf;
+    this.colq = colq;
+  }
+
+  public String getRowSuffix() {
+    return this.getPathStr();
+  }
+
+  public Text getServerAddress() {
+    return this.colf;
+  }
+
+  public Text getServerLockUUID() {
+    return this.colq;
+  }
+
+  public Value getValue() {
+    return NULL_VALUE;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((colf == null) ? 0 : colf.hashCode());
+    result = prime * result + ((colq == null) ? 0 : colq.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ScanServerRefTabletFile other = (ScanServerRefTabletFile) obj;
+    if (colf == null) {

Review Comment:
   An EqualsBuilder may make this a little cleaner.



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors
+      long busyAttempts = params.getAttempts(tablet).stream()
+          .filter(sa -> sa.getResult() == ScanAttempt.Result.BUSY).count();
+
+      maxBusyAttempts = Math.max(maxBusyAttempts, busyAttempts);
+
+      String serverToUse = null;
+
+      var hashCode = hashTablet(tablet);
+
+      int numServers;
+
+      if (busyAttempts < maxDepth) {
+        numServers = (int) Math
+            .round(initialServers * Math.pow(orderedScanServers.size() / (double) initialServers,
+                busyAttempts / (double) maxDepth));
+      } else {
+        numServers = orderedScanServers.size();
+      }
+
+      int serverIndex =
+          (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServers)) % orderedScanServers.size();
+
+      // TODO could check if errors were seen on this server in past attempts

Review Comment:
   TODO ?



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors

Review Comment:
   TODO ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r866147045


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors

Review Comment:
   I'll take a look at this.



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors
+      long busyAttempts = params.getAttempts(tablet).stream()
+          .filter(sa -> sa.getResult() == ScanAttempt.Result.BUSY).count();
+
+      maxBusyAttempts = Math.max(maxBusyAttempts, busyAttempts);
+
+      String serverToUse = null;
+
+      var hashCode = hashTablet(tablet);
+
+      int numServers;
+
+      if (busyAttempts < maxDepth) {
+        numServers = (int) Math
+            .round(initialServers * Math.pow(orderedScanServers.size() / (double) initialServers,
+                busyAttempts / (double) maxDepth));
+      } else {
+        numServers = orderedScanServers.size();
+      }
+
+      int serverIndex =
+          (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServers)) % orderedScanServers.size();
+
+      // TODO could check if errors were seen on this server in past attempts

Review Comment:
   I'll also look at this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r883197611


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   >That makes sense if the desired primary outcome was "eventual consistency". It's not, though. It's merely an acceptable side-effect of what the user is really asking for, and not its essential/sufficient characteristic.
   
   I agree that a developer will enable eventual consistency because they want better performance and/or availability and they are ok with stale data.  One important thing to consider is that for evaluating code for correctness (like I am reviewing Accumulo code that someone else wrote) is that the consistency level of the data returned by the scanner is extremely important (probably more important than the performance benefits desired by the author).  When considering other names, the fact that a scanner may return stale data needs to be very clearly communicated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884057335


##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -395,6 +395,74 @@ public enum Property {
           + "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
           + "threshold is set greater than 0. Added with version 1.10",
       "1.10.0"),
+  // properties that are specific to scan server behavior
+  @Experimental
+  SSERV_PREFIX("sserver.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the scan servers", "2.1.0"),
+  @Experimental
+  SSERV_DATACACHE_SIZE("sserver.cache.data.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile data blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_INDEXCACHE_SIZE("sserver.cache.index.size", "25%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile index blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_SUMMARYCACHE_SIZE("sserver.cache.summary.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for summary data on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
+      "Specifies a default blocksize for the scan server caches", "2.1.0"),
+  @Experimental
+  SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
+      PropertyType.TIMEDURATION, "The time after which cached tablet metadata will be refreshed.",
+      "2.1.0"),
+  @Experimental
+  SSERV_PORTSEARCH("sserver.port.search", "true", PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available", "2.1.0"),
+  @Experimental
+  SSERV_CLIENTPORT("sserver.port.client", "9996", PropertyType.PORT,
+      "The port used for handling client connections on the tablet servers", "2.1.0"),
+  @Experimental
+  SSERV_MAX_MESSAGE_SIZE("sserver.server.message.size.max", "1G", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a scan server.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS("sserver.server.threads.minimum", "2", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS_TIMEOUT("sserver.server.threads.timeout", "10s", PropertyType.TIMEDURATION,

Review Comment:
   Based on the problems I saw in testing, I think this default should be different.  But not sure what it should be yet. Also want to go back compare the tserver and sserver thread pool behavior/creating/config to 1.10.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1151357227

   I started up a 2 node cluster from commit `fcd2738` and my accumulo-testing [branch ](https://github.com/dlmarion/accumulo-testing/tree/ci-verify-consistency-level) which includes changes to set the consistency level for the continuous walker, batch walker, scanner, and verify applications. I loaded data for an hour using `cingest ingest`, then used `cingest verify` jobs to test my changes in commits abcd8a7, 406df63, and c7dd148. The first job ran with consistency level `IMMEDIATE` (which uses the tserver) and the second job ran with `EVENTUAL` (which uses the sserver). I confirmed that the scans during the second job showed up on the Active Scans page in the monitor and confirmed that the metrics worked using Grafana. In the Grafana screenshot below you can see the end of the ingest process happening and then the two verify jobs, first one using the tserver and the second one using the sserver.
   
   ![Screenshot at 2022-06-09 12-27-22](https://user-images.githubusercontent.com/567911/172898698-a2c32a38-a731-499b-8ff5-a61256506e19.png)
   
   
   I should also note that the verify jobs both came back with the same results.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1149136177

   @dlmarion made a few suggestions about the testing.   I updated and the report and ran some new tests based on that.  The new tests results are in the report. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1138630511

   Using `set/getConsistencyLevel` on ScannerBase allows us to change the implementation without changing the API. Using a name tied to the implementation will cause API churn if the implementation changes. I'm not tied to `set/getConsistencyLevel`.
   
   I'm waiting for some consensus before change the API method names.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156539782

   Okay, thanks. That helps. I guess I was thinking the current dispatcher ran in the client side. So, using scan hints wouldn't even work the way I imagined it. We need the scan server dispatcher plugin to run on the client side first.
   
   A few thoughts based on my new understanding:
   
   1. The client side dispatcher concept is very different from the executor dispatching that is done in the tserver, but has a very similar name. It might be helpful to have this named completely differently... like "server chooser" or "tablet scanner server type selector" or something along those lines (not necessarily as verbose as that latter one... but something to make it clearly distinct from the executor pool dispatching inside the server).
   2. It would be nice if the scan server dispatching inside the scan server worked the same as the existing per-table dispatcher inside the tserver. Ideally, the same dispatcher would work in both servers, without a new dispatcher SPI specifically for the scan server.
   
   One thing is clear, my previous suggestion about using scan hints can be thrown out. Those are specifically scan executor hints, and should be used only by the dispatcher inside the server, because the dispatcher inside the server dispatches to an executor. The choice of which server type is an entirely different kind of dispatching, not to a scan executor, but to a server. So, that does imply a new kind of SPI or configuration to do the server selection inside the client (or... a different client entirely rather than modify the existing client).
   
   This is what I'm picturing now (very similar to yours, but tries to keep the servers similar, and makes server type selection more explicit):
   ![PXL_20220615_141638023](https://user-images.githubusercontent.com/1280725/173851044-09da651e-ca10-450b-a627-5c62cb13664d.jpg)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939087220


##########
server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java:
##########
@@ -176,6 +178,23 @@ public void execute(String[] args) throws Exception {
 
       }
 
+      if (opts.zapScanServers) {
+        String sserversPath = Constants.ZROOT + "/" + iid + Constants.ZSSERVERS;
+        try {
+          List<String> children = zoo.getChildren(sserversPath);
+          for (String child : children) {
+            message("Deleting " + sserversPath + "/" + child + " from zookeeper", opts);
+
+            var zLockPath = ServiceLock.path(sserversPath + "/" + child);
+            if (!zoo.getChildren(zLockPath.toString()).isEmpty()) {
+              ServiceLock.deleteLock(zoo, zLockPath);
+            }
+          }
+        } catch (Exception e) {
+          log.error("{}", e.getMessage(), e);

Review Comment:
   it follows the pattern of the existing code. I will create another issue to fix ZooZap in its entirety



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939136422


##########
server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.cli.ServerUtilOpts;
+
+public class ScanServerMetadataEntries {

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r940242670


##########
core/src/test/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParserTest.java:
##########
@@ -121,12 +131,21 @@ public void testShellOutput() throws Exception {
     ClusterConfigParser.outputShellVariables(contents, ps);
     ps.close();
 
-    Map<String,
-        String> expected = Map.of("MANAGER_HOSTS", "\"localhost1 localhost2\"", "MONITOR_HOSTS",
-            "\"localhost1 localhost2\"", "GC_HOSTS", "\"localhost\"", "TSERVER_HOSTS",
-            "\"localhost1 localhost2 localhost3 localhost4\"", "COORDINATOR_HOSTS",
-            "\"localhost1 localhost2\"", "COMPACTION_QUEUES", "\"q1 q2\"", "COMPACTOR_HOSTS_q1",
-            "\"localhost1 localhost2\"", "COMPACTOR_HOSTS_q2", "\"localhost3 localhost4\"");
+    Map<String,String> expected = new HashMap<>();
+    expected.put("MANAGER_HOSTS", "localhost1 localhost2");
+    expected.put("MONITOR_HOSTS", "localhost1 localhost2");
+    expected.put("GC_HOSTS", "localhost");
+    expected.put("TSERVER_HOSTS", "localhost1 localhost2 localhost3 localhost4");
+    expected.put("COORDINATOR_HOSTS", "localhost1 localhost2");
+    expected.put("COMPACTION_QUEUES", "q1 q2");
+    expected.put("COMPACTOR_HOSTS_q1", "localhost1 localhost2");
+    expected.put("COMPACTOR_HOSTS_q2", "localhost3 localhost4");
+    expected.put("SSERVER_GROUPS", "default highmem cheap");
+    expected.put("SSERVER_HOSTS_default", "localhost1 localhost2");
+    expected.put("SSERVER_HOSTS_highmem", "hmvm1 hmvm2 hmvm3");
+    expected.put("SSERVER_HOSTS_cheap", "burstyvm1 burstyvm2");
+
+    expected = Maps.transformValues(expected, v -> '"' + v + '"');

Review Comment:
   Removed use of Guava in d416900f14ac0c0e88af37aff49cb3b97a6a8173



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r899632712


##########
test/src/main/java/org/apache/accumulo/test/ScanServerIT.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+
+      // Timeout scan sessions after being idle for 3 seconds
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+
+      // Configure the scan server to only have 1 scan executor thread. This means
+      // that the scan server will run scans serially, not concurrently.
+      cfg.setProperty(Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS, "1");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        // if scanning against tserver would see the following, but should not on scan server
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testBatchScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (BatchScanner scanner = client.createBatchScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRanges(Collections.singletonList(new Range()));
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testScanOfflineTable() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+      client.tableOperations().offline(tableName, true);
+
+      assertThrows(TableOfflineException.class, () -> {
+        try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+          scanner.setRange(new Range());
+          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+          assertEquals(100, Iterables.size(scanner));
+        } // when the scanner is closed, all open sessions should be closed
+      });
+    }
+  }
+
+  @Test
+  @Disabled("Scanner.setTimeout does not work, issue #2606")
+  @Timeout(value = 20)
+  public void testScannerTimeout() throws Exception {
+    // Configure the client to use different scan server dispatcher property values
+    Properties props = getClientProps();
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "initialBusyTimeout",
+        "PT0.100S");
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "maxBusyTimeout",
+        "PT1.000S");
+
+    String tName = null;
+    try (AccumuloClient client = Accumulo.newClient().from(props).build()) {
+      tName = getUniqueNames(1)[0];
+      client.tableOperations().create(tName);
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tName);
+      client.tableOperations().flush(tName, null, null, true);
+
+      Scanner scanner = client.createScanner(tName, Authorizations.EMPTY);
+      IteratorSetting slow = new IteratorSetting(30, "slow", SlowIterator.class);
+      SlowIterator.setSleepTime(slow, 30000);
+      SlowIterator.setSeekSleepTime(slow, 30000);
+      scanner.addScanIterator(slow);
+      scanner.setRange(new Range());
+      scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+      scanner.setTimeout(10, TimeUnit.SECONDS);
+      Iterator<Entry<Key,Value>> iter = scanner.iterator();
+      if (iter.hasNext()) {
+        fail("Should not get here");
+      }
+    }
+  }
+
+  @Test
+  @Timeout(value = 20)
+  public void testBatchScannerTimeout() throws Exception {
+    // Configure the client to use different scan server dispatcher property values
+    Properties props = getClientProps();
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "initialBusyTimeout",
+        "PT0.100S");
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "maxBusyTimeout",
+        "PT1.000S");

Review Comment:
   ```suggestion
       Properties props = getClientProps();
       String profiles = "[{'isDefault':true,'maxBusyTimeout':'1s'," + "'busyTimeoutMultiplier':8, "
           + "'attemptPlans':[{'servers':'3', 'busyTimeout':'100ms'},"
           + "{'servers':'100%', 'busyTimeout':'100ms'}]}]";
       props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "profiles", profiles);
   ```



##########
test/src/main/java/org/apache/accumulo/test/ScanServerIT.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+
+      // Timeout scan sessions after being idle for 3 seconds
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+
+      // Configure the scan server to only have 1 scan executor thread. This means
+      // that the scan server will run scans serially, not concurrently.
+      cfg.setProperty(Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS, "1");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        // if scanning against tserver would see the following, but should not on scan server
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testBatchScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (BatchScanner scanner = client.createBatchScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRanges(Collections.singletonList(new Range()));
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testScanOfflineTable() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+      client.tableOperations().offline(tableName, true);
+
+      assertThrows(TableOfflineException.class, () -> {
+        try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+          scanner.setRange(new Range());
+          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+          assertEquals(100, Iterables.size(scanner));
+        } // when the scanner is closed, all open sessions should be closed
+      });
+    }
+  }
+
+  @Test
+  @Disabled("Scanner.setTimeout does not work, issue #2606")
+  @Timeout(value = 20)
+  public void testScannerTimeout() throws Exception {
+    // Configure the client to use different scan server dispatcher property values
+    Properties props = getClientProps();
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "initialBusyTimeout",
+        "PT0.100S");
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "maxBusyTimeout",
+        "PT1.000S");

Review Comment:
   ```suggestion
       String profiles = "[{'isDefault':true,'maxBusyTimeout':'1s'," + "'busyTimeoutMultiplier':8, "
           + "'attemptPlans':[{'servers':'3', 'busyTimeout':'100ms'},"
           + "{'servers':'100%', 'busyTimeout':'100ms'}]}]";
       props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "profiles", profiles);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881941373


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   My thinking on the naming is that there are multiple way to implement eventual consistency, with scan servers being one.  From the the API perspective I was thinking a user would declare they are ok with eventual consistency and that runtime configuration would determine how that actually executes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1140181288

   > I think the design of this change is great so far but it is a major new feature, with a drastic increase in complexity, touching major parts of Accumulo (scans, API, metadata, configuration, scripts, and introduces another new server). I don't think this should get merged into 2.1. The complexity of this change on top of all the changes already in 2.1 will only further delay the release of 2.1. Main already has many major new features (ZK Prop Store, Overhaul of Compactions code, External Compactions, AMPLE, Master Rename, New Tracing, New Metrics, New SPI, Root Table change) not to mention the 1,130+ tickets marked done for 2.1.
   
   I have been working really hard on testing this new feature over the past few weeks.  Tonight I finally got to a point where I was seeing scan servers work really well on a small cluster (12 scan servers, 3 tablets servers, 3 datanodes).  I was running 600 ish  random concurrent greps over random large ranges in a single tablet that scanned lots of data and returned little data.   To get to that point #2700, #2744, and #2745 needed to be debugged and fixed.  Each of these took a lot of time to find and fix. The interesting thing is that these problems were not all specific to scan servers, they were a result of all of the complex changes you mentioned and just happened to be found during intense scan server testing.   I think with or without scan servers that 2.1 need s a lot of testing to shake out more latent problems.  I think it would be nice to stop adding new features and release a 2.1.0-beta-1 and use it to do that testing and refining.  If we did that I would like to see s
 can servers in 2.1.0 as the last big feature.   The feature has gotten a good bit of review.  While working on this @dlmarion  and I reviewed each others work in addition to this review.  If anyone is interested, @dlmarion  and I could give a talk about the concepts on slack sometime in order to help guide anyone reviewing.
   
   In addition to reviews, the feature does need more testing.   Next week I hope to scale up the scan server testing to larger clusters now that I am getting small cluster to work well.  If anyone wants to help I would be happy to show you how to run scan server and some new test in kubernetes.  The source code for the testing I have been doing is [here](https://github.com/keith-turner/accumulo-testing/tree/scan-server-testing/sstest), its not documented but I would be happy to offer guidance on how to run it (need Kubernetes+Accumulo+Zookeper+DFS).  It supports multiple different test scenarios and I am running through those. 
   
   After that, for 2.1.0 in general I would like to set up a test scenario w/ continuous bulk import+external compactions+scan servers+heavy query load and compare that to continuous  bulk import+tservers only+heavy query load.  This testing could be done w/ a 2.1.0-beta-1 release possibly.  If anyone is interested in collaborating on that let me know, could definitely use help.
   
   I do agree that this new feature could break existing functionality unrelated to scan servers.  I am optimistic that we can mitigate this with good testing though.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939134389


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+
+public class ClientServiceEnvironmentImpl implements ServiceEnvironment {
+
+  private final ClientContext context;
+
+  public ClientServiceEnvironmentImpl(ClientContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.instanceOperations().getSystemConfiguration()));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting system configuration", e);
+    }
+  }
+
+  @Override
+  public Configuration getConfiguration(TableId tableId) {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.tableOperations().getConfiguration(getTableName(tableId))));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting table configuration", e);
+    }

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1155349342

   > Having this as behavior with an explicit API method to configure isn't any different. A configured dispatcher could just ignore that configuration and dispatch to an eventually consistent ScanServer instead of a TabletServer. 
   
   That can not happen in this PR as its currently coded.  When a scanner is set to immediate consistency the scan server dispatcher plugin is never consulted and only the tserver is used.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1155935221

   > > Having this as behavior with an explicit API method to configure isn't any different. A configured dispatcher could just ignore that configuration and dispatch to an eventually consistent ScanServer instead of a TabletServer.
   > 
   > That can not happen in this PR as its currently coded. When a scanner is set to immediate consistency the scan server dispatcher plugin is never consulted and only the tserver is used.
   
   So, there's two separate dispatching paths? And the path for the scan server bypasses the per-table configured dispatcher `table.scan.dispatcher`? That seems very confusing, especially since this PR adds new properties to configure `sserver.scan.executors`... but there's no way to select an executor via hints to a dispatcher? My understanding is that we wanted to try to mirror the tserver scan behavior as much as possible. Having two paths, without a dispatcher in the scan server path is very confusing. Have two paths, period, is very confusing. It just seems dramatically simpler to have a per-table dispatcher that is aware of scan servers and capable of dispatching to `sserver.scan.executors` (depending on hints) in addition to the usual `tserver.scan.executors`.
   
   Even if a dispatcher is added to the path for the scan server, it still seems far more complex than just relying on a single dispatcher... which also gives us the "enabled or disabled for a table" feature that Dave mentioned... because you can choose whether to configure a dispatcher that is scan server aware or not for a given table... without any new APIs.
   
   I'm probably completely misunderstanding how the dispatching changed in this PR. Is there a diagram or something to help illustrate the new dispatching design?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r882269924


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   > My thinking on the naming is that there are multiple way to implement eventual consistency, with scan servers being one. From the the API perspective I was thinking a user would declare they are ok with eventual consistency and that runtime configuration would determine how that actually executes.
   
   That makes sense if the desired primary outcome was "eventual consistency". It's not, though. It's merely an acceptable side-effect of what the user is really asking for, and not its essential/sufficient characteristic.
   
   This would be like calling the BatchScanner an UnorderedScanner. There are lots of ways to implement a scanner that returns data out of order. However, the primary characteristic in that case is that it scans batches of ranges in parallel, not that it's unordered. We've previously discussed that it really should be called ParallelScanner instead, because that is its essential characteristic. It is *necessarily* unordered, but it is *sufficient* to describe it as parallel, because all its secondary characteristics, including the fact that it returns results out of order, are merely side-effects of that essential characteristic. The same principle is applicable to the naming for this feature. We should really be naming it based on what is sufficient to describe its essential characteristics, not describing a subset of its necessary characteristics.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1153019085

   > It's not the scan execution hints that are modifying the behavior... it's the configured dispatcher. And, the scan hints are still not affecting the data returned... it's the server that it was dispatched to that is doing that.
   
   If scan hints+config can change the behavior of a scanner from immediate to eventual I think this could lead to disaster.  Consider something like the Accumulo GC algorithm where its correctness relies on only using scanners with immediate consistency.  Consider the following situation.
   
     * Person A writes a scanner that requires immediate consistency and sets a scan hint with intention of changing cache behavior to be opportunistic.
     * Later Person B changes Accumulo configuration such that it causes the scan hints set by person A to now make the scanner coded by person A be eventually consistent.
   
   If the code in question were the Accumulo GC, this could cause files to be deleted when they should not be.  Eventual vs immediate consistency is so important to some algorithms that it should always be explicitly declared per scanner and never be overridden by config that could impact all scanners in an indiscriminate manner without consideration of individual circumstances and per scanner intent.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1152820616

   > I'm thinking it would be better to leverage the scan hints to control a ScanServer-aware dispatcher, rather than add a new API for the consistency level.
   
   I think this goes against the purpose of scan execution hints.  They were created to modify execution behavior like priority, caching, and thread pool selection.  They were never intended to change anything about data returned, it says so in their javadocs.
   
   https://github.com/apache/accumulo/blob/d5f81877fcc794c8158f38b840d02331e3c563dc/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java#L342-L361
   
   Slightly related I created a new default scan server dispatcher.  Its currently a PR against Dave's branch: dlmarion/accumulo#29.  When running 100+ test scenarios this is what I realized I wanted.  I wish I had had it when running all of those test,  I could have a run a few more test that I wanted to but could not.  This new dispatcher is completely configuration driven (replacing the algorithm the previous default dispatcher had) and can be influenced by scan execution hints.  If we merge this PR, I could close the PR on Dave's fork and make a PR on the main Accumulo GH.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] milleruntime commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
milleruntime commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r938999875


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -373,6 +384,23 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
     }
   }
 
+  /**
+   * Get the configured consistency level
+   *
+   * @return consistency level
+   * @since 2.1.0
+   */
+  public ConsistencyLevel getConsistencyLevel();
+
+  /**
+   * Set the desired consistency level for this scanner.
+   *
+   * @param level
+   *          consistency level
+   * @since 2.1.0
+   */
+  public void setConsistencyLevel(ConsistencyLevel level);
+

Review Comment:
   What about just a single method like `useSnapshot()` which will override the default behavior of the Scanner. This is less confusing to me then using the Consistency terminology and doesn't require an Enum. From a user perspective, I think the idea that you are using a snapshot of the data from some previous point in time is pretty clear and easier to explain.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939133882


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+
+public class ClientServiceEnvironmentImpl implements ServiceEnvironment {
+
+  private final ClientContext context;
+
+  public ClientServiceEnvironmentImpl(ClientContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.instanceOperations().getSystemConfiguration()));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting system configuration", e);

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884057335


##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -395,6 +395,74 @@ public enum Property {
           + "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
           + "threshold is set greater than 0. Added with version 1.10",
       "1.10.0"),
+  // properties that are specific to scan server behavior
+  @Experimental
+  SSERV_PREFIX("sserver.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the scan servers", "2.1.0"),
+  @Experimental
+  SSERV_DATACACHE_SIZE("sserver.cache.data.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile data blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_INDEXCACHE_SIZE("sserver.cache.index.size", "25%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile index blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_SUMMARYCACHE_SIZE("sserver.cache.summary.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for summary data on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
+      "Specifies a default blocksize for the scan server caches", "2.1.0"),
+  @Experimental
+  SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
+      PropertyType.TIMEDURATION, "The time after which cached tablet metadata will be refreshed.",
+      "2.1.0"),
+  @Experimental
+  SSERV_PORTSEARCH("sserver.port.search", "true", PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available", "2.1.0"),
+  @Experimental
+  SSERV_CLIENTPORT("sserver.port.client", "9996", PropertyType.PORT,
+      "The port used for handling client connections on the tablet servers", "2.1.0"),
+  @Experimental
+  SSERV_MAX_MESSAGE_SIZE("sserver.server.message.size.max", "1G", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a scan server.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS("sserver.server.threads.minimum", "2", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS_TIMEOUT("sserver.server.threads.timeout", "10s", PropertyType.TIMEDURATION,

Review Comment:
   Based on the problems I saw in testing, I think this default should be different.  But not sure what it should be yet.



##########
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java:
##########
@@ -474,6 +474,8 @@ public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long
   public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority,
       boolean emitThreadPoolMetrics) {
+    LOG.debug("Creating ThreadPoolExecutor for {} with {} core threads and {} max threads", name,
+        coreThreads, maxThreads);

Review Comment:
   ```suggestion
       LOG.debug(
           "Creating ThreadPoolExecutor for {} with {} core threads and {} max threads {} {} timeout",
           name, coreThreads, maxThreads, timeOut, units);
   ```
   
   During testing of scan servers under high load I was seeing really erratic behavior. I noticed that the thread pool that serviced RPC request was cycling through lots of threads (like 1,000's of thread being created and destroyed in the pool per minute).  Turns out the thread pool timeout defaults to 10s.  I changed it to zero in my config for no thread pool timeout and the performance problems and erratic behavior went away.  I don't fully understand the problem at this point, but adding the timeout to this debug is useful.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r882040710


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   The ScanServer serves data from a Tablet's files based on the Tablet metadata in the cache in the ScanServer. The duration for which the metadata is cached (and then refreshed) is based on the property SSERV_CACHED_TABLET_METADATA_EXPIRATION. When the Tablet metadata cache in the ScanServer is refreshed, then the current set of files (newly bulk loaded or compacted) will then be available. Does this answer your question @wjsl  ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881939327


##########
test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java:
##########
@@ -131,12 +131,12 @@ public void closeMultiScan(TInfo tinfo, long scanID) {}
     public void closeScan(TInfo tinfo, long scanID) {}
 
     @Override
-    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID) {
+    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID, long busyTimeout) {

Review Comment:
   It could, but it was only added for the scanserver.  Not sure it would be make sense on its own.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881549909


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java:
##########
@@ -253,7 +279,9 @@ private ScanResult continueScan(TInfo tinfo, long scanID, SingleScanSession scan
     } catch (CancellationException ce) {
       server.getSessionManager().removeSession(scanID);
       Tablet tablet = server.getOnlineTablet(scanSession.extent);

Review Comment:
   Fix applied in ec7cbad



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1147458786

   I can't get the ContinuousVerify M/R job in https://github.com/dlmarion/accumulo-testing/tree/ci-verify-consistency-level to use the ScanServer. I have modifed the AccumuloInputFormat configuration several times, I must be missing something. The CI scan, walk, and batch walk do use the ScanServer. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1148520608

   I opened #2757 for the M/R issue above.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ivakegg commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ivakegg commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r866144092


##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);
+
+      int count2 = 0;
+      while (iter2.hasNext()) {
+        iter2.next();
+        count2++;
+      }
+      assertEquals(1000, count2);
+
+      scanner1.close();
+
+      try (Scanner scanner2 = client.createScanner(tableName, Authorizations.EMPTY)) {
+        assertEquals(1100, Iterables.size(scanner2));
+      }
+    }
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheDisabled() throws Exception {
+
+    startScanServer(false);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1100 k/v because the tablet metadata is not cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {

Review Comment:
   same comment as above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156644517

   > It would be nice if the scan server dispatching inside the scan server worked the same as the existing per-table dispatcher inside the tserver. Ideally, the same dispatcher would work in both servers, without a new dispatcher SPI specifically for the scan server.
   
   The plugins in the scan server does work exactly the same as in the tserver with these changes.  The only difference is they have a different set of config prefixes (well the ones that are not per table).  Config for dispatching to thread pools is per table and has the exact same config across tserver and sserver.  Config for creating thread pools and cache to service scan is at the server level and therefore has  diff config prefix for sserver but the exact same config props.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r900061736


##########
test/src/main/java/org/apache/accumulo/test/ScanServerIT.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableOfflineException;
+import org.apache.accumulo.core.client.TimedOutException;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.accumulo.test.functional.SlowIterator;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+
+      // Timeout scan sessions after being idle for 3 seconds
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+
+      // Configure the scan server to only have 1 scan executor thread. This means
+      // that the scan server will run scans serially, not concurrently.
+      cfg.setProperty(Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS, "1");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        // if scanning against tserver would see the following, but should not on scan server
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testBatchScan() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (BatchScanner scanner = client.createBatchScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRanges(Collections.singletonList(new Range()));
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+        ReadWriteIT.ingest(client, 10, 10, 50, 10, tableName);
+        assertEquals(100, Iterables.size(scanner));
+        scanner.setConsistencyLevel(ConsistencyLevel.IMMEDIATE);
+        assertEquals(200, Iterables.size(scanner));
+      } // when the scanner is closed, all open sessions should be closed
+    }
+  }
+
+  @Test
+  public void testScanOfflineTable() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+      client.tableOperations().offline(tableName, true);
+
+      assertThrows(TableOfflineException.class, () -> {
+        try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+          scanner.setRange(new Range());
+          scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+          assertEquals(100, Iterables.size(scanner));
+        } // when the scanner is closed, all open sessions should be closed
+      });
+    }
+  }
+
+  @Test
+  @Disabled("Scanner.setTimeout does not work, issue #2606")
+  @Timeout(value = 20)
+  public void testScannerTimeout() throws Exception {
+    // Configure the client to use different scan server dispatcher property values
+    Properties props = getClientProps();
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "initialBusyTimeout",
+        "PT0.100S");
+    props.put(ClientProperty.SCAN_SERVER_DISPATCHER_OPTS_PREFIX.getKey() + "maxBusyTimeout",
+        "PT1.000S");

Review Comment:
   applied. Thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r940254230


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A client side plugin that determines what scan servers to use for eventually consistent scans.
+ * When a scanner sets
+ * {@link org.apache.accumulo.core.client.ScannerBase#setConsistencyLevel(ScannerBase.ConsistencyLevel)}
+ * to {@link org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel#EVENTUAL} then this plugin
+ * is used to determine which scan servers to use for a given tablet. To configure a class to use
+ * for this plugin set its name using the client config {@code scan.server.selector.impl}
+ *
+ * @since 2.1.0
+ */
+public interface ScanServerSelector {
+
+  /**
+   * The scan server group name that will be used when one is not specified.
+   */
+  public static final String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
+
+  /**
+   * Information about a scan server.
+   *
+   * @since 2.1.0
+   */
+  public interface ScanServer {
+    /**
+     * @return the address in the form of {@code <host>:<port>} where the scan server is running.
+     */
+    String getAddress();
+
+    /**
+     * @return the group name set when the scan server was started. If a group name was not set for
+     *         the scan server, then the string {@value #DEFAULT_SCAN_SERVER_GROUP_NAME} is
+     *         returned.
+     */
+    String getGroup();
+  }
+
+  /**
+   * This interface exists so that is easier to evolve what is passed to
+   * {@link #init(InitParameters)} without having to make breaking changes.
+   *
+   * @since 2.1.0
+   */
+  public interface InitParameters {

Review Comment:
   Created issue #2855 to determine whether moving the interfaces to their own files makes sense. I'm 50-50 on this personally.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r865139710


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftScanClientHandler.java:
##########
@@ -253,7 +279,9 @@ private ScanResult continueScan(TInfo tinfo, long scanID, SingleScanSession scan
     } catch (CancellationException ce) {
       server.getSessionManager().removeSession(scanID);
       Tablet tablet = server.getOnlineTablet(scanSession.extent);

Review Comment:
   Ran into an exception while testing.  Think the following is a fix for it.
   
   ```suggestion
         TabletBase tablet = scanSession.getTabletResolver().getTablet(scanSession.extent);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r865367569


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   > But in that case we don't get the advantage of the code in the else statement.
   
   That was an intentional decision to leave the else stmt just for the tserver/immediate path. Was thinking for the eventual path that all control is delegated to the ScanServerDispatcher plugin avoiding hardcoded decisions.  The ScanServerDispatcher plugin does not currently support sending multiple threads to a single scan server (that is what the else code is doing for tservers).  I had wanted to refactor the ScanServerDispatcher to support this use case, but did not get to it.  I think this would be worthwhile and could look into that after this PR is merged. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1152760227

   I'm thinking it would be better to leverage the scan hints to control a ScanServer-aware dispatcher, rather than add a new API for the consistency level.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884060596


##########
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java:
##########
@@ -474,6 +474,8 @@ public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long
   public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority,
       boolean emitThreadPoolMetrics) {
+    LOG.debug("Creating ThreadPoolExecutor for {} with {} core threads and {} max threads", name,
+        coreThreads, maxThreads);

Review Comment:
   This is where I configured the timeout to zero for my scan servers to avoid this problem.  
   
   https://github.com/keith-turner/accumulo-testing/blob/322b32c62113b974b9dd18404d0e57902dc13b8e/sstest/accumulo-scanservers.yaml#L19



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r880853288


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   The names should imply "use an eventually consistent scan server" instead of "create an eventually consistent scanner". In other words, the name should explicitly declare the resource that the scanner is allowed to use, rather than a characteristic trait of the scanner itself. The scanner characteristics are not inherent to the scanner, but the result of using the particular server type.
   
   For the shell, `--use-scan-server` (with description that says scan servers offer eventual consistency) would be better than `--consistency eventual` on the scanner.
   
   For the Scanner API, perhaps `permitScanServer()`, which allows the scanner to use scan server's, if they are available, with a javadoc that explains that scan servers are eventually consistent.
   
   This changes the way we think about the Scanner object, based on the resources it is using (and characteristics that result from using those resources; eventual consistency is only one such characteristic... other characteristics are also applicable... like potential for less resource contention on the server-side, potentially dynamically scalable, etc.), rather than altering the inherent characteristics of the Scanner object itself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881687226


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.Reference;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.gc.GCRun;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
+
+  private static class ScanServerMetadataEntriesITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerMetadataEntriesITConfiguration c = new ScanServerMetadataEntriesITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  @Test
+  public void testServerContextMethods() throws Exception {
+
+    try (AccumuloClient ac = Accumulo.newClient().from(getClientProps()).build()) {
+      HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+      UUID serverLockUUID = UUID.randomUUID();
+
+      String[] files =
+          new String[] {"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+              "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf"};
+
+      Set<ScanServerRefTabletFile> scanRefs = new HashSet<>();
+      for (String file : files) {
+        scanRefs.add(new ScanServerRefTabletFile(file, server.toString(), serverLockUUID));
+      }
+
+      ServerContext ctx = getCluster().getServerContext();
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      Set<ScanServerRefTabletFile> scanRefs2 =
+          ctx.getAmple().getScanServerFileReferences().collect(Collectors.toSet());
+
+      assertEquals(scanRefs, scanRefs2);
+
+      ctx.getAmple().deleteScanServerFileReferences("127.0.0.1:1234", serverLockUUID);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().deleteScanServerFileReferences(scanRefs);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+    }
+  }
+
+  @Test
+  public void testScanServerMetadataEntries() throws Exception {
+
+    ServerContext ctx = getCluster().getServerContext();
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);

Review Comment:
   This suggestion causes the test to fail with:
   ```
   org.apache.accumulo.core.client.AccumuloSecurityException: Error USER_DOESNT_EXIST for user !SYSTEM - The user does not exist
   	at org.apache.accumulo.core.rpc.clients.TServerClient.executeVoid(TServerClient.java:138)
   	at org.apache.accumulo.core.rpc.clients.ClientServiceThriftClient.executeVoid(ClientServiceThriftClient.java:58)
   	at org.apache.accumulo.core.clientImpl.SecurityOperationsImpl.executeVoid(SecurityOperationsImpl.java:69)
   	at org.apache.accumulo.core.clientImpl.SecurityOperationsImpl.changeUserAuthorizations(SecurityOperationsImpl.java:186)
   	at org.apache.accumulo.test.TestIngest.ingest(TestIngest.java:279)
   	at org.apache.accumulo.test.TestIngest.ingest(TestIngest.java:402)
   	at org.apache.accumulo.test.functional.ReadWriteIT.ingest(ReadWriteIT.java:203)
   	at org.apache.accumulo.test.functional.ReadWriteIT.ingest(ReadWriteIT.java:192)
   ...
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r880783668


##########
test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java:
##########
@@ -131,12 +131,12 @@ public void closeMultiScan(TInfo tinfo, long scanID) {}
     public void closeScan(TInfo tinfo, long scanID) {}
 
     @Override
-    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID) {
+    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID, long busyTimeout) {

Review Comment:
   Could this addition of a `busyTimeout` parameter be done in a separate distinct smaller change first?



##########
test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMultipleScansIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+    }
+  }
+
+  private static final int NUM_SCANS = 4;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testMutipleScansSameTablet() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      var executor = Executors.newCachedThreadPool();
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRange(new Range());
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+      executor.shutdown();
+    }
+  }
+
+  @Test
+  public void testSingleScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+      }
+    }
+  }
+
+  @Test
+  public void testMultipleScansDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+      System.out.println(splits);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      final AtomicInteger counter = new AtomicInteger(0);
+
+      var executor = Executors.newCachedThreadPool();

Review Comment:
   Creating this threadpool could be done in a `@BeforeEach` method, and shutting it down could be done in an `@AfterEach`. If there's any other common setup/teardown code, moving it into a before/after method will make the test smaller and easier to understand what is being tested.



##########
test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.Reference;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.gc.GCRun;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
+
+  private static class ScanServerMetadataEntriesITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerMetadataEntriesITConfiguration c = new ScanServerMetadataEntriesITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  @Test
+  public void testServerContextMethods() throws Exception {
+
+    try (AccumuloClient ac = Accumulo.newClient().from(getClientProps()).build()) {
+      HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+      UUID serverLockUUID = UUID.randomUUID();
+
+      String[] files =
+          new String[] {"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+              "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf"};
+
+      Set<ScanServerRefTabletFile> scanRefs = new HashSet<>();
+      for (String file : files) {
+        scanRefs.add(new ScanServerRefTabletFile(file, server.toString(), serverLockUUID));
+      }
+
+      ServerContext ctx = getCluster().getServerContext();
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      Set<ScanServerRefTabletFile> scanRefs2 =
+          ctx.getAmple().getScanServerFileReferences().collect(Collectors.toSet());
+
+      assertEquals(scanRefs, scanRefs2);
+
+      ctx.getAmple().deleteScanServerFileReferences("127.0.0.1:1234", serverLockUUID);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().deleteScanServerFileReferences(scanRefs);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+    }
+  }
+
+  @Test
+  public void testScanServerMetadataEntries() throws Exception {
+
+    ServerContext ctx = getCluster().getServerContext();
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);

Review Comment:
   ServerContext is a ClientContext is an AccumuloClient. So, constructing a separate client is a bit unnecessary.
   
   ```suggestion
       ServerContext client = getCluster().getServerContext();
       String tableName = getUniqueNames(1)[0];
   
       client.tableOperations().create(tableName);
   ```



##########
test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.Reference;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.gc.GCRun;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
+
+  private static class ScanServerMetadataEntriesITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerMetadataEntriesITConfiguration c = new ScanServerMetadataEntriesITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  @Test
+  public void testServerContextMethods() throws Exception {
+
+    try (AccumuloClient ac = Accumulo.newClient().from(getClientProps()).build()) {
+      HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+      UUID serverLockUUID = UUID.randomUUID();
+
+      String[] files =
+          new String[] {"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+              "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf"};
+
+      Set<ScanServerRefTabletFile> scanRefs = new HashSet<>();
+      for (String file : files) {
+        scanRefs.add(new ScanServerRefTabletFile(file, server.toString(), serverLockUUID));
+      }

Review Comment:
   This code would be much smaller if you used streams:
   ```java
     Set<ScanServerRefTabletFile> scanRefs = Stream.of("F0000070.rf", "F0000071.rf")
         .map(f -> "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/" + f)
         .map(f -> new ScanServerRefTabletFile(f, server.toString(), serverLockUUID))
         .collect(Collectors.toSet());
   ```
   



##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   I'm not a fan of using "immediate" and "eventual" as the user-facing control knob. Those characteristics are [*necessary* conditions of using one server type vs. another, but they are not *sufficient* conditions to communicate which resources you're relying on to execute the scan](https://en.wikipedia.org/wiki/Necessity_and_sufficiency).
   
   In general, I think API terminology should be *sufficient* to communicate to the interfacing user/component what they are interfacing with.
   
   So, I think it would be better to explicitly ask for a scan server, and include in the docs that scan servers have an eventual consistency.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanAttemptsImpl.class);
+
+  static class ScanAttemptImpl
+      implements org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt {

Review Comment:
   This class is imported, yet it's still using a fully qualified class name here.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   It's weird to ask for eventual consistency. Nobody *wants* eventual consistency. Eventual consistency is always tolerated, never desired.



##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -373,6 +384,23 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
     }
   }
 
+  /**
+   * Get the configured consistency level
+   *
+   * @return consistency level
+   * @since 2.1.0
+   */
+  public ConsistencyLevel getConsistencyLevel();
+
+  /**
+   * Set the desired consistency level for this scanner.
+   *
+   * @param level
+   *          consistency level
+   * @since 2.1.0
+   */
+  public void setConsistencyLevel(ConsistencyLevel level);
+

Review Comment:
   See my previous comment about *necessary* and *sufficient* API terminology. I think interfacing with a "consistency level" is insufficient terminology for the API.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +340,75 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  /**
+   * @return map of live scan server addresses to lock uuids.
+   */
+  public Map<String,UUID> getScanServers() {
+    Map<String,UUID> liveScanServers = new HashMap<>();
+    String root = this.getZooKeeperRoot() + Constants.ZSSERVERS;
+    var addrs = this.getZooCache().getChildren(root);
+    for (String addr : addrs) {
+      try {
+        final var zLockPath = ServiceLock.path(root + "/" + addr);
+        ZcStat stat = new ZcStat();
+        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (lockData != null) {
+          UUID uuid = UUID.fromString(new String(lockData, UTF_8));
+          liveScanServers.put(addr, uuid);
+        }
+      } catch (Exception e) {
+        log.error("Error validating zookeeper scan server node: " + addr, e);
+      }
+    }
+    return liveScanServers;
+  }
+
+  /**
+   * @return the scan server dispatcher implementation used for determining which scan servers will
+   *         be used when performing an eventually consistent scan
+   */
+  public synchronized ScanServerDispatcher getScanServerDispatcher() {
+    ensureOpen();
+    if (scanServerDispatcher == null) {
+      String clazz = ClientProperty.SCAN_SERVER_DISPATCHER.getValue(info.getProperties());

Review Comment:
   You should avoid synchronization and lazily instantiate this by following the memoized Supplier mechanism that was done elsewhere in this class for similar objects.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanAttemptsImpl.class);
+
+  static class ScanAttemptImpl
+      implements org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt {
+
+    private final String server;
+    private final long time;
+    private final Result result;
+    private volatile long mutationCount = Long.MAX_VALUE;
+
+    ScanAttemptImpl(Result result, String server, long time) {
+      this.result = result;
+      this.server = Objects.requireNonNull(server);
+      this.time = time;
+    }
+
+    @Override
+    public String getServer() {
+      return server;
+    }
+
+    @Override
+    public long getEndTime() {
+      return time;
+    }
+
+    @Override
+    public Result getResult() {
+      return result;
+    }
+
+    private void setMutationCount(long mc) {
+      this.mutationCount = mc;
+    }
+
+    public long getMutationCount() {
+      return mutationCount;
+    }
+  }
+
+  private Map<TabletId,Collection<ScanAttemptImpl>> attempts = new ConcurrentHashMap<>();
+  private long mutationCounter = 0;
+
+  private void add(TabletId tablet, ScanAttempt.Result result, String server, long endTime) {
+
+    ScanAttemptImpl sa = new ScanAttemptImpl(result, server, endTime);
+
+    attempts.computeIfAbsent(tablet, k -> ConcurrentHashMap.newKeySet()).add(sa);
+
+    synchronized (this) {
+      // now that the scan attempt obj is added to all concurrent data structs, make it visible
+      // need to atomically increment the counter AND set the counter on the object
+      sa.setMutationCount(mutationCounter++);
+    }
+
+  }
+
+  public static interface ScanAttemptReporter {
+    void report(ScanAttempt.Result result);
+  }
+
+  ScanAttemptReporter createReporter(String server, TabletId tablet) {
+    return new ScanAttemptReporter() {
+      @Override
+      public void report(ScanAttempt.Result result) {
+        LOG.trace("Received result: {}", result);
+        add(tablet, result, server, System.currentTimeMillis());
+      }
+    };
+  }
+
+  Map<TabletId,Collection<ScanAttemptImpl>> snapshot() {
+    // allows only seeing scan attempt objs that were added before this call
+
+    long snapMC;
+    synchronized (ScanAttemptsImpl.this) {
+      snapMC = mutationCounter;
+    }
+    var tmp = Maps.transformValues(attempts, tabletAttemptList -> Collections2
+        .filter(tabletAttemptList, sai -> sai.getMutationCount() < snapMC));
+
+    return Maps.filterEntries(tmp, entry -> !entry.getValue().isEmpty());

Review Comment:
   This appears to be using Guava-specific Functional APIs. It's better to use Java built-in Functional APIs/Streams instead.



##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);
+
+      int count2 = 0;
+      while (iter2.hasNext()) {
+        iter2.next();
+        count2++;
+      }
+      assertEquals(1000, count2);
+
+      scanner1.close();
+
+      try (Scanner scanner2 = client.createScanner(tableName, Authorizations.EMPTY)) {
+        assertEquals(1100, Iterables.size(scanner2));
+      }
+    }
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheDisabled() throws Exception {
+
+    startScanServer(false);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1100 k/v because the tablet metadata is not cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);

Review Comment:
   Pretty sure you could just do the following one-liner instead of dealing with looping over an iterator.
   
   ```java
   assertEquals(1000, scanner1.stream().count());
   ```



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -612,6 +741,7 @@ void errorOccured() {
     }
 
     /**
+     *
      */

Review Comment:
   :+1:  :smiley_cat: 
   
   This was probably an editor thing. But, this empty javadoc can be deleted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1138619325

   > > It's weird to ask for eventual consistency. Nobody wants eventual consistency. Eventual consistency is always tolerated, never desired.
   > 
   > Eventual consistency for reads is desired in this case
   
   No, I don't think so. Like you said yourself, what the user desires is increased availability. Eventual consistency is the trade-off they tolerate to get increased availability. The only scenarios where I can imagine an actual *preference* for the possibility of stale data are nefarious ones where someone is trying to exploit or deceive on the basis of old information.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r886196192


##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -395,6 +395,74 @@ public enum Property {
           + "indefinitely. Default is 0 to block indefinitely. Only valid when tserver available "
           + "threshold is set greater than 0. Added with version 1.10",
       "1.10.0"),
+  // properties that are specific to scan server behavior
+  @Experimental
+  SSERV_PREFIX("sserver.", null, PropertyType.PREFIX,
+      "Properties in this category affect the behavior of the scan servers", "2.1.0"),
+  @Experimental
+  SSERV_DATACACHE_SIZE("sserver.cache.data.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile data blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_INDEXCACHE_SIZE("sserver.cache.index.size", "25%", PropertyType.MEMORY,
+      "Specifies the size of the cache for RFile index blocks on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_SUMMARYCACHE_SIZE("sserver.cache.summary.size", "10%", PropertyType.MEMORY,
+      "Specifies the size of the cache for summary data on each scan server.", "2.1.0"),
+  @Experimental
+  SSERV_DEFAULT_BLOCKSIZE("sserver.default.blocksize", "1M", PropertyType.BYTES,
+      "Specifies a default blocksize for the scan server caches", "2.1.0"),
+  @Experimental
+  SSERV_CACHED_TABLET_METADATA_EXPIRATION("sserver.cache.metadata.expiration", "5m",
+      PropertyType.TIMEDURATION, "The time after which cached tablet metadata will be refreshed.",
+      "2.1.0"),
+  @Experimental
+  SSERV_PORTSEARCH("sserver.port.search", "true", PropertyType.BOOLEAN,
+      "if the ports above are in use, search higher ports until one is available", "2.1.0"),
+  @Experimental
+  SSERV_CLIENTPORT("sserver.port.client", "9996", PropertyType.PORT,
+      "The port used for handling client connections on the tablet servers", "2.1.0"),
+  @Experimental
+  SSERV_MAX_MESSAGE_SIZE("sserver.server.message.size.max", "1G", PropertyType.BYTES,
+      "The maximum size of a message that can be sent to a scan server.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS("sserver.server.threads.minimum", "2", PropertyType.COUNT,
+      "The minimum number of threads to use to handle incoming requests.", "2.1.0"),
+  @Experimental
+  SSERV_MINTHREADS_TIMEOUT("sserver.server.threads.timeout", "10s", PropertyType.TIMEDURATION,

Review Comment:
   ```suggestion
     SSERV_MINTHREADS_TIMEOUT("sserver.server.threads.timeout", "0s", PropertyType.TIMEDURATION,
   ```
   
   Make this consistent w/ tserver setting as mentioned in #2745



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884060098


##########
assemble/conf/accumulo-env.sh:
##########
@@ -94,6 +94,7 @@ case "$cmd" in
   tserver) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx768m' '-Xms768m') ;;
   compaction-coordinator) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx512m' '-Xms512m') ;;
   compactor) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx256m' '-Xms256m') ;;
+  sserver) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx512m' '-Xms512m') ;;

Review Comment:
   When testing running scan servers in Kubernetes using this [configuration](https://github.com/keith-turner/accumulo-testing/blob/322b32c62113b974b9dd18404d0e57902dc13b8e/sstest/accumulo-scanservers.yaml#L92) I had to make the following change inorder to be able to set the memory in the deployment descriptor.
   
   ```suggestion
     sserver) JAVA_OPTS=( '-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
   ```
   
   I don't think we should apply this change in this branch. Thinking if the change makes sense that it should be applied to all the server config as its own PR.  Just wanted to comment here in case anyone is running Kubernetes tests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1138641601

   > Using `set/getConsistencyLevel` on ScannerBase allows us to change the implementation without changing the API. Using a name tied to the implementation will cause API churn if the implementation changes. I'm not tied to `set/getConsistencyLevel`.
   
   Yes, but it would allow us to change implementation of that specific characteristic... but that's not the essential characteristic here.
   
   > I'm waiting for some consensus before change the API method names.
   
   That's reasonable. Thus far, I haven't really been thinking alternative names. I've only been trying to make the case that consistency level isn't the right way of exposing the feature to users. To think of new names, I think it would be good to think about what the essential characteristics are for scan servers:
   
   * higher availability?
   * dynamically scalable hosting?
   * read-only service?
   * some other sufficient description?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156641115

   > I kinda like ScanServerChooser
   
   I'm partial to `ScanServerSelector`.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881778731


##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {

Review Comment:
   Implemented suggest in fb4f193



##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);
+
+      int count2 = 0;
+      while (iter2.hasNext()) {
+        iter2.next();
+        count2++;
+      }
+      assertEquals(1000, count2);
+
+      scanner1.close();
+
+      try (Scanner scanner2 = client.createScanner(tableName, Authorizations.EMPTY)) {
+        assertEquals(1100, Iterables.size(scanner2));
+      }
+    }
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheDisabled() throws Exception {
+
+    startScanServer(false);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1100 k/v because the tablet metadata is not cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {

Review Comment:
   Implemented suggest in fb4f193



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] wjsl commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
wjsl commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r882034845


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   Are we serving stale data if tables only ever have data bulk loaded?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r882264711


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.Reference;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.gc.GCRun;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
+
+  private static class ScanServerMetadataEntriesITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerMetadataEntriesITConfiguration c = new ScanServerMetadataEntriesITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  @Test
+  public void testServerContextMethods() throws Exception {
+
+    try (AccumuloClient ac = Accumulo.newClient().from(getClientProps()).build()) {
+      HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+      UUID serverLockUUID = UUID.randomUUID();
+
+      String[] files =
+          new String[] {"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+              "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf"};
+
+      Set<ScanServerRefTabletFile> scanRefs = new HashSet<>();
+      for (String file : files) {
+        scanRefs.add(new ScanServerRefTabletFile(file, server.toString(), serverLockUUID));
+      }
+
+      ServerContext ctx = getCluster().getServerContext();
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      Set<ScanServerRefTabletFile> scanRefs2 =
+          ctx.getAmple().getScanServerFileReferences().collect(Collectors.toSet());
+
+      assertEquals(scanRefs, scanRefs2);
+
+      ctx.getAmple().deleteScanServerFileReferences("127.0.0.1:1234", serverLockUUID);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().putScanServerFileReferences(scanRefs);
+      assertEquals(2, ctx.getAmple().getScanServerFileReferences().count());
+
+      ctx.getAmple().deleteScanServerFileReferences(scanRefs);
+      assertEquals(0, ctx.getAmple().getScanServerFileReferences().count());
+
+    }
+  }
+
+  @Test
+  public void testScanServerMetadataEntries() throws Exception {
+
+    ServerContext ctx = getCluster().getServerContext();
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);

Review Comment:
   Ah, oops. Yeah, I guess the system user can't do some things via the normal RPC paths.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939141263


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Map<? extends KeyExtent,? extends TabletMetadata>
+        loadAll(Set<? extends KeyExtent> keys) {
+      long t1 = System.currentTimeMillis();
+      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
+          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
+      return tms;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanServer.class);
+
+  protected ThriftScanClientHandler delegate;
+  private UUID serverLockUUID;
+  private final TabletMetadataLoader tabletMetadataLoader;
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+  // tracks file reservations that are in the process of being added or removed from the metadata
+  // table
+  private final Set<StoredTabletFile> influxFiles = new HashSet<>();
+  // a read lock that ensures files are not removed from reservedFiles while its held
+  private final ReentrantReadWriteLock.ReadLock reservationsReadLock;
+  // a write lock that must be held when mutating influxFiles or when removing entries from
+  // reservedFiles
+  private final ReentrantReadWriteLock.WriteLock reservationsWriteLock;
+  // this condition is used to signal changes to influxFiles
+  private final Condition reservationCondition;
+  // the key is the set of files that have reservations in the metadata table, the value contains
+  // information about which scans are currently using the file
+  private final Map<StoredTabletFile,ReservedFile> reservedFiles = new ConcurrentHashMap<>();
+  private final AtomicLong nextScanReservationId = new AtomicLong();
+
+  private final ServerContext context;
+  private final SessionManager sessionManager;
+  private final TabletServerResourceManager resourceManager;
+  HostAndPort clientAddress;
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+
+  private volatile boolean serverStopRequested = false;
+  private ServiceLock scanServerLock;
+  protected TabletServerScanMetrics scanMetrics;
+
+  private ZooCache managerLockCache;
+
+  private final String groupName;
+
+  public ScanServer(ScanServerOpts opts, String[] args) {
+    super("sserver", opts, args);
+
+    context = super.getContext();
+    log.info("Version " + Constants.VERSION);
+    log.info("Instance " + getContext().getInstanceID());
+    this.sessionManager = new SessionManager(context);
+
+    this.resourceManager = new TabletServerResourceManager(context, this);
+
+    this.managerLockCache = new ZooCache(context.getZooReader(), null);
+
+    var readWriteLock = new ReentrantReadWriteLock();
+    reservationsReadLock = readWriteLock.readLock();
+    reservationsWriteLock = readWriteLock.writeLock();
+    reservationCondition = readWriteLock.writeLock().newCondition();
+
+    // Note: The way to control the number of concurrent scans that a ScanServer will
+    // perform is by using Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS or the number
+    // of threads in Property.SSERV_SCAN_EXECUTORS_PREFIX.
+
+    long cacheExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION);
+
+    long scanServerReservationExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME);
+
+    tabletMetadataLoader = new TabletMetadataLoader(getContext().getAmple());
+
+    if (cacheExpiration == 0L) {
+      LOG.warn("Tablet metadata caching disabled, may cause excessive scans on metadata table.");
+      tabletMetadataCache = null;
+    } else {
+      if (cacheExpiration < 60000) {
+        LOG.warn(
+            "Tablet metadata caching less than one minute, may cause excessive scans on metadata table.");
+      }
+      tabletMetadataCache =
+          Caffeine.newBuilder().expireAfterWrite(cacheExpiration, TimeUnit.MILLISECONDS)
+              .scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+    }
+
+    delegate = newThriftScanClientHandler(new WriteTracker());
+
+    this.groupName = Objects.requireNonNull(opts.getGroupName());
+
+    ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
+        .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
+            scanServerReservationExpiration, scanServerReservationExpiration,
+            TimeUnit.MILLISECONDS));
+
+  }
+
+  @VisibleForTesting
+  protected ThriftScanClientHandler newThriftScanClientHandler(WriteTracker writeTracker) {
+    return new ThriftScanClientHandler(this, writeTracker);
+  }
+
+  /**
+   * Start the thrift service to handle incoming client requests
+   *
+   * @return address of this client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startScanServerClientService() throws UnknownHostException {
+
+    // This class implements TabletClientService.Iface and then delegates calls. Be sure
+    // to set up the ThriftProcessor using this class, not the delegate.
+    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+
+    Property maxMessageSizeProperty =
+        (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
+            ? Property.SSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
+        Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.SSERV_PORTSEARCH, Property.SSERV_MINTHREADS,
+        Property.SSERV_MINTHREADS_TIMEOUT, Property.SSERV_THREADCHECK, maxMessageSizeProperty);
+
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  public String getClientAddressString() {
+    if (clientAddress == null) {
+      return null;
+    }
+    return clientAddress.getHost() + ":" + clientAddress.getPort();
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   */
+  private ServiceLock announceExistence() {
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    try {
+
+      var zLockPath = ServiceLock.path(
+          getContext().getZooKeeperRoot() + Constants.ZSSERVERS + "/" + getClientAddressString());
+
+      try {
+        // Old zk nodes can be cleaned up by ZooZap
+        zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
+      } catch (KeeperException e) {
+        if (e.code() == KeeperException.Code.NOAUTH) {
+          LOG.error("Failed to write to ZooKeeper. Ensure that"
+              + " accumulo.properties, specifically instance.secret, is consistent.");
+        }
+        throw e;
+      }
+
+      serverLockUUID = UUID.randomUUID();
+      scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID);
+
+      LockWatcher lw = new LockWatcher() {
+
+        @Override
+        public void lostLock(final LockLossReason reason) {
+          Halt.halt(serverStopRequested ? 0 : 1, () -> {
+            if (!serverStopRequested) {
+              LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
+            }
+            gcLogger.logGCInfo(getConfiguration());
+          });
+        }
+
+        @Override
+        public void unableToMonitorLockNode(final Exception e) {
+          Halt.halt(1, () -> LOG.error("Lost ability to monitor scan server lock, exiting.", e));
+        }
+      };
+
+      // Don't use the normal ServerServices lock content, instead put the server UUID here.
+      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
+
+      for (int i = 0; i < 120 / 5; i++) {
+        zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
+
+        if (scanServerLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
+          return scanServerLock;
+        }
+        LOG.info("Waiting for scan server lock");
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
+      }
+      String msg = "Too many retries, exiting.";
+      LOG.info(msg);
+      throw new RuntimeException(msg);
+    } catch (Exception e) {
+      LOG.info("Could not obtain scan server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void run() {
+    SecurityUtil.serverLogin(getConfiguration());
+
+    ServerAddress address = null;
+    try {
+      address = startScanServerClientService();
+      clientAddress = address.getAddress();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the compactor client service", e1);
+    }
+
+    try {
+      MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName,
+          clientAddress);
+    } catch (Exception e1) {
+      LOG.error("Error initializing metrics, metrics will not be emitted.", e1);

Review Comment:
   Create #2851 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939145047


##########
server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java:
##########
@@ -176,6 +178,23 @@ public void execute(String[] args) throws Exception {
 
       }
 
+      if (opts.zapScanServers) {
+        String sserversPath = Constants.ZROOT + "/" + iid + Constants.ZSSERVERS;
+        try {
+          List<String> children = zoo.getChildren(sserversPath);
+          for (String child : children) {
+            message("Deleting " + sserversPath + "/" + child + " from zookeeper", opts);
+
+            var zLockPath = ServiceLock.path(sserversPath + "/" + child);
+            if (!zoo.getChildren(zLockPath.toString()).isEmpty()) {
+              ServiceLock.deleteLock(zoo, zLockPath);
+            }
+          }
+        } catch (Exception e) {
+          log.error("{}", e.getMessage(), e);

Review Comment:
   Created #2852 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939081834


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Map<? extends KeyExtent,? extends TabletMetadata>
+        loadAll(Set<? extends KeyExtent> keys) {
+      long t1 = System.currentTimeMillis();
+      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
+          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
+      return tms;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanServer.class);
+
+  protected ThriftScanClientHandler delegate;
+  private UUID serverLockUUID;
+  private final TabletMetadataLoader tabletMetadataLoader;
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+  // tracks file reservations that are in the process of being added or removed from the metadata
+  // table
+  private final Set<StoredTabletFile> influxFiles = new HashSet<>();
+  // a read lock that ensures files are not removed from reservedFiles while its held
+  private final ReentrantReadWriteLock.ReadLock reservationsReadLock;
+  // a write lock that must be held when mutating influxFiles or when removing entries from
+  // reservedFiles
+  private final ReentrantReadWriteLock.WriteLock reservationsWriteLock;
+  // this condition is used to signal changes to influxFiles
+  private final Condition reservationCondition;
+  // the key is the set of files that have reservations in the metadata table, the value contains
+  // information about which scans are currently using the file
+  private final Map<StoredTabletFile,ReservedFile> reservedFiles = new ConcurrentHashMap<>();
+  private final AtomicLong nextScanReservationId = new AtomicLong();
+
+  private final ServerContext context;
+  private final SessionManager sessionManager;
+  private final TabletServerResourceManager resourceManager;
+  HostAndPort clientAddress;
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+
+  private volatile boolean serverStopRequested = false;
+  private ServiceLock scanServerLock;
+  protected TabletServerScanMetrics scanMetrics;
+
+  private ZooCache managerLockCache;
+
+  private final String groupName;
+
+  public ScanServer(ScanServerOpts opts, String[] args) {
+    super("sserver", opts, args);
+
+    context = super.getContext();
+    log.info("Version " + Constants.VERSION);
+    log.info("Instance " + getContext().getInstanceID());
+    this.sessionManager = new SessionManager(context);
+
+    this.resourceManager = new TabletServerResourceManager(context, this);
+
+    this.managerLockCache = new ZooCache(context.getZooReader(), null);
+
+    var readWriteLock = new ReentrantReadWriteLock();
+    reservationsReadLock = readWriteLock.readLock();
+    reservationsWriteLock = readWriteLock.writeLock();
+    reservationCondition = readWriteLock.writeLock().newCondition();
+
+    // Note: The way to control the number of concurrent scans that a ScanServer will
+    // perform is by using Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS or the number
+    // of threads in Property.SSERV_SCAN_EXECUTORS_PREFIX.
+
+    long cacheExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION);
+
+    long scanServerReservationExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME);
+
+    tabletMetadataLoader = new TabletMetadataLoader(getContext().getAmple());
+
+    if (cacheExpiration == 0L) {
+      LOG.warn("Tablet metadata caching disabled, may cause excessive scans on metadata table.");
+      tabletMetadataCache = null;
+    } else {
+      if (cacheExpiration < 60000) {
+        LOG.warn(
+            "Tablet metadata caching less than one minute, may cause excessive scans on metadata table.");
+      }
+      tabletMetadataCache =
+          Caffeine.newBuilder().expireAfterWrite(cacheExpiration, TimeUnit.MILLISECONDS)
+              .scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+    }
+
+    delegate = newThriftScanClientHandler(new WriteTracker());
+
+    this.groupName = Objects.requireNonNull(opts.getGroupName());
+
+    ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
+        .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
+            scanServerReservationExpiration, scanServerReservationExpiration,
+            TimeUnit.MILLISECONDS));
+
+  }
+
+  @VisibleForTesting
+  protected ThriftScanClientHandler newThriftScanClientHandler(WriteTracker writeTracker) {
+    return new ThriftScanClientHandler(this, writeTracker);
+  }
+
+  /**
+   * Start the thrift service to handle incoming client requests
+   *
+   * @return address of this client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startScanServerClientService() throws UnknownHostException {
+
+    // This class implements TabletClientService.Iface and then delegates calls. Be sure
+    // to set up the ThriftProcessor using this class, not the delegate.
+    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+
+    Property maxMessageSizeProperty =
+        (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
+            ? Property.SSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
+        Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.SSERV_PORTSEARCH, Property.SSERV_MINTHREADS,
+        Property.SSERV_MINTHREADS_TIMEOUT, Property.SSERV_THREADCHECK, maxMessageSizeProperty);
+
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  public String getClientAddressString() {
+    if (clientAddress == null) {
+      return null;
+    }
+    return clientAddress.getHost() + ":" + clientAddress.getPort();
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   */
+  private ServiceLock announceExistence() {
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    try {
+
+      var zLockPath = ServiceLock.path(
+          getContext().getZooKeeperRoot() + Constants.ZSSERVERS + "/" + getClientAddressString());
+
+      try {
+        // Old zk nodes can be cleaned up by ZooZap
+        zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
+      } catch (KeeperException e) {
+        if (e.code() == KeeperException.Code.NOAUTH) {
+          LOG.error("Failed to write to ZooKeeper. Ensure that"
+              + " accumulo.properties, specifically instance.secret, is consistent.");
+        }
+        throw e;
+      }
+
+      serverLockUUID = UUID.randomUUID();
+      scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID);
+
+      LockWatcher lw = new LockWatcher() {
+
+        @Override
+        public void lostLock(final LockLossReason reason) {
+          Halt.halt(serverStopRequested ? 0 : 1, () -> {
+            if (!serverStopRequested) {
+              LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
+            }
+            gcLogger.logGCInfo(getConfiguration());
+          });
+        }
+
+        @Override
+        public void unableToMonitorLockNode(final Exception e) {
+          Halt.halt(1, () -> LOG.error("Lost ability to monitor scan server lock, exiting.", e));
+        }
+      };
+
+      // Don't use the normal ServerServices lock content, instead put the server UUID here.
+      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
+
+      for (int i = 0; i < 120 / 5; i++) {
+        zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
+
+        if (scanServerLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
+          return scanServerLock;
+        }
+        LOG.info("Waiting for scan server lock");
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
+      }
+      String msg = "Too many retries, exiting.";
+      LOG.info(msg);
+      throw new RuntimeException(msg);
+    } catch (Exception e) {
+      LOG.info("Could not obtain scan server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void run() {
+    SecurityUtil.serverLogin(getConfiguration());
+
+    ServerAddress address = null;
+    try {
+      address = startScanServerClientService();
+      clientAddress = address.getAddress();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the compactor client service", e1);
+    }
+
+    try {
+      MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName,
+          clientAddress);
+    } catch (Exception e1) {
+      LOG.error("Error initializing metrics, metrics will not be emitted.", e1);

Review Comment:
   MetricsUtil.initializeMetrics throws `Exception`. I will create another issue to fix that



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r940243305


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMultipleScansIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+    }
+  }
+
+  private static final int NUM_SCANS = 4;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @BeforeEach
+  public void before() throws Exception {
+    executor = Executors.newCachedThreadPool();
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    executor.shutdown();
+  }
+
+  private ExecutorService executor;
+
+  @Test
+  public void testMutipleScansSameTablet() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRange(new Range());
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+    }
+  }
+
+  @Test
+  public void testSingleScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+      }
+    }
+  }
+
+  @Test
+  public void testMultipleScansDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+      System.out.println(splits);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      final AtomicInteger counter = new AtomicInteger(0);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+
+      for (int i = 0; i < NUM_SCANS; i++) {
+        final int threadNum = i;
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            switch (threadNum) {
+              case 0:
+                scanner.setRange(new Range("row_0000000000", "row_0000000002"));
+                break;
+              case 1:
+                scanner.setRange(new Range("row_0000000003", "row_0000000005"));
+                break;
+              case 2:
+                scanner.setRange(new Range("row_0000000006", "row_0000000008"));
+                break;
+              case 3:
+                scanner.setRange(new Range("row_0000000009"));
+                break;
+              default:
+                fail("Invalid threadNum");
+            }
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+            counter.addAndGet(Iterables.size(scanner));
+
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+      assertEquals(100, counter.get());
+    }
+  }
+
+  @Test
+  public void testMutipleBatchScansSameTablet() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (BatchScanner scanner = client.createBatchScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRanges(Collections.singletonList(new Range()));
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+    }
+
+  }
+
+  @Test
+  public void testSingleBatchScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);

Review Comment:
   Updated to use `listSplits` in d416900f14ac0c0e88af37aff49cb3b97a6a8173. Good catch.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1155946603

   >  Is there a diagram or something to help illustrate the new dispatching design?
   
   Here is one.
   
   ![scan_server](https://user-images.githubusercontent.com/1268739/173731016-6c948612-52b7-4bba-9b08-ab0641f1a35a.png)
   
   The ScanServerDispatcher plugin is new and it runs client side to determine which scan server to use for eventually consistent scans and what busy timeout to use.  If the scan fails because of busy timeout its consulted again and can pick another scan server.  The busy timeout feature is unique to scan servers.  This new plugin is configured via accumulo client config,
   
   On the server side (tserver and scan server) there is an existing per table ScanDispatcher plugin that determines what thread pools to use and how to use cache.  This existing plugin makes no decision about which scan server to use as that needs to be done client side for efficiency.  This existig plugin is configured via table props.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156655946

   I like ScanServerSelector also as a name.  I do think it would be good to have something besides Dispatcher in the name to avoid confusion w/ the server side plugin.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939147515


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
+    try {
+      Scanner scanner = context.createScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
+      scanner.setRange(ScanServerFileReferenceSection.getRange());
+      int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
+      return StreamSupport.stream(scanner.spliterator(), false)
+          .map(e -> new ScanServerRefTabletFile(e.getKey().getRowData().toString().substring(pLen),
+              e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()));
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   Created #2853



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939160177


##########
test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java:
##########
@@ -142,6 +143,9 @@ private void doWorkToGenerateMetrics() throws Exception {
         writer.addMutation(m);
       }
       client.tableOperations().compact(tableName, new CompactionConfig());
+      try (Scanner scanner = client.createScanner(tableName)) {
+        scanner.forEach((k, v) -> {});
+      }

Review Comment:
   This method is just performing work to generate metrics. It creates a table, inserts some data, compacts the table, performs a scan, delete the table, etc. The test method that calls this method verifies that the metrics we expect to see based on the activity that is performed are seen. I don't think there is anything to do here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1208167050

   @ctubbsii - I believe that I have addressed most, if not all, of your comments either by responding, creating a follow-on issue, or fixing the code.
   
   @ctubbsii @keith-turner @milleruntime @EdColeman - If there is no further discussion, then I intend on merging this by COB today.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r885719570


##########
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java:
##########
@@ -474,6 +474,8 @@ public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long
   public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority,
       boolean emitThreadPoolMetrics) {
+    LOG.debug("Creating ThreadPoolExecutor for {} with {} core threads and {} max threads", name,
+        coreThreads, maxThreads);

Review Comment:
   It looks like SSERV_MINTHREADS_TIMEOUT defaults to 10s, but TSERV_MINTHREADS_TIMEOUT defaults to 0s.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1138930239

   > what the user desires is increased availability
   
   I think that depends on who the "user" is. If we are talking about an app developer, the person writing the code that uses the Scanner, then I'm not sure that they will give any thought to system or data availability. If the user is a system administrator, then they will deploy ScanServers for system and data availability.
   
   Looking at [PACELC](http://www.cs.umd.edu/~abadi/papers/abadi-pacelc.pdf), Accumulo prioritizes Consistency when operating normally (PC/EC). Using ScanServers enables the system administrator and app developer to use Accumulo in a PC/EL manner.
   
   Since this is an API for app developers, do we talk about this in terms of latency? `prioritizeLatency()` ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ivakegg commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ivakegg commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r866143883


##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {

Review Comment:
   Would it be worth it if we combine these two while loops and actually alternate between iter1 and iter2 to ensure everything is running concurrently?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881549361


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors
+      long busyAttempts = params.getAttempts(tablet).stream()
+          .filter(sa -> sa.getResult() == ScanAttempt.Result.BUSY).count();
+
+      maxBusyAttempts = Math.max(maxBusyAttempts, busyAttempts);
+
+      String serverToUse = null;
+
+      var hashCode = hashTablet(tablet);
+
+      int numServers;
+
+      if (busyAttempts < maxDepth) {
+        numServers = (int) Math
+            .round(initialServers * Math.pow(orderedScanServers.size() / (double) initialServers,
+                busyAttempts / (double) maxDepth));
+      } else {
+        numServers = orderedScanServers.size();
+      }
+
+      int serverIndex =
+          (Math.abs(hashCode.asInt()) + RANDOM.nextInt(numServers)) % orderedScanServers.size();
+
+      // TODO could check if errors were seen on this server in past attempts

Review Comment:
   resolved in ec7cbad



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881549161


##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerDispatcher.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+
+/**
+ * The default Accumulo dispatcher for scan servers. This dispatcher will hash tablets to a few
+ * random scan servers (defaults to 3). So a given tablet will always go to the same 3 scan servers.
+ * When scan servers are busy, this dispatcher will rapidly expand the number of scan servers it
+ * randomly chooses from for a given tablet. With the default settings and 1000 scan servers that
+ * are busy, this dispatcher would randomly choose from 3, 21, 144, and then 1000 scan servers.
+ * After getting to a point where we are raondomly choosing from all scan server, if busy is still
+ * being observed then this dispatcher will start to exponentially increase the busy timeout. If all
+ * scan servers are busy then its best to just go to one and wait for your scan to run, which is why
+ * the busy timeout increases exponentially when it seems like everything is busy.
+ *
+ * <p>
+ * The following options are accepted in {@link #init(InitParameters)}
+ * </p>
+ *
+ * <ul>
+ * <li><b>initialServers</b> the initial number of servers to randomly choose from for a given
+ * tablet. Defaults to 3.</li>
+ * <li><b>initialBusyTimeout</b>The initial busy timeout to use when contacting a scan servers. If
+ * the scan does start running within the busy timeout then another scan server can be tried.
+ * Defaults to PT0.033S see {@link Duration#parse(CharSequence)}</li>
+ * <li><b>maxBusyTimeout</b>When busy is repeatedly seen, then the busy timeout will be increased
+ * exponentially. This setting controls the maximum busyTimeout. Defaults to PT30M</li>
+ * <li><b>maxDepth</b>When busy is observed the number of servers to randomly chose from is
+ * expanded. This setting controls how many busy observations it will take before we choose from all
+ * servers.</li>
+ * </ul>
+ *
+ *
+ */
+public class DefaultScanServerDispatcher implements ScanServerDispatcher {
+
+  private static final Logger LOG = LoggerFactory.getLogger(DefaultScanServerDispatcher.class);
+
+  private static final SecureRandom RANDOM = new SecureRandom();
+
+  protected Duration initialBusyTimeout;
+  protected Duration maxBusyTimeout;
+
+  protected int initialServers;
+  protected int maxDepth;
+
+  private Supplier<List<String>> orderedScanServersSupplier;
+
+  private static final Set<String> OPT_NAMES =
+      Set.of("initialServers", "maxDepth", "initialBusyTimeout", "maxBusyTimeout");
+
+  @Override
+  public void init(InitParameters params) {
+    // avoid constantly resorting the scan servers, just do it periodically in case they change
+    orderedScanServersSupplier = Suppliers.memoizeWithExpiration(() -> {
+      List<String> oss = new ArrayList<>(params.getScanServers().get());
+      Collections.sort(oss);
+      return Collections.unmodifiableList(oss);
+    }, 100, TimeUnit.MILLISECONDS);
+
+    var opts = params.getOptions();
+
+    var diff = Sets.difference(opts.keySet(), OPT_NAMES);
+
+    Preconditions.checkArgument(diff.isEmpty(), "Unknown options %s", diff);
+
+    initialServers = Integer.parseInt(opts.getOrDefault("initialServers", "3"));
+    maxDepth = Integer.parseInt(opts.getOrDefault("maxDepth", "3"));
+    initialBusyTimeout = Duration.parse(opts.getOrDefault("initialBusyTimeout", "PT0.033S"));
+    maxBusyTimeout = Duration.parse(opts.getOrDefault("maxBusyTimeout", "PT30M"));
+
+    Preconditions.checkArgument(initialServers > 0, "initialServers must be positive : %s",
+        initialServers);
+    Preconditions.checkArgument(maxDepth > 0, "maxDepth must be positive : %s", maxDepth);
+    Preconditions.checkArgument(initialBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "initialBusyTimeout must be positive %s", initialBusyTimeout);
+    Preconditions.checkArgument(maxBusyTimeout.compareTo(Duration.ZERO) > 0,
+        "maxBusyTimeout must be positive %s", maxBusyTimeout);
+
+    LOG.debug(
+        "DefaultScanServerDispatcher configured with initialServers: {}"
+            + ", maxDepth: {}, initialBusyTimeout: {}, maxBusyTimeout: {}",
+        initialServers, maxDepth, initialBusyTimeout, maxBusyTimeout);
+  }
+
+  @Override
+  public Actions determineActions(DispatcherParameters params) {
+
+    // only get this once and use it for the entire method so that the method uses a consistent
+    // snapshot
+    List<String> orderedScanServers = orderedScanServersSupplier.get();
+
+    if (orderedScanServers.isEmpty()) {
+      return new Actions() {
+        @Override
+        public String getScanServer(TabletId tabletId) {
+          return null;
+        }
+
+        @Override
+        public Duration getDelay() {
+          return Duration.ZERO;
+        }
+
+        @Override
+        public Duration getBusyTimeout() {
+          return Duration.ZERO;
+        }
+      };
+    }
+
+    Map<TabletId,String> serversToUse = new HashMap<>();
+
+    long maxBusyAttempts = 0;
+
+    for (TabletId tablet : params.getTablets()) {
+
+      // TODO handle io errors

Review Comment:
   resolved in ec7cbad



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r882270535


##########
test/src/main/java/org/apache/accumulo/test/performance/NullTserver.java:
##########
@@ -131,12 +131,12 @@ public void closeMultiScan(TInfo tinfo, long scanID) {}
     public void closeScan(TInfo tinfo, long scanID) {}
 
     @Override
-    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID) {
+    public MultiScanResult continueMultiScan(TInfo tinfo, long scanID, long busyTimeout) {

Review Comment:
   I'm just thinking that adding such things as groundwork makes it easier to incrementally review and merge discrete elements of a larger set of changes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881725527


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.Reference;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample.DataLevel;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ScanServerFileReferenceSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.gc.GCRun;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.slf4j.LoggerFactory;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase {
+
+  private static class ScanServerMetadataEntriesITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME, "5s");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerMetadataEntriesITConfiguration c = new ScanServerMetadataEntriesITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    stopMiniCluster();
+  }
+
+  @Test
+  public void testServerContextMethods() throws Exception {
+
+    try (AccumuloClient ac = Accumulo.newClient().from(getClientProps()).build()) {
+      HostAndPort server = HostAndPort.fromParts("127.0.0.1", 1234);
+      UUID serverLockUUID = UUID.randomUUID();
+
+      String[] files =
+          new String[] {"hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000070.rf",
+              "hdfs://localhost:8020/accumulo/tables/2a/default_tablet/F0000071.rf"};
+
+      Set<ScanServerRefTabletFile> scanRefs = new HashSet<>();
+      for (String file : files) {
+        scanRefs.add(new ScanServerRefTabletFile(file, server.toString(), serverLockUUID));
+      }

Review Comment:
   Implemented suggestion in 5c039df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] milleruntime commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
milleruntime commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1139585195

   I think the design of this change is great so far but it is a major new feature, with a drastic increase in complexity, touching  major parts of Accumulo (scans, API, metadata, configuration, scripts, and introduces another new server). I don't think this should get merged into 2.1. The complexity of this change on top of all the changes already in 2.1 will only further delay the release of 2.1. Main already has many major new features (ZK Prop Store, Overhaul of Compactions code, External Compactions, AMPLE, Master Rename, New Tracing, New Metrics, New SPI, Root Table change) not to mention the 1,130+ tickets marked done for 2.1.
   
   Here is a timeline of the past 4 years:
   - 2.0.0-alpha-1 was released 14 Oct 2018 and then 2.0.0-alpha-2 was released 31 Jan 2019
   - Non-LTM 2.0.0 was released 02 Aug 2019
   - LTM 1.10 was released 03 Sep 2020.
   - Bug fix release 1.10.1 was 22 Dec 2020
   - Bug fix 2.0.1 was released 24 Dec 2020
   - Sadly the first time ever, Accumulo did not have a release in 2021 :frowning_face: 
   - Bug fix for 1.10.2 was 13 Feb 2022
   
   I think 2.1.0 needs to be released ASAP so it can be tagged as an LTM. Users who are not on 1.10, should upgrade to 1.10.3. Users who upgraded to 2.0 at the time of release have been waiting 3 years (users who snagged the alpha will be 4 years) for the next LTM release. I think it would be a disservice to not have an LTM release this year. Our original plan was to Periodically release a new LTM approximately every 2 years.
   
   That being said Main may already be a major release and need to be called 3.0 so if that is the case, merge away!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939063078


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+
+      Map<String,String> sserverProps = new HashMap<>();
+      ClientProperty
+          .getPrefix(info.getProperties(), ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey())
+          .forEach((k, v) -> {
+            sserverProps.put(
+                k.toString()
+                    .substring(ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey().length()),
+                v.toString());
+          });

Review Comment:
   This returns Map<Object,Object>, not Map<String,String>, so it doesn't work. Do you have an alternative implementation?
   
   ```
         Map<String,String> sserverProps = ClientProperty
             .getPrefix(info.getProperties(), ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey())
             .entrySet()
             .stream()
             .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getvalue()),
                 (prev, next) -> next, HashMap::new));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939104754


##########
core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/BlockCacheManagerTest.java:
##########
@@ -27,6 +27,7 @@
 public class BlockCacheManagerTest {
 
   @Test
+  @SuppressWarnings("deprecation")

Review Comment:
   `BlockCacheManager.getFullyQualifiedPropertyPrefix` methods are marked @Deprecated(since = "2.1.0")



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r880844146


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   In #2411 I called it an Isolated Scanner, but we already have something along those lines. In that issue @keith-turner made a comment which was the genesis for the current name. I'm not tied to the name, do you have a different suggestion on what it should be?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881723777


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -612,6 +741,7 @@ void errorOccured() {
     }
 
     /**
+     *
      */

Review Comment:
   Removed in 5c039df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1149969820

   Summarizing the remaining unresolved issues from the conversation above:
   
   1. There is disagreement on the naming of the API methods.
   2. There is concern about merging this into 2.1 due to the complexity of this change, on top of the other complex changes already in 2.1, that it will require too much time to test that it will delay the release of 2.1. 
   
   Did I capture those correctly? Did I miss anything?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156648560

   It may be useful to add information to the following that informs a server side dispatcher if its running in a tserver or a scan server.  Not sure about this.  Would be follow on work.
   
   https://github.com/apache/accumulo/blob/418eee8fa0abed50f30cb6c97184964f924d2c67/core/src/main/java/org/apache/accumulo/core/spi/scan/ScanDispatcher.java#L98-L110


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939103483


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   Almost every other method in ServerAmpleImpl throws RuntimeException. I will create an issue to fix ServerAmpleImpl.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939054555


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+

Review Comment:
   You might be thinking of ClassLoaderUtil.loadClass. I don't think it's used in the client code. Looking at it, I'm not sure it will work in the client code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939161117


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Map<? extends KeyExtent,? extends TabletMetadata>
+        loadAll(Set<? extends KeyExtent> keys) {
+      long t1 = System.currentTimeMillis();
+      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
+          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
+      return tms;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanServer.class);
+
+  protected ThriftScanClientHandler delegate;
+  private UUID serverLockUUID;
+  private final TabletMetadataLoader tabletMetadataLoader;
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+  // tracks file reservations that are in the process of being added or removed from the metadata
+  // table
+  private final Set<StoredTabletFile> influxFiles = new HashSet<>();
+  // a read lock that ensures files are not removed from reservedFiles while its held
+  private final ReentrantReadWriteLock.ReadLock reservationsReadLock;
+  // a write lock that must be held when mutating influxFiles or when removing entries from
+  // reservedFiles
+  private final ReentrantReadWriteLock.WriteLock reservationsWriteLock;
+  // this condition is used to signal changes to influxFiles
+  private final Condition reservationCondition;
+  // the key is the set of files that have reservations in the metadata table, the value contains
+  // information about which scans are currently using the file
+  private final Map<StoredTabletFile,ReservedFile> reservedFiles = new ConcurrentHashMap<>();
+  private final AtomicLong nextScanReservationId = new AtomicLong();
+
+  private final ServerContext context;
+  private final SessionManager sessionManager;
+  private final TabletServerResourceManager resourceManager;
+  HostAndPort clientAddress;
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+
+  private volatile boolean serverStopRequested = false;
+  private ServiceLock scanServerLock;
+  protected TabletServerScanMetrics scanMetrics;
+
+  private ZooCache managerLockCache;
+
+  private final String groupName;
+
+  public ScanServer(ScanServerOpts opts, String[] args) {
+    super("sserver", opts, args);
+
+    context = super.getContext();
+    log.info("Version " + Constants.VERSION);
+    log.info("Instance " + getContext().getInstanceID());
+    this.sessionManager = new SessionManager(context);
+
+    this.resourceManager = new TabletServerResourceManager(context, this);
+
+    this.managerLockCache = new ZooCache(context.getZooReader(), null);
+
+    var readWriteLock = new ReentrantReadWriteLock();
+    reservationsReadLock = readWriteLock.readLock();
+    reservationsWriteLock = readWriteLock.writeLock();
+    reservationCondition = readWriteLock.writeLock().newCondition();
+
+    // Note: The way to control the number of concurrent scans that a ScanServer will
+    // perform is by using Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS or the number
+    // of threads in Property.SSERV_SCAN_EXECUTORS_PREFIX.
+
+    long cacheExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION);
+
+    long scanServerReservationExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME);
+
+    tabletMetadataLoader = new TabletMetadataLoader(getContext().getAmple());
+
+    if (cacheExpiration == 0L) {
+      LOG.warn("Tablet metadata caching disabled, may cause excessive scans on metadata table.");
+      tabletMetadataCache = null;
+    } else {
+      if (cacheExpiration < 60000) {
+        LOG.warn(
+            "Tablet metadata caching less than one minute, may cause excessive scans on metadata table.");
+      }
+      tabletMetadataCache =
+          Caffeine.newBuilder().expireAfterWrite(cacheExpiration, TimeUnit.MILLISECONDS)
+              .scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+    }
+
+    delegate = newThriftScanClientHandler(new WriteTracker());
+
+    this.groupName = Objects.requireNonNull(opts.getGroupName());
+
+    ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
+        .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
+            scanServerReservationExpiration, scanServerReservationExpiration,
+            TimeUnit.MILLISECONDS));
+
+  }
+
+  @VisibleForTesting
+  protected ThriftScanClientHandler newThriftScanClientHandler(WriteTracker writeTracker) {
+    return new ThriftScanClientHandler(this, writeTracker);
+  }
+
+  /**
+   * Start the thrift service to handle incoming client requests
+   *
+   * @return address of this client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startScanServerClientService() throws UnknownHostException {
+
+    // This class implements TabletClientService.Iface and then delegates calls. Be sure
+    // to set up the ThriftProcessor using this class, not the delegate.
+    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+
+    Property maxMessageSizeProperty =
+        (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
+            ? Property.SSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
+        Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.SSERV_PORTSEARCH, Property.SSERV_MINTHREADS,
+        Property.SSERV_MINTHREADS_TIMEOUT, Property.SSERV_THREADCHECK, maxMessageSizeProperty);
+
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  public String getClientAddressString() {
+    if (clientAddress == null) {
+      return null;
+    }
+    return clientAddress.getHost() + ":" + clientAddress.getPort();
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   */
+  private ServiceLock announceExistence() {
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    try {
+
+      var zLockPath = ServiceLock.path(
+          getContext().getZooKeeperRoot() + Constants.ZSSERVERS + "/" + getClientAddressString());
+
+      try {
+        // Old zk nodes can be cleaned up by ZooZap
+        zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
+      } catch (KeeperException e) {
+        if (e.code() == KeeperException.Code.NOAUTH) {
+          LOG.error("Failed to write to ZooKeeper. Ensure that"
+              + " accumulo.properties, specifically instance.secret, is consistent.");
+        }
+        throw e;
+      }
+
+      serverLockUUID = UUID.randomUUID();
+      scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID);
+
+      LockWatcher lw = new LockWatcher() {
+
+        @Override
+        public void lostLock(final LockLossReason reason) {
+          Halt.halt(serverStopRequested ? 0 : 1, () -> {
+            if (!serverStopRequested) {
+              LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
+            }
+            gcLogger.logGCInfo(getConfiguration());
+          });
+        }
+
+        @Override
+        public void unableToMonitorLockNode(final Exception e) {
+          Halt.halt(1, () -> LOG.error("Lost ability to monitor scan server lock, exiting.", e));
+        }
+      };
+
+      // Don't use the normal ServerServices lock content, instead put the server UUID here.
+      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
+
+      for (int i = 0; i < 120 / 5; i++) {
+        zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
+
+        if (scanServerLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
+          return scanServerLock;
+        }
+        LOG.info("Waiting for scan server lock");
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
+      }
+      String msg = "Too many retries, exiting.";
+      LOG.info(msg);
+      throw new RuntimeException(msg);
+    } catch (Exception e) {
+      LOG.info("Could not obtain scan server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void run() {
+    SecurityUtil.serverLogin(getConfiguration());
+
+    ServerAddress address = null;
+    try {
+      address = startScanServerClientService();
+      clientAddress = address.getAddress();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the compactor client service", e1);
+    }
+
+    try {
+      MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName,
+          clientAddress);
+    } catch (Exception e1) {
+      LOG.error("Error initializing metrics, metrics will not be emitted.", e1);
+    }
+    scanMetrics = new TabletServerScanMetrics();
+    MetricsUtil.initializeProducers(scanMetrics);
+
+    // We need to set the compaction manager so that we don't get an NPE in CompactableImpl.close
+
+    ServiceLock lock = announceExistence();
+
+    try {
+      while (!serverStopRequested) {
+        UtilWaitThread.sleep(1000);
+      }
+    } finally {
+      LOG.info("Stopping Thrift Servers");
+      address.server.stop();
+
+      LOG.info("Removing server scan references");
+      this.getContext().getAmple().deleteScanServerFileReferences(clientAddress.toString(),
+          serverLockUUID);
+
+      try {
+        LOG.debug("Closing filesystems");
+        VolumeManager mgr = getContext().getVolumeManager();
+        if (null != mgr) {
+          mgr.close();
+        }
+      } catch (IOException e) {
+        LOG.warn("Failed to close filesystem : {}", e.getMessage(), e);
+      }
+
+      gcLogger.logGCInfo(getConfiguration());
+      LOG.info("stop requested. exiting ... ");
+      try {
+        if (null != lock) {
+          lock.unlock();
+        }
+      } catch (Exception e) {
+        LOG.warn("Failed to release scan server lock", e);
+      }
+
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<KeyExtent,TabletMetadata> getTabletMetadata(Collection<KeyExtent> extents) {
+    if (tabletMetadataCache == null) {
+      return (Map<KeyExtent,TabletMetadata>) tabletMetadataLoader
+          .loadAll((Set<? extends KeyExtent>) extents);
+    } else {
+      return tabletMetadataCache.getAll(extents);
+    }
+  }
+
+  static class ReservedFile {
+    final Set<Long> activeReservations = new ConcurrentSkipListSet<>();
+    final AtomicLong lastUseTime = new AtomicLong(0);
+
+    boolean shouldDelete(long expireTimeMs) {
+      return activeReservations.isEmpty()
+          && System.currentTimeMillis() - lastUseTime.get() > expireTimeMs;
+    }
+  }
+
+  class ScanReservation implements AutoCloseable {
+
+    private final Collection<StoredTabletFile> files;
+    private final long myReservationId;
+    private final Map<KeyExtent,TabletMetadata> tabletsMetadata;
+
+    ScanReservation(Map<KeyExtent,TabletMetadata> tabletsMetadata, long myReservationId) {
+      this.tabletsMetadata = tabletsMetadata;
+      this.files = tabletsMetadata.values().stream().flatMap(tm -> tm.getFiles().stream())
+          .collect(Collectors.toUnmodifiableSet());
+      this.myReservationId = myReservationId;
+    }
+
+    ScanReservation(Collection<StoredTabletFile> files, long myReservationId) {
+      this.tabletsMetadata = null;
+      this.files = files;
+      this.myReservationId = myReservationId;
+    }
+
+    public TabletMetadata getTabletMetadata(KeyExtent extent) {
+      return tabletsMetadata.get(extent);
+    }
+
+    SnapshotTablet newTablet(ScanServer server, KeyExtent extent) throws IOException {
+      var tabletMetadata = getTabletMetadata(extent);
+      TabletResourceManager trm =
+          resourceManager.createTabletResourceManager(tabletMetadata.getExtent(),
+              context.getTableConfiguration(tabletMetadata.getExtent().tableId()));
+      return new SnapshotTablet(server, tabletMetadata, trm);
+    }
+
+    @Override
+    public void close() {
+      // There is no need to get a lock for removing our reservations. The reservation was added
+      // with a lock held and once its added that prevents file from being removed.
+      for (StoredTabletFile file : files) {
+        var reservedFile = reservedFiles.get(file);
+
+        if (!reservedFile.activeReservations.remove(myReservationId)) {
+          throw new IllegalStateException("reservation id was not in set as expected");
+        }
+
+        LOG.trace("RFFS {} unreserved reference for file {}", myReservationId, file);
+
+        reservedFile.lastUseTime.set(System.currentTimeMillis());
+      }
+    }
+  }
+
+  private Map<KeyExtent,TabletMetadata> reserveFilesInner(Collection<KeyExtent> extents,
+      long myReservationId) throws NotServingTabletException, AccumuloException {
+    // RFS is an acronym for Reference files for scan
+    LOG.debug("RFFS {} ensuring files are referenced for scan of extents {}", myReservationId,
+        extents);
+
+    Map<KeyExtent,TabletMetadata> tabletsMetadata = getTabletMetadata(extents);
+
+    for (KeyExtent extent : extents) {
+      var tabletMetadata = tabletsMetadata.get(extent);
+      if (tabletMetadata == null) {
+        LOG.info("RFFS {} extent not found in metadata table {}", myReservationId, extent);
+        throw new NotServingTabletException(extent.toThrift());
+      }
+
+      if (!AssignmentHandler.checkTabletMetadata(extent, null, tabletMetadata, true)) {
+        LOG.info("RFFS {} extent unable to load {} as AssignmentHandler returned false",
+            myReservationId, extent);
+        throw new NotServingTabletException(extent.toThrift());
+      }
+    }
+
+    Map<StoredTabletFile,KeyExtent> allFiles = new HashMap<>();
+
+    tabletsMetadata.forEach((extent, tm) -> {
+      tm.getFiles().forEach(file -> allFiles.put(file, extent));
+    });
+
+    // The read lock prevents anything from being removed from reservedFiles while adding
+    // reservations to the values of reservedFiles. Using a read lock avoids scans from having to
+    // wait on each other their files are already reserved.
+    reservationsReadLock.lock();
+    try {
+      if (reservedFiles.keySet().containsAll(allFiles.keySet())) {
+        // all files already have reservations in the metadata table, so we can add ourself
+        for (StoredTabletFile file : allFiles.keySet()) {
+          if (!reservedFiles.get(file).activeReservations.add(myReservationId)) {
+            throw new IllegalStateException("reservation id unexpectedly already in set");
+          }
+        }
+
+        return tabletsMetadata;
+      }
+    } finally {
+      reservationsReadLock.unlock();
+    }
+
+    // reservations do not exist in the metadata table, so we will attempt to add them
+    reservationsWriteLock.lock();
+    try {
+      // wait if another thread is working on the files we are interested in
+      while (!Collections.disjoint(influxFiles, allFiles.keySet())) {
+        reservationCondition.await();
+      }
+
+      // add files to reserve to influxFiles so that no other thread tries to add or remove these
+      // file from the metadata table or the reservedFiles map
+      influxFiles.addAll(allFiles.keySet());
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    } finally {
+      reservationsWriteLock.unlock();
+    }
+
+    // do not add any code here that could cause an exception which could lead to files not being
+    // removed from influxFiles
+
+    try {
+      Set<StoredTabletFile> filesToReserve = new HashSet<>();
+      List<ScanServerRefTabletFile> refs = new ArrayList<>();
+      Set<KeyExtent> tabletsToCheck = new HashSet<>();
+
+      String serverAddress = clientAddress.toString();
+
+      for (StoredTabletFile file : allFiles.keySet()) {
+        if (!reservedFiles.containsKey(file)) {
+          refs.add(new ScanServerRefTabletFile(file.getPathStr(), serverAddress, serverLockUUID));
+          filesToReserve.add(file);
+          tabletsToCheck.add(Objects.requireNonNull(allFiles.get(file)));
+          LOG.trace("RFFS {} need to add scan ref for file {}", myReservationId, file);
+        }
+      }
+
+      if (!filesToReserve.isEmpty()) {
+        getContext().getAmple().putScanServerFileReferences(refs);
+
+        // After we insert the scan server refs we need to check and see if the tablet is still
+        // using the file. As long as the tablet is still using the files then the Accumulo GC
+        // should not have deleted the files. This assumes the Accumulo GC reads scan server refs
+        // after tablet refs from the metadata table.
+
+        if (tabletMetadataCache != null) {
+          // lets clear the cache so we get the latest
+          tabletMetadataCache.invalidateAll(tabletsToCheck);
+        }
+
+        var tabletsToCheckMetadata = getTabletMetadata(tabletsToCheck);
+
+        for (KeyExtent extent : tabletsToCheck) {
+          TabletMetadata metadataAfter = tabletsToCheckMetadata.get(extent);
+          if (metadataAfter == null) {
+            getContext().getAmple().deleteScanServerFileReferences(refs);
+            LOG.info("RFFS {} extent unable to load {} as metadata no longer referencing files",
+                myReservationId, extent);
+            throw new NotServingTabletException(extent.toThrift());
+          }
+
+          // remove files that are still referenced
+          filesToReserve.removeAll(metadataAfter.getFiles());
+        }
+
+        // if this is not empty it means some files that we reserved are no longer referenced by
+        // tablets. This means there could have been a time gap where nothing referenced a file
+        // meaning it could have been GCed.
+        if (!filesToReserve.isEmpty()) {
+          LOG.info("RFFS {} tablet files changed while attempting to reference files {}",
+              myReservationId, filesToReserve);
+          getContext().getAmple().deleteScanServerFileReferences(refs);
+          return null;
+        }
+      }
+
+      // we do not hold a lock but the files we are adding are in influxFiles so its ok to add to
+      // reservedFiles
+      for (StoredTabletFile file : allFiles.keySet()) {
+        if (!reservedFiles.computeIfAbsent(file, k -> new ReservedFile()).activeReservations
+            .add(myReservationId)) {
+          throw new IllegalStateException("reservation id unexpectedly already in set");
+        }
+
+        LOG.trace("RFFS {} reserved reference for startScan {}", myReservationId, file);
+      }
+
+      return tabletsMetadata;
+    } finally {
+      reservationsWriteLock.lock();
+      try {
+        allFiles.keySet().forEach(file -> Preconditions.checkState(influxFiles.remove(file)));
+        reservationCondition.signal();
+      } finally {
+        reservationsWriteLock.unlock();
+      }
+    }
+  }
+
+  protected ScanReservation reserveFiles(Collection<KeyExtent> extents)
+      throws NotServingTabletException, AccumuloException {
+
+    long myReservationId = nextScanReservationId.incrementAndGet();
+
+    Map<KeyExtent,TabletMetadata> tabletsMetadata = reserveFilesInner(extents, myReservationId);
+    while (tabletsMetadata == null) {
+      tabletsMetadata = reserveFilesInner(extents, myReservationId);
+    }
+
+    return new ScanReservation(tabletsMetadata, myReservationId);
+  }
+
+  protected ScanReservation reserveFiles(long scanId) throws NoSuchScanIDException {
+    var session = (ScanSession) sessionManager.getSession(scanId);
+    if (session == null) {
+      throw new NoSuchScanIDException();
+    }
+
+    Set<StoredTabletFile> scanSessionFiles;
+
+    if (session instanceof SingleScanSession) {
+      var sss = (SingleScanSession) session;
+      scanSessionFiles =
+          Set.copyOf(session.getTabletResolver().getTablet(sss.extent).getDatafiles().keySet());
+    } else if (session instanceof MultiScanSession) {
+      var mss = (MultiScanSession) session;
+      scanSessionFiles = mss.exents.stream()
+          .flatMap(e -> mss.getTabletResolver().getTablet(e).getDatafiles().keySet().stream())
+          .collect(Collectors.toUnmodifiableSet());
+    } else {
+      throw new IllegalArgumentException("Unknown session type " + session.getClass().getName());
+    }
+
+    long myReservationId = nextScanReservationId.incrementAndGet();
+    // we are only reserving if the files already exists in reservedFiles, so only need the read
+    // lock which prevents deletions from reservedFiles while we mutate the values of reservedFiles
+    reservationsReadLock.lock();
+    try {
+      if (!reservedFiles.keySet().containsAll(scanSessionFiles)) {
+        // the files are no longer reserved in the metadata table, so lets pretend there is no scan
+        // session
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("RFFS {} files are no longer referenced on continue scan {} {}",
+              myReservationId, scanId, Sets.difference(scanSessionFiles, reservedFiles.keySet()));
+        }
+        throw new NoSuchScanIDException();
+      }
+
+      for (StoredTabletFile file : scanSessionFiles) {
+        if (!reservedFiles.get(file).activeReservations.add(myReservationId)) {
+          throw new IllegalStateException("reservation id unexpectedly already in set");
+        }
+
+        LOG.trace("RFFS {} reserved reference for continue scan {} {}", myReservationId, scanId,
+            file);
+      }
+    } finally {
+      reservationsReadLock.unlock();
+    }
+
+    return new ScanReservation(scanSessionFiles, myReservationId);
+  }
+
+  private void cleanUpReservedFiles(long expireTimeMs) {
+
+    // Do a quick check to see if there is any potential work. This check is done to avoid acquiring
+    // the write lock unless its needed since the write lock can be disruptive for the read lock.
+    if (reservedFiles.values().stream().anyMatch(rf -> rf.shouldDelete(expireTimeMs))) {
+
+      List<ScanServerRefTabletFile> refsToDelete = new ArrayList<>();
+      List<StoredTabletFile> confirmed = new ArrayList<>();
+      String serverAddress = clientAddress.toString();
+
+      reservationsWriteLock.lock();
+      try {
+        var reservedIter = reservedFiles.entrySet().iterator();
+
+        while (reservedIter.hasNext()) {
+          var entry = reservedIter.next();
+          var file = entry.getKey();

Review Comment:
   an iterator is used here because `reservedIter.remove()` is called a few lines down.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939158782


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver.tablet;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.TabletHostingServer;
+import org.apache.accumulo.tserver.TabletServerResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tablet that can not be written to and operates off of a snapshot of tablet metadata for its
+ * entire lifetime. The set of files and therefore the data backing this tablet will never change
+ * for its lifetime.
+ */
+public class SnapshotTablet extends TabletBase {
+
+  private static final Logger log = LoggerFactory.getLogger(SnapshotTablet.class);
+
+  private final TabletHostingServer server;
+  private final SortedMap<StoredTabletFile,DataFileValue> files;
+  private final TabletServerResourceManager.TabletResourceManager tabletResources;
+  private boolean closed = false;
+  private final AtomicLong dataSourceDeletions = new AtomicLong(0);
+
+  public SnapshotTablet(TabletHostingServer server, TabletMetadata metadata,
+      TabletServerResourceManager.TabletResourceManager tabletResources) {
+    super(server, metadata.getExtent());
+    this.server = server;
+    this.files = Collections.unmodifiableSortedMap(new TreeMap<>(metadata.getFilesMap()));
+    this.tabletResources = tabletResources;
+  }
+
+  @Override
+  public synchronized boolean isClosed() {
+    return closed;
+  }
+
+  @Override
+  public SortedMap<StoredTabletFile,DataFileValue> getDatafiles() {
+    return files;
+  }
+
+  @Override
+  public void addToYieldMetric(int i) {
+    this.server.getScanMetrics().addYield(i);
+  }
+
+  @Override
+  public long getDataSourceDeletions() {
+    return dataSourceDeletions.get();
+  }
+
+  @Override
+  TabletServerResourceManager.TabletResourceManager getTabletResources() {
+    return tabletResources;
+  }
+
+  @Override
+  public List<InMemoryMap.MemoryIterator> getMemIterators(SamplerConfigurationImpl samplerConfig) {
+    return List.of();
+  }
+
+  @Override
+  public void returnMemIterators(List<InMemoryMap.MemoryIterator> iters) {
+
+  }
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  @Override
+  public Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan() {
+    return new Pair(0L, getDatafiles());
+  }

Review Comment:
   doing so results in a compilation error



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r938929295


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+

Review Comment:
   I thought we had a centralized utility class for loading classes from the configuration. This is probably redundant, but I haven't dug to find it.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+
+      Map<String,String> sserverProps = new HashMap<>();
+      ClientProperty
+          .getPrefix(info.getProperties(), ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey())
+          .forEach((k, v) -> {
+            sserverProps.put(
+                k.toString()
+                    .substring(ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey().length()),
+                v.toString());
+          });

Review Comment:
   Creating a map, then using forEach to populate it is a bit more clunky than using a stream map collector to construct the map directly from the iteration.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SnapshotTablet.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver.tablet;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.TabletHostingServer;
+import org.apache.accumulo.tserver.TabletServerResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tablet that can not be written to and operates off of a snapshot of tablet metadata for its
+ * entire lifetime. The set of files and therefore the data backing this tablet will never change
+ * for its lifetime.
+ */
+public class SnapshotTablet extends TabletBase {
+
+  private static final Logger log = LoggerFactory.getLogger(SnapshotTablet.class);
+
+  private final TabletHostingServer server;
+  private final SortedMap<StoredTabletFile,DataFileValue> files;
+  private final TabletServerResourceManager.TabletResourceManager tabletResources;
+  private boolean closed = false;
+  private final AtomicLong dataSourceDeletions = new AtomicLong(0);
+
+  public SnapshotTablet(TabletHostingServer server, TabletMetadata metadata,
+      TabletServerResourceManager.TabletResourceManager tabletResources) {
+    super(server, metadata.getExtent());
+    this.server = server;
+    this.files = Collections.unmodifiableSortedMap(new TreeMap<>(metadata.getFilesMap()));
+    this.tabletResources = tabletResources;
+  }
+
+  @Override
+  public synchronized boolean isClosed() {
+    return closed;
+  }
+
+  @Override
+  public SortedMap<StoredTabletFile,DataFileValue> getDatafiles() {
+    return files;
+  }
+
+  @Override
+  public void addToYieldMetric(int i) {
+    this.server.getScanMetrics().addYield(i);
+  }
+
+  @Override
+  public long getDataSourceDeletions() {
+    return dataSourceDeletions.get();
+  }
+
+  @Override
+  TabletServerResourceManager.TabletResourceManager getTabletResources() {
+    return tabletResources;
+  }
+
+  @Override
+  public List<InMemoryMap.MemoryIterator> getMemIterators(SamplerConfigurationImpl samplerConfig) {
+    return List.of();
+  }
+
+  @Override
+  public void returnMemIterators(List<InMemoryMap.MemoryIterator> iters) {
+
+  }
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  @Override
+  public Pair<Long,Map<TabletFile,DataFileValue>> reserveFilesForScan() {
+    return new Pair(0L, getDatafiles());
+  }

Review Comment:
   It doesn't make sense to suppress a warning when it can just be fixed by adding the appropriate generics



##########
server/base/src/main/java/org/apache/accumulo/server/util/ScanServerMetadataEntries.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.cli.ServerUtilOpts;
+
+public class ScanServerMetadataEntries {

Review Comment:
   This main class has no class level javadoc explaining what the utility is supposed to do.



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/DefaultScanServerSelector.java:
##########
@@ -0,0 +1,415 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+
+import java.lang.reflect.Type;
+import java.security.SecureRandom;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
+import org.apache.accumulo.core.data.TabletId;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Suppliers;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashCode;
+import com.google.common.hash.Hashing;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+/**
+ * The default Accumulo selector for scan servers. This dispatcher will :
+ *
+ * <ul>
+ * <li>Hash each tablet to a per attempt configurable number of scan servers and then randomly
+ * choose one of those scan servers. Using hashing allows different client to select the same scan
+ * servers for a given tablet.</li>
+ * <li>Use a per attempt configurable busy timeout.</li>
+ * </ul>
+ *
+ * <p>
+ * This class accepts a single configuration that has a json value. To configure this class set
+ * {@code scan.server.selector.opts.profiles=<json>} in the accumulo client configuration along with
+ * the config for the class. The following is the default configuration value.
+ * </p>
+ * <p>
+ * {@value DefaultScanServerSelector#PROFILES_DEFAULT}
+ * </p>
+ *
+ * The json is structured as a list of profiles, with each profile having the following fields.
+ *
+ * <ul>
+ * <li><b>isDefault : </b> A boolean that specifies whether this is the default profile. One and
+ * only one profile must set this to true.</li>
+ * <li><b>maxBusyTimeout : </b> The maximum busy timeout to use. The busy timeout from the last
+ * attempt configuration grows exponentially up to this max.</li>
+ * <li><b>scanTypeActivations : </b> A list of scan types that will activate this profile. Scan
+ * types are specified by setting {@code scan_type=<scan_type>} as execution on the scanner. See
+ * {@link org.apache.accumulo.core.client.ScannerBase#setExecutionHints(Map)}</li>
+ * <li><b>group : </b> Scan servers can be started with an optional group. If specified, this option
+ * will limit the scan servers used to those that were started with this group name. If not
+ * specified, the set of scan servers that did not specify a group will be used. Grouping scan
+ * servers supports at least two use cases. First groups can be used to dedicate resources for
+ * certain scans. Second groups can be used to have different hardware/VM types for scans, for
+ * example could have some scans use expensive high memory VMs and others use cheaper burstable VMs.
+ * <li><b>attemptPlans : </b> A list of configuration to use for each scan attempt. Each list object
+ * has the following fields:
+ * <ul>
+ * <li><b>servers : </b> The number of servers to randomly choose from for this attempt.</li>
+ * <li><b>busyTimeout : </b> The busy timeout to use for this attempt.</li>
+ * <li><b>salt : </b> An optional string to append when hashing the tablet. When this is set
+ * differently for attempts it has the potential to cause the set of servers chosen from to be
+ * disjoint. When not set or the same, the servers between attempts will be subsets.</li>
+ * </ul>
+ * </li>
+ * </ul>
+ *
+ * <p>
+ * Below is an example configuration with two profiles, one is the default and the other is used
+ * when the scan execution hint {@code scan_type=slow} is set.
+ * </p>
+ *
+ * <pre>
+ *    [
+ *     {
+ *       "isDefault":true,
+ *       "maxBusyTimeout":"5m",
+ *       "busyTimeoutMultiplier":4,
+ *       "attemptPlans":[
+ *         {"servers":"3", "busyTimeout":"33ms"},
+ *         {"servers":"100%", "busyTimeout":"100ms"}
+ *       ]
+ *     },
+ *     {
+ *       "scanTypeActivations":["slow"],
+ *       "maxBusyTimeout":"20m",
+ *       "busyTimeoutMultiplier":8,
+ *       "group":"lowcost",
+ *       "attemptPlans":[
+ *         {"servers":"1", "busyTimeout":"10s"},
+ *         {"servers":"3", "busyTimeout":"30s","salt":"42"},
+ *         {"servers":"9", "busyTimeout":"60s","salt":"84"}
+ *       ]
+ *     }
+ *    ]
+ * </pre>
+ *
+ * <p>
+ * For the default profile in the example it will start off by choosing randomly from 3 scan servers
+ * based on a hash of the tablet with no salt. For the first attempt it will use a busy timeout of
+ * 33 milliseconds. If the first attempt returns with busy, then it will randomly choose from 100%
+ * or all servers for the second attempt and use a busy timeout of 100ms. For subsequent attempts it
+ * will keep choosing from all servers and start multiplying the busy timeout by 4 until the max
+ * busy timeout of 4 minutes is reached.
+ * </p>
+ *
+ * <p>
+ * For the profile activated by {@code scan_type=slow} it start off by choosing randomly from 1 scan
+ * server based on a hash of the tablet with no salt and a busy timeout of 10s. The second attempt
+ * will choose from 3 scan servers based on a hash of the tablet plus the salt {@literal 42}.
+ * Without the salt, the single scan servers from the first attempt would always be included in the
+ * set of 3. With the salt the single scan server from the first attempt may not be included. The
+ * third attempt will choose a scan server from 9 using the salt {@literal 84} and a busy timeout of
+ * 60s. The different salt means the set of servers that attempts 2 and 3 choose from may be
+ * disjoint. Attempt 4 and greater will continue to choose from the same 9 servers as attempt 3 and
+ * will keep increasing the busy timeout by multiplying 8 until the maximum of 20 minutes is
+ * reached. For this profile it will choose from scan servers in the group {@literal lowcost}.
+ * </p>
+ */
+public class DefaultScanServerSelector implements ScanServerSelector {

Review Comment:
   This class can be configured as the default, but it shouldn't be *named* "Default". If it ever becomes *not* the default, then having it still named "Default" creates a lot of confusion. It should be named after its general strategy.
   
   For example, git's default merge strategy is called "recursive", not "default".



##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   here too



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanAttemptsImpl.class);
+
+  static class ScanAttemptImpl
+      implements org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt {
+
+    private final String server;
+    private final long time;
+    private final Result result;
+    private volatile long mutationCount = Long.MAX_VALUE;
+
+    ScanAttemptImpl(Result result, String server, long time) {
+      this.result = result;
+      this.server = Objects.requireNonNull(server);
+      this.time = time;
+    }
+
+    @Override
+    public String getServer() {
+      return server;
+    }
+
+    @Override
+    public long getEndTime() {
+      return time;
+    }
+
+    @Override
+    public Result getResult() {
+      return result;
+    }
+
+    private void setMutationCount(long mc) {
+      this.mutationCount = mc;
+    }
+
+    public long getMutationCount() {
+      return mutationCount;
+    }
+  }
+
+  private Map<TabletId,Collection<ScanAttemptImpl>> attempts = new ConcurrentHashMap<>();
+  private long mutationCounter = 0;
+
+  private void add(TabletId tablet, ScanAttempt.Result result, String server, long endTime) {
+
+    ScanAttemptImpl sa = new ScanAttemptImpl(result, server, endTime);
+
+    attempts.computeIfAbsent(tablet, k -> ConcurrentHashMap.newKeySet()).add(sa);
+
+    synchronized (this) {
+      // now that the scan attempt obj is added to all concurrent data structs, make it visible
+      // need to atomically increment the counter AND set the counter on the object
+      sa.setMutationCount(mutationCounter++);
+    }
+
+  }
+
+  public static interface ScanAttemptReporter {
+    void report(ScanAttempt.Result result);
+  }
+
+  ScanAttemptReporter createReporter(String server, TabletId tablet) {
+    return new ScanAttemptReporter() {
+      @Override
+      public void report(ScanAttempt.Result result) {
+        LOG.trace("Received result: {}", result);
+        add(tablet, result, server, System.currentTimeMillis());
+      }
+    };
+  }
+
+  Map<TabletId,Collection<ScanAttemptImpl>> snapshot() {
+    // allows only seeing scan attempt objs that were added before this call
+
+    long snapMC;
+    synchronized (ScanAttemptsImpl.this) {
+      snapMC = mutationCounter;
+    }
+    var tmp = Maps.transformValues(attempts, tabletAttemptList -> Collections2
+        .filter(tabletAttemptList, sai -> sai.getMutationCount() < snapMC));
+
+    return Maps.filterEntries(tmp, entry -> !entry.getValue().isEmpty());

Review Comment:
   This comment is still applicable.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +392,51 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  public static class ScanServerInfo {
+    public final UUID uuid;
+    public final String group;
+
+    public ScanServerInfo(UUID uuid, String group) {
+      this.uuid = uuid;
+      this.group = group;
+    }
+
+  }
+
+  /**
+   * @return map of live scan server addresses to lock uuids.
+   */
+  public Map<String,ScanServerInfo> getScanServers() {
+    Map<String,ScanServerInfo> liveScanServers = new HashMap<>();
+    String root = this.getZooKeeperRoot() + Constants.ZSSERVERS;
+    var addrs = this.getZooCache().getChildren(root);
+    for (String addr : addrs) {
+      try {
+        final var zLockPath = ServiceLock.path(root + "/" + addr);
+        ZcStat stat = new ZcStat();
+        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (lockData != null) {
+          String[] fields = new String(lockData, UTF_8).split(",", 2);
+          UUID uuid = UUID.fromString(fields[0]);
+          String group = fields[1];
+          liveScanServers.put(addr, new ScanServerInfo(uuid, group));
+        }
+      } catch (Exception e) {
+        log.error("Error validating zookeeper scan server node: " + addr, e);

Review Comment:
   Overly broad exception catching, especially since they're merely logged and continued. Could let RTEs fall through and only catch checked exceptions.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+
+public class ClientServiceEnvironmentImpl implements ServiceEnvironment {
+
+  private final ClientContext context;
+
+  public ClientServiceEnvironmentImpl(ClientContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.instanceOperations().getSystemConfiguration()));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting system configuration", e);
+    }
+  }
+
+  @Override
+  public Configuration getConfiguration(TableId tableId) {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.tableOperations().getConfiguration(getTableName(tableId))));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting table configuration", e);
+    }

Review Comment:
   here too



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/ScanTask.java:
##########
@@ -127,9 +154,6 @@ public T get(long timeout, TimeUnit unit)
     // returned
     resultQueue = null;
 
-    if (r instanceof Error)
-      throw (Error) r; // don't wrap an Error
-

Review Comment:
   Why was this removed? I think this will prevent the OOM handler from detecting OOMs thrown from threads.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientServiceEnvironmentImpl.java:
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+import org.apache.accumulo.core.util.ConfigurationImpl;
+
+public class ClientServiceEnvironmentImpl implements ServiceEnvironment {
+
+  private final ClientContext context;
+
+  public ClientServiceEnvironmentImpl(ClientContext context) {
+    this.context = context;
+  }
+
+  @Override
+  public Configuration getConfiguration() {
+    try {
+      return new ConfigurationImpl(
+          new ConfigurationCopy(context.instanceOperations().getSystemConfiguration()));
+    } catch (Exception e) {
+      throw new RuntimeException("Error getting system configuration", e);

Review Comment:
   Please avoid overly broad exception catching, and use more specific exception types than generic RTE when throwing..



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +392,51 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  public static class ScanServerInfo {
+    public final UUID uuid;
+    public final String group;
+
+    public ScanServerInfo(UUID uuid, String group) {
+      this.uuid = uuid;
+      this.group = group;
+    }
+
+  }

Review Comment:
   This class doesn't need to be an inner class, and could either be its own class or just be a `Pair<UUID, String>`



##########
test/src/main/java/org/apache/accumulo/test/metrics/MetricsIT.java:
##########
@@ -142,6 +143,9 @@ private void doWorkToGenerateMetrics() throws Exception {
         writer.addMutation(m);
       }
       client.tableOperations().compact(tableName, new CompactionConfig());
+      try (Scanner scanner = client.createScanner(tableName)) {
+        scanner.forEach((k, v) -> {});
+      }

Review Comment:
   How do we know this returned anything? Should we count these results?



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")

Review Comment:
   This suppression could be made more narrow by putting it on a variable assignment rather than the whole method, if it's necessary at all.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+
+      Map<String,String> sserverProps = new HashMap<>();
+      ClientProperty
+          .getPrefix(info.getProperties(), ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey())
+          .forEach((k, v) -> {
+            sserverProps.put(
+                k.toString()
+                    .substring(ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey().length()),
+                v.toString());
+          });
+
+      scanServerSelector.init(new ScanServerSelector.InitParameters() {
+        @Override
+        public Map<String,String> getOptions() {
+          return Collections.unmodifiableMap(sserverProps);
+        }
+
+        @Override
+        public ServiceEnvironment getServiceEnv() {
+          return new ClientServiceEnvironmentImpl(ClientContext.this);
+        }
+
+        @Override
+        public Supplier<Collection<ScanServer>> getScanServers() {
+          return () -> ClientContext.this.getScanServers().entrySet().stream()
+              .map(entry -> new ScanServer() {
+                @Override
+                public String getAddress() {
+                  return entry.getKey();
+                }
+
+                @Override
+                public String getGroup() {
+                  return entry.getValue().group;
+                }
+              }).collect(Collectors.toSet());
+        }
+      });
+      return scanServerSelector;
+    } catch (Exception e) {

Review Comment:
   This exception is a bit broad. Could just catch the non-RTEs with a multi-catch, and let the RTEs fall through rather than be caught and re-thrown as an RTE wrapping another RTE.



##########
server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java:
##########
@@ -176,6 +178,23 @@ public void execute(String[] args) throws Exception {
 
       }
 
+      if (opts.zapScanServers) {
+        String sserversPath = Constants.ZROOT + "/" + iid + Constants.ZSSERVERS;
+        try {
+          List<String> children = zoo.getChildren(sserversPath);
+          for (String child : children) {
+            message("Deleting " + sserversPath + "/" + child + " from zookeeper", opts);
+
+            var zLockPath = ServiceLock.path(sserversPath + "/" + child);
+            if (!zoo.getChildren(zLockPath.toString()).isEmpty()) {
+              ServiceLock.deleteLock(zoo, zLockPath);
+            }
+          }
+        } catch (Exception e) {
+          log.error("{}", e.getMessage(), e);

Review Comment:
   overly broad exception swallowing



##########
core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/BlockCacheManagerTest.java:
##########
@@ -27,6 +27,7 @@
 public class BlockCacheManagerTest {
 
   @Test
+  @SuppressWarnings("deprecation")

Review Comment:
   What is being deprecated in this PR? This feature is being added as experimental. New experimental features don't typically cause new deprecations.



##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
+    try {
+      Scanner scanner = context.createScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
+      scanner.setRange(ScanServerFileReferenceSection.getRange());
+      int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
+      return StreamSupport.stream(scanner.spliterator(), false)
+          .map(e -> new ScanServerRefTabletFile(e.getKey().getRowData().toString().substring(pLen),
+              e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()));
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   Try to throw more specific RTEs



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Map<? extends KeyExtent,? extends TabletMetadata>
+        loadAll(Set<? extends KeyExtent> keys) {
+      long t1 = System.currentTimeMillis();
+      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
+          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
+      return tms;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanServer.class);
+
+  protected ThriftScanClientHandler delegate;
+  private UUID serverLockUUID;
+  private final TabletMetadataLoader tabletMetadataLoader;
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+  // tracks file reservations that are in the process of being added or removed from the metadata
+  // table
+  private final Set<StoredTabletFile> influxFiles = new HashSet<>();
+  // a read lock that ensures files are not removed from reservedFiles while its held
+  private final ReentrantReadWriteLock.ReadLock reservationsReadLock;
+  // a write lock that must be held when mutating influxFiles or when removing entries from
+  // reservedFiles
+  private final ReentrantReadWriteLock.WriteLock reservationsWriteLock;
+  // this condition is used to signal changes to influxFiles
+  private final Condition reservationCondition;
+  // the key is the set of files that have reservations in the metadata table, the value contains
+  // information about which scans are currently using the file
+  private final Map<StoredTabletFile,ReservedFile> reservedFiles = new ConcurrentHashMap<>();
+  private final AtomicLong nextScanReservationId = new AtomicLong();
+
+  private final ServerContext context;
+  private final SessionManager sessionManager;
+  private final TabletServerResourceManager resourceManager;
+  HostAndPort clientAddress;
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+
+  private volatile boolean serverStopRequested = false;
+  private ServiceLock scanServerLock;
+  protected TabletServerScanMetrics scanMetrics;
+
+  private ZooCache managerLockCache;
+
+  private final String groupName;
+
+  public ScanServer(ScanServerOpts opts, String[] args) {
+    super("sserver", opts, args);
+
+    context = super.getContext();
+    log.info("Version " + Constants.VERSION);
+    log.info("Instance " + getContext().getInstanceID());
+    this.sessionManager = new SessionManager(context);
+
+    this.resourceManager = new TabletServerResourceManager(context, this);
+
+    this.managerLockCache = new ZooCache(context.getZooReader(), null);
+
+    var readWriteLock = new ReentrantReadWriteLock();
+    reservationsReadLock = readWriteLock.readLock();
+    reservationsWriteLock = readWriteLock.writeLock();
+    reservationCondition = readWriteLock.writeLock().newCondition();
+
+    // Note: The way to control the number of concurrent scans that a ScanServer will
+    // perform is by using Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS or the number
+    // of threads in Property.SSERV_SCAN_EXECUTORS_PREFIX.
+
+    long cacheExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION);
+
+    long scanServerReservationExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME);
+
+    tabletMetadataLoader = new TabletMetadataLoader(getContext().getAmple());
+
+    if (cacheExpiration == 0L) {
+      LOG.warn("Tablet metadata caching disabled, may cause excessive scans on metadata table.");
+      tabletMetadataCache = null;
+    } else {
+      if (cacheExpiration < 60000) {
+        LOG.warn(
+            "Tablet metadata caching less than one minute, may cause excessive scans on metadata table.");
+      }
+      tabletMetadataCache =
+          Caffeine.newBuilder().expireAfterWrite(cacheExpiration, TimeUnit.MILLISECONDS)
+              .scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+    }
+
+    delegate = newThriftScanClientHandler(new WriteTracker());
+
+    this.groupName = Objects.requireNonNull(opts.getGroupName());
+
+    ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
+        .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
+            scanServerReservationExpiration, scanServerReservationExpiration,
+            TimeUnit.MILLISECONDS));
+
+  }
+
+  @VisibleForTesting
+  protected ThriftScanClientHandler newThriftScanClientHandler(WriteTracker writeTracker) {
+    return new ThriftScanClientHandler(this, writeTracker);
+  }
+
+  /**
+   * Start the thrift service to handle incoming client requests
+   *
+   * @return address of this client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startScanServerClientService() throws UnknownHostException {
+
+    // This class implements TabletClientService.Iface and then delegates calls. Be sure
+    // to set up the ThriftProcessor using this class, not the delegate.
+    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+
+    Property maxMessageSizeProperty =
+        (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
+            ? Property.SSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
+        Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.SSERV_PORTSEARCH, Property.SSERV_MINTHREADS,
+        Property.SSERV_MINTHREADS_TIMEOUT, Property.SSERV_THREADCHECK, maxMessageSizeProperty);
+
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  public String getClientAddressString() {
+    if (clientAddress == null) {
+      return null;
+    }
+    return clientAddress.getHost() + ":" + clientAddress.getPort();
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   */
+  private ServiceLock announceExistence() {
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    try {
+
+      var zLockPath = ServiceLock.path(
+          getContext().getZooKeeperRoot() + Constants.ZSSERVERS + "/" + getClientAddressString());
+
+      try {
+        // Old zk nodes can be cleaned up by ZooZap
+        zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
+      } catch (KeeperException e) {
+        if (e.code() == KeeperException.Code.NOAUTH) {
+          LOG.error("Failed to write to ZooKeeper. Ensure that"
+              + " accumulo.properties, specifically instance.secret, is consistent.");
+        }
+        throw e;
+      }
+
+      serverLockUUID = UUID.randomUUID();
+      scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID);
+
+      LockWatcher lw = new LockWatcher() {
+
+        @Override
+        public void lostLock(final LockLossReason reason) {
+          Halt.halt(serverStopRequested ? 0 : 1, () -> {
+            if (!serverStopRequested) {
+              LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
+            }
+            gcLogger.logGCInfo(getConfiguration());
+          });
+        }
+
+        @Override
+        public void unableToMonitorLockNode(final Exception e) {
+          Halt.halt(1, () -> LOG.error("Lost ability to monitor scan server lock, exiting.", e));
+        }
+      };
+
+      // Don't use the normal ServerServices lock content, instead put the server UUID here.
+      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
+
+      for (int i = 0; i < 120 / 5; i++) {
+        zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
+
+        if (scanServerLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
+          return scanServerLock;
+        }
+        LOG.info("Waiting for scan server lock");
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
+      }
+      String msg = "Too many retries, exiting.";
+      LOG.info(msg);
+      throw new RuntimeException(msg);
+    } catch (Exception e) {
+      LOG.info("Could not obtain scan server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void run() {
+    SecurityUtil.serverLogin(getConfiguration());
+
+    ServerAddress address = null;
+    try {
+      address = startScanServerClientService();
+      clientAddress = address.getAddress();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the compactor client service", e1);
+    }
+
+    try {
+      MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName,
+          clientAddress);
+    } catch (Exception e1) {
+      LOG.error("Error initializing metrics, metrics will not be emitted.", e1);
+    }
+    scanMetrics = new TabletServerScanMetrics();
+    MetricsUtil.initializeProducers(scanMetrics);
+
+    // We need to set the compaction manager so that we don't get an NPE in CompactableImpl.close
+
+    ServiceLock lock = announceExistence();
+
+    try {
+      while (!serverStopRequested) {
+        UtilWaitThread.sleep(1000);
+      }
+    } finally {
+      LOG.info("Stopping Thrift Servers");
+      address.server.stop();
+
+      LOG.info("Removing server scan references");
+      this.getContext().getAmple().deleteScanServerFileReferences(clientAddress.toString(),
+          serverLockUUID);
+
+      try {
+        LOG.debug("Closing filesystems");
+        VolumeManager mgr = getContext().getVolumeManager();
+        if (null != mgr) {
+          mgr.close();
+        }
+      } catch (IOException e) {
+        LOG.warn("Failed to close filesystem : {}", e.getMessage(), e);
+      }
+
+      gcLogger.logGCInfo(getConfiguration());
+      LOG.info("stop requested. exiting ... ");
+      try {
+        if (null != lock) {
+          lock.unlock();
+        }
+      } catch (Exception e) {
+        LOG.warn("Failed to release scan server lock", e);
+      }
+
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<KeyExtent,TabletMetadata> getTabletMetadata(Collection<KeyExtent> extents) {
+    if (tabletMetadataCache == null) {
+      return (Map<KeyExtent,TabletMetadata>) tabletMetadataLoader
+          .loadAll((Set<? extends KeyExtent>) extents);
+    } else {
+      return tabletMetadataCache.getAll(extents);
+    }
+  }
+
+  static class ReservedFile {
+    final Set<Long> activeReservations = new ConcurrentSkipListSet<>();
+    final AtomicLong lastUseTime = new AtomicLong(0);
+
+    boolean shouldDelete(long expireTimeMs) {
+      return activeReservations.isEmpty()
+          && System.currentTimeMillis() - lastUseTime.get() > expireTimeMs;
+    }
+  }
+
+  class ScanReservation implements AutoCloseable {
+
+    private final Collection<StoredTabletFile> files;
+    private final long myReservationId;
+    private final Map<KeyExtent,TabletMetadata> tabletsMetadata;
+
+    ScanReservation(Map<KeyExtent,TabletMetadata> tabletsMetadata, long myReservationId) {
+      this.tabletsMetadata = tabletsMetadata;
+      this.files = tabletsMetadata.values().stream().flatMap(tm -> tm.getFiles().stream())
+          .collect(Collectors.toUnmodifiableSet());
+      this.myReservationId = myReservationId;
+    }
+
+    ScanReservation(Collection<StoredTabletFile> files, long myReservationId) {
+      this.tabletsMetadata = null;
+      this.files = files;
+      this.myReservationId = myReservationId;
+    }
+
+    public TabletMetadata getTabletMetadata(KeyExtent extent) {
+      return tabletsMetadata.get(extent);
+    }
+
+    SnapshotTablet newTablet(ScanServer server, KeyExtent extent) throws IOException {
+      var tabletMetadata = getTabletMetadata(extent);
+      TabletResourceManager trm =
+          resourceManager.createTabletResourceManager(tabletMetadata.getExtent(),
+              context.getTableConfiguration(tabletMetadata.getExtent().tableId()));
+      return new SnapshotTablet(server, tabletMetadata, trm);
+    }
+
+    @Override
+    public void close() {
+      // There is no need to get a lock for removing our reservations. The reservation was added
+      // with a lock held and once its added that prevents file from being removed.
+      for (StoredTabletFile file : files) {
+        var reservedFile = reservedFiles.get(file);
+
+        if (!reservedFile.activeReservations.remove(myReservationId)) {
+          throw new IllegalStateException("reservation id was not in set as expected");
+        }
+
+        LOG.trace("RFFS {} unreserved reference for file {}", myReservationId, file);
+
+        reservedFile.lastUseTime.set(System.currentTimeMillis());
+      }
+    }
+  }
+
+  private Map<KeyExtent,TabletMetadata> reserveFilesInner(Collection<KeyExtent> extents,
+      long myReservationId) throws NotServingTabletException, AccumuloException {
+    // RFS is an acronym for Reference files for scan
+    LOG.debug("RFFS {} ensuring files are referenced for scan of extents {}", myReservationId,
+        extents);
+
+    Map<KeyExtent,TabletMetadata> tabletsMetadata = getTabletMetadata(extents);
+
+    for (KeyExtent extent : extents) {
+      var tabletMetadata = tabletsMetadata.get(extent);
+      if (tabletMetadata == null) {
+        LOG.info("RFFS {} extent not found in metadata table {}", myReservationId, extent);
+        throw new NotServingTabletException(extent.toThrift());
+      }
+
+      if (!AssignmentHandler.checkTabletMetadata(extent, null, tabletMetadata, true)) {
+        LOG.info("RFFS {} extent unable to load {} as AssignmentHandler returned false",
+            myReservationId, extent);
+        throw new NotServingTabletException(extent.toThrift());
+      }
+    }
+
+    Map<StoredTabletFile,KeyExtent> allFiles = new HashMap<>();
+
+    tabletsMetadata.forEach((extent, tm) -> {
+      tm.getFiles().forEach(file -> allFiles.put(file, extent));
+    });
+
+    // The read lock prevents anything from being removed from reservedFiles while adding
+    // reservations to the values of reservedFiles. Using a read lock avoids scans from having to
+    // wait on each other their files are already reserved.
+    reservationsReadLock.lock();
+    try {
+      if (reservedFiles.keySet().containsAll(allFiles.keySet())) {
+        // all files already have reservations in the metadata table, so we can add ourself
+        for (StoredTabletFile file : allFiles.keySet()) {
+          if (!reservedFiles.get(file).activeReservations.add(myReservationId)) {
+            throw new IllegalStateException("reservation id unexpectedly already in set");
+          }
+        }
+
+        return tabletsMetadata;
+      }
+    } finally {
+      reservationsReadLock.unlock();
+    }
+
+    // reservations do not exist in the metadata table, so we will attempt to add them
+    reservationsWriteLock.lock();
+    try {
+      // wait if another thread is working on the files we are interested in
+      while (!Collections.disjoint(influxFiles, allFiles.keySet())) {
+        reservationCondition.await();
+      }
+
+      // add files to reserve to influxFiles so that no other thread tries to add or remove these
+      // file from the metadata table or the reservedFiles map
+      influxFiles.addAll(allFiles.keySet());
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    } finally {
+      reservationsWriteLock.unlock();
+    }
+
+    // do not add any code here that could cause an exception which could lead to files not being
+    // removed from influxFiles
+
+    try {
+      Set<StoredTabletFile> filesToReserve = new HashSet<>();
+      List<ScanServerRefTabletFile> refs = new ArrayList<>();
+      Set<KeyExtent> tabletsToCheck = new HashSet<>();
+
+      String serverAddress = clientAddress.toString();
+
+      for (StoredTabletFile file : allFiles.keySet()) {
+        if (!reservedFiles.containsKey(file)) {
+          refs.add(new ScanServerRefTabletFile(file.getPathStr(), serverAddress, serverLockUUID));
+          filesToReserve.add(file);
+          tabletsToCheck.add(Objects.requireNonNull(allFiles.get(file)));
+          LOG.trace("RFFS {} need to add scan ref for file {}", myReservationId, file);
+        }
+      }
+
+      if (!filesToReserve.isEmpty()) {
+        getContext().getAmple().putScanServerFileReferences(refs);
+
+        // After we insert the scan server refs we need to check and see if the tablet is still
+        // using the file. As long as the tablet is still using the files then the Accumulo GC
+        // should not have deleted the files. This assumes the Accumulo GC reads scan server refs
+        // after tablet refs from the metadata table.
+
+        if (tabletMetadataCache != null) {
+          // lets clear the cache so we get the latest
+          tabletMetadataCache.invalidateAll(tabletsToCheck);
+        }
+
+        var tabletsToCheckMetadata = getTabletMetadata(tabletsToCheck);
+
+        for (KeyExtent extent : tabletsToCheck) {
+          TabletMetadata metadataAfter = tabletsToCheckMetadata.get(extent);
+          if (metadataAfter == null) {
+            getContext().getAmple().deleteScanServerFileReferences(refs);
+            LOG.info("RFFS {} extent unable to load {} as metadata no longer referencing files",
+                myReservationId, extent);
+            throw new NotServingTabletException(extent.toThrift());
+          }
+
+          // remove files that are still referenced
+          filesToReserve.removeAll(metadataAfter.getFiles());
+        }
+
+        // if this is not empty it means some files that we reserved are no longer referenced by
+        // tablets. This means there could have been a time gap where nothing referenced a file
+        // meaning it could have been GCed.
+        if (!filesToReserve.isEmpty()) {
+          LOG.info("RFFS {} tablet files changed while attempting to reference files {}",
+              myReservationId, filesToReserve);
+          getContext().getAmple().deleteScanServerFileReferences(refs);
+          return null;
+        }
+      }
+
+      // we do not hold a lock but the files we are adding are in influxFiles so its ok to add to
+      // reservedFiles
+      for (StoredTabletFile file : allFiles.keySet()) {
+        if (!reservedFiles.computeIfAbsent(file, k -> new ReservedFile()).activeReservations
+            .add(myReservationId)) {
+          throw new IllegalStateException("reservation id unexpectedly already in set");
+        }
+
+        LOG.trace("RFFS {} reserved reference for startScan {}", myReservationId, file);
+      }
+
+      return tabletsMetadata;
+    } finally {
+      reservationsWriteLock.lock();
+      try {
+        allFiles.keySet().forEach(file -> Preconditions.checkState(influxFiles.remove(file)));
+        reservationCondition.signal();
+      } finally {
+        reservationsWriteLock.unlock();
+      }
+    }
+  }
+
+  protected ScanReservation reserveFiles(Collection<KeyExtent> extents)
+      throws NotServingTabletException, AccumuloException {
+
+    long myReservationId = nextScanReservationId.incrementAndGet();
+
+    Map<KeyExtent,TabletMetadata> tabletsMetadata = reserveFilesInner(extents, myReservationId);
+    while (tabletsMetadata == null) {
+      tabletsMetadata = reserveFilesInner(extents, myReservationId);
+    }
+
+    return new ScanReservation(tabletsMetadata, myReservationId);
+  }
+
+  protected ScanReservation reserveFiles(long scanId) throws NoSuchScanIDException {
+    var session = (ScanSession) sessionManager.getSession(scanId);
+    if (session == null) {
+      throw new NoSuchScanIDException();
+    }
+
+    Set<StoredTabletFile> scanSessionFiles;
+
+    if (session instanceof SingleScanSession) {
+      var sss = (SingleScanSession) session;
+      scanSessionFiles =
+          Set.copyOf(session.getTabletResolver().getTablet(sss.extent).getDatafiles().keySet());
+    } else if (session instanceof MultiScanSession) {
+      var mss = (MultiScanSession) session;
+      scanSessionFiles = mss.exents.stream()
+          .flatMap(e -> mss.getTabletResolver().getTablet(e).getDatafiles().keySet().stream())
+          .collect(Collectors.toUnmodifiableSet());
+    } else {
+      throw new IllegalArgumentException("Unknown session type " + session.getClass().getName());
+    }
+
+    long myReservationId = nextScanReservationId.incrementAndGet();
+    // we are only reserving if the files already exists in reservedFiles, so only need the read
+    // lock which prevents deletions from reservedFiles while we mutate the values of reservedFiles
+    reservationsReadLock.lock();
+    try {
+      if (!reservedFiles.keySet().containsAll(scanSessionFiles)) {
+        // the files are no longer reserved in the metadata table, so lets pretend there is no scan
+        // session
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("RFFS {} files are no longer referenced on continue scan {} {}",
+              myReservationId, scanId, Sets.difference(scanSessionFiles, reservedFiles.keySet()));
+        }
+        throw new NoSuchScanIDException();
+      }
+
+      for (StoredTabletFile file : scanSessionFiles) {
+        if (!reservedFiles.get(file).activeReservations.add(myReservationId)) {
+          throw new IllegalStateException("reservation id unexpectedly already in set");
+        }
+
+        LOG.trace("RFFS {} reserved reference for continue scan {} {}", myReservationId, scanId,
+            file);
+      }
+    } finally {
+      reservationsReadLock.unlock();
+    }
+
+    return new ScanReservation(scanSessionFiles, myReservationId);
+  }
+
+  private void cleanUpReservedFiles(long expireTimeMs) {
+
+    // Do a quick check to see if there is any potential work. This check is done to avoid acquiring
+    // the write lock unless its needed since the write lock can be disruptive for the read lock.
+    if (reservedFiles.values().stream().anyMatch(rf -> rf.shouldDelete(expireTimeMs))) {
+
+      List<ScanServerRefTabletFile> refsToDelete = new ArrayList<>();
+      List<StoredTabletFile> confirmed = new ArrayList<>();
+      String serverAddress = clientAddress.toString();
+
+      reservationsWriteLock.lock();
+      try {
+        var reservedIter = reservedFiles.entrySet().iterator();
+
+        while (reservedIter.hasNext()) {
+          var entry = reservedIter.next();
+          var file = entry.getKey();

Review Comment:
   `.forEach()` might make more sense than an entry set iterator (unless you're avoiding that to allow checked exceptions to get out of the block). Still, in that case, a regular for loop over the entry set would probably be better than getting an iterator and doing a while loop.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScannerIterator.java:
##########
@@ -84,7 +85,8 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
         new ScanState(context, tableId, authorizations, new Range(range), options.fetchedColumns,
             size, options.serverSideIteratorList, options.serverSideIteratorOptions, isolated,
             readaheadThreshold, options.getSamplerConfiguration(), options.batchTimeOut,
-            options.classLoaderContext, options.executionHints);
+            options.classLoaderContext, options.executionHints,
+            (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) ? true : false);

Review Comment:
   This ternary operator is redundant... the equals method already returns true or false. Also, this looks like an enum comparison and should use `==` for type safety instead of `.equals`.



##########
server/tserver/src/main/java/org/apache/accumulo/tserver/ScanServer.java:
##########
@@ -0,0 +1,987 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.accumulo.fate.util.UtilWaitThread.sleepUninterruptibly;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.clientImpl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.InitialMultiScan;
+import org.apache.accumulo.core.dataImpl.thrift.InitialScan;
+import org.apache.accumulo.core.dataImpl.thrift.IterInfo;
+import org.apache.accumulo.core.dataImpl.thrift.MultiScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.ScanResult;
+import org.apache.accumulo.core.dataImpl.thrift.TColumn;
+import org.apache.accumulo.core.dataImpl.thrift.TKeyExtent;
+import org.apache.accumulo.core.dataImpl.thrift.TRange;
+import org.apache.accumulo.core.file.blockfile.cache.impl.BlockCacheConfiguration;
+import org.apache.accumulo.core.metadata.ScanServerRefTabletFile;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.schema.Ample;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metrics.MetricsUtil;
+import org.apache.accumulo.core.securityImpl.thrift.TCredentials;
+import org.apache.accumulo.core.spi.scan.ScanServerSelector;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveScan;
+import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException;
+import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
+import org.apache.accumulo.core.tabletserver.thrift.TSampleNotPresentException;
+import org.apache.accumulo.core.tabletserver.thrift.TSamplerConfiguration;
+import org.apache.accumulo.core.tabletserver.thrift.TabletScanClientService;
+import org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.core.util.Halt;
+import org.apache.accumulo.core.util.HostAndPort;
+import org.apache.accumulo.core.util.threads.ThreadPools;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.ServiceLock;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockLossReason;
+import org.apache.accumulo.fate.zookeeper.ServiceLock.LockWatcher;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.server.AbstractServer;
+import org.apache.accumulo.server.GarbageCollectionLogger;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.rpc.ServerAddress;
+import org.apache.accumulo.server.rpc.TServerUtils;
+import org.apache.accumulo.server.rpc.ThriftProcessorTypes;
+import org.apache.accumulo.server.security.SecurityUtil;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.ScanSession.TabletResolver;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.SessionManager;
+import org.apache.accumulo.tserver.session.SingleScanSession;
+import org.apache.accumulo.tserver.tablet.SnapshotTablet;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletBase;
+import org.apache.thrift.TException;
+import org.apache.thrift.TProcessor;
+import org.apache.zookeeper.KeeperException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.beust.jcommander.Parameter;
+import com.github.benmanes.caffeine.cache.CacheLoader;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import com.github.benmanes.caffeine.cache.LoadingCache;
+import com.github.benmanes.caffeine.cache.Scheduler;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
+public class ScanServer extends AbstractServer
+    implements TabletScanClientService.Iface, TabletHostingServer {
+
+  public static class ScanServerOpts extends ServerOpts {
+    @Parameter(required = false, names = {"-g", "--group"},
+        description = "Optional group name that will be made available to the ScanServerSelector client plugin.  If not specified will be set to '"
+            + ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME
+            + "'.  Groups support at least two use cases : dedicating resources to scans and/or using different hardware for scans.")
+    private String groupName = ScanServerSelector.DEFAULT_SCAN_SERVER_GROUP_NAME;
+
+    public String getGroupName() {
+      return groupName;
+    }
+  }
+
+  private static final Logger log = LoggerFactory.getLogger(ScanServer.class);
+
+  private static class TabletMetadataLoader implements CacheLoader<KeyExtent,TabletMetadata> {
+
+    private final Ample ample;
+
+    private TabletMetadataLoader(Ample ample) {
+      this.ample = ample;
+    }
+
+    @Override
+    public @Nullable TabletMetadata load(KeyExtent keyExtent) {
+      long t1 = System.currentTimeMillis();
+      var tm = ample.readTablet(keyExtent);
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for 1 tablet in {} ms", t2 - t1);
+      return tm;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Map<? extends KeyExtent,? extends TabletMetadata>
+        loadAll(Set<? extends KeyExtent> keys) {
+      long t1 = System.currentTimeMillis();
+      var tms = ample.readTablets().forTablets((Collection<KeyExtent>) keys).build().stream()
+          .collect(Collectors.toMap(tm -> tm.getExtent(), tm -> tm));
+      long t2 = System.currentTimeMillis();
+      LOG.trace("Read metadata for {} tablets in {} ms", keys.size(), t2 - t1);
+      return tms;
+    }
+  }
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanServer.class);
+
+  protected ThriftScanClientHandler delegate;
+  private UUID serverLockUUID;
+  private final TabletMetadataLoader tabletMetadataLoader;
+  private final LoadingCache<KeyExtent,TabletMetadata> tabletMetadataCache;
+  // tracks file reservations that are in the process of being added or removed from the metadata
+  // table
+  private final Set<StoredTabletFile> influxFiles = new HashSet<>();
+  // a read lock that ensures files are not removed from reservedFiles while its held
+  private final ReentrantReadWriteLock.ReadLock reservationsReadLock;
+  // a write lock that must be held when mutating influxFiles or when removing entries from
+  // reservedFiles
+  private final ReentrantReadWriteLock.WriteLock reservationsWriteLock;
+  // this condition is used to signal changes to influxFiles
+  private final Condition reservationCondition;
+  // the key is the set of files that have reservations in the metadata table, the value contains
+  // information about which scans are currently using the file
+  private final Map<StoredTabletFile,ReservedFile> reservedFiles = new ConcurrentHashMap<>();
+  private final AtomicLong nextScanReservationId = new AtomicLong();
+
+  private final ServerContext context;
+  private final SessionManager sessionManager;
+  private final TabletServerResourceManager resourceManager;
+  HostAndPort clientAddress;
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+
+  private volatile boolean serverStopRequested = false;
+  private ServiceLock scanServerLock;
+  protected TabletServerScanMetrics scanMetrics;
+
+  private ZooCache managerLockCache;
+
+  private final String groupName;
+
+  public ScanServer(ScanServerOpts opts, String[] args) {
+    super("sserver", opts, args);
+
+    context = super.getContext();
+    log.info("Version " + Constants.VERSION);
+    log.info("Instance " + getContext().getInstanceID());
+    this.sessionManager = new SessionManager(context);
+
+    this.resourceManager = new TabletServerResourceManager(context, this);
+
+    this.managerLockCache = new ZooCache(context.getZooReader(), null);
+
+    var readWriteLock = new ReentrantReadWriteLock();
+    reservationsReadLock = readWriteLock.readLock();
+    reservationsWriteLock = readWriteLock.writeLock();
+    reservationCondition = readWriteLock.writeLock().newCondition();
+
+    // Note: The way to control the number of concurrent scans that a ScanServer will
+    // perform is by using Property.SSERV_SCAN_EXECUTORS_DEFAULT_THREADS or the number
+    // of threads in Property.SSERV_SCAN_EXECUTORS_PREFIX.
+
+    long cacheExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION);
+
+    long scanServerReservationExpiration =
+        getConfiguration().getTimeInMillis(Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME);
+
+    tabletMetadataLoader = new TabletMetadataLoader(getContext().getAmple());
+
+    if (cacheExpiration == 0L) {
+      LOG.warn("Tablet metadata caching disabled, may cause excessive scans on metadata table.");
+      tabletMetadataCache = null;
+    } else {
+      if (cacheExpiration < 60000) {
+        LOG.warn(
+            "Tablet metadata caching less than one minute, may cause excessive scans on metadata table.");
+      }
+      tabletMetadataCache =
+          Caffeine.newBuilder().expireAfterWrite(cacheExpiration, TimeUnit.MILLISECONDS)
+              .scheduler(Scheduler.systemScheduler()).build(tabletMetadataLoader);
+    }
+
+    delegate = newThriftScanClientHandler(new WriteTracker());
+
+    this.groupName = Objects.requireNonNull(opts.getGroupName());
+
+    ThreadPools.watchCriticalScheduledTask(getContext().getScheduledExecutor()
+        .scheduleWithFixedDelay(() -> cleanUpReservedFiles(scanServerReservationExpiration),
+            scanServerReservationExpiration, scanServerReservationExpiration,
+            TimeUnit.MILLISECONDS));
+
+  }
+
+  @VisibleForTesting
+  protected ThriftScanClientHandler newThriftScanClientHandler(WriteTracker writeTracker) {
+    return new ThriftScanClientHandler(this, writeTracker);
+  }
+
+  /**
+   * Start the thrift service to handle incoming client requests
+   *
+   * @return address of this client service
+   * @throws UnknownHostException
+   *           host unknown
+   */
+  protected ServerAddress startScanServerClientService() throws UnknownHostException {
+
+    // This class implements TabletClientService.Iface and then delegates calls. Be sure
+    // to set up the ThriftProcessor using this class, not the delegate.
+    TProcessor processor = ThriftProcessorTypes.getScanServerTProcessor(this, getContext());
+
+    Property maxMessageSizeProperty =
+        (getConfiguration().get(Property.SSERV_MAX_MESSAGE_SIZE) != null
+            ? Property.SSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
+    ServerAddress sp = TServerUtils.startServer(getContext(), getHostname(),
+        Property.SSERV_CLIENTPORT, processor, this.getClass().getSimpleName(),
+        "Thrift Client Server", Property.SSERV_PORTSEARCH, Property.SSERV_MINTHREADS,
+        Property.SSERV_MINTHREADS_TIMEOUT, Property.SSERV_THREADCHECK, maxMessageSizeProperty);
+
+    LOG.info("address = {}", sp.address);
+    return sp;
+  }
+
+  public String getClientAddressString() {
+    if (clientAddress == null) {
+      return null;
+    }
+    return clientAddress.getHost() + ":" + clientAddress.getPort();
+  }
+
+  /**
+   * Set up nodes and locks in ZooKeeper for this Compactor
+   */
+  private ServiceLock announceExistence() {
+    ZooReaderWriter zoo = getContext().getZooReaderWriter();
+    try {
+
+      var zLockPath = ServiceLock.path(
+          getContext().getZooKeeperRoot() + Constants.ZSSERVERS + "/" + getClientAddressString());
+
+      try {
+        // Old zk nodes can be cleaned up by ZooZap
+        zoo.putPersistentData(zLockPath.toString(), new byte[] {}, NodeExistsPolicy.SKIP);
+      } catch (KeeperException e) {
+        if (e.code() == KeeperException.Code.NOAUTH) {
+          LOG.error("Failed to write to ZooKeeper. Ensure that"
+              + " accumulo.properties, specifically instance.secret, is consistent.");
+        }
+        throw e;
+      }
+
+      serverLockUUID = UUID.randomUUID();
+      scanServerLock = new ServiceLock(zoo.getZooKeeper(), zLockPath, serverLockUUID);
+
+      LockWatcher lw = new LockWatcher() {
+
+        @Override
+        public void lostLock(final LockLossReason reason) {
+          Halt.halt(serverStopRequested ? 0 : 1, () -> {
+            if (!serverStopRequested) {
+              LOG.error("Lost tablet server lock (reason = {}), exiting.", reason);
+            }
+            gcLogger.logGCInfo(getConfiguration());
+          });
+        }
+
+        @Override
+        public void unableToMonitorLockNode(final Exception e) {
+          Halt.halt(1, () -> LOG.error("Lost ability to monitor scan server lock, exiting.", e));
+        }
+      };
+
+      // Don't use the normal ServerServices lock content, instead put the server UUID here.
+      byte[] lockContent = (serverLockUUID.toString() + "," + groupName).getBytes(UTF_8);
+
+      for (int i = 0; i < 120 / 5; i++) {
+        zoo.putPersistentData(zLockPath.toString(), new byte[0], NodeExistsPolicy.SKIP);
+
+        if (scanServerLock.tryLock(lw, lockContent)) {
+          LOG.debug("Obtained scan server lock {}", scanServerLock.getLockPath());
+          return scanServerLock;
+        }
+        LOG.info("Waiting for scan server lock");
+        sleepUninterruptibly(5, TimeUnit.SECONDS);
+      }
+      String msg = "Too many retries, exiting.";
+      LOG.info(msg);
+      throw new RuntimeException(msg);
+    } catch (Exception e) {
+      LOG.info("Could not obtain scan server lock, exiting.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void run() {
+    SecurityUtil.serverLogin(getConfiguration());
+
+    ServerAddress address = null;
+    try {
+      address = startScanServerClientService();
+      clientAddress = address.getAddress();
+    } catch (UnknownHostException e1) {
+      throw new RuntimeException("Failed to start the compactor client service", e1);
+    }
+
+    try {
+      MetricsUtil.initializeMetrics(getContext().getConfiguration(), this.applicationName,
+          clientAddress);
+    } catch (Exception e1) {
+      LOG.error("Error initializing metrics, metrics will not be emitted.", e1);

Review Comment:
   Please avoid swallowing overly broad exceptions. RTEs should fall through.



##########
core/src/main/java/org/apache/accumulo/core/spi/scan/ScanServerSelector.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.spi.scan;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.common.ServiceEnvironment;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A client side plugin that determines what scan servers to use for eventually consistent scans.
+ * When a scanner sets
+ * {@link org.apache.accumulo.core.client.ScannerBase#setConsistencyLevel(ScannerBase.ConsistencyLevel)}
+ * to {@link org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel#EVENTUAL} then this plugin
+ * is used to determine which scan servers to use for a given tablet. To configure a class to use
+ * for this plugin set its name using the client config {@code scan.server.selector.impl}
+ *
+ * @since 2.1.0
+ */
+public interface ScanServerSelector {
+
+  /**
+   * The scan server group name that will be used when one is not specified.
+   */
+  public static final String DEFAULT_SCAN_SERVER_GROUP_NAME = "default";
+
+  /**
+   * Information about a scan server.
+   *
+   * @since 2.1.0
+   */
+  public interface ScanServer {
+    /**
+     * @return the address in the form of {@code <host>:<port>} where the scan server is running.
+     */
+    String getAddress();
+
+    /**
+     * @return the group name set when the scan server was started. If a group name was not set for
+     *         the scan server, then the string {@value #DEFAULT_SCAN_SERVER_GROUP_NAME} is
+     *         returned.
+     */
+    String getGroup();
+  }
+
+  /**
+   * This interface exists so that is easier to evolve what is passed to
+   * {@link #init(InitParameters)} without having to make breaking changes.
+   *
+   * @since 2.1.0
+   */
+  public interface InitParameters {

Review Comment:
   There are a lot of nested interfaces inside this class. These should probably be separate files in the same package. Or, if they aren't required to be implemented by the user's SPI implementation, then they could be removed outside of the spi package to reduce user confusion (for example, if they are objects we create and pass to the user's plugin code).



##########
core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java:
##########
@@ -76,7 +76,9 @@ public void testConfiguration() {
     // this is not set for the cache type, so should fall back to default
     cc.set(defaultPrefix + LruBlockCacheConfiguration.MAP_LOAD_PROPERTY, "0.53");
 
-    BlockCacheConfiguration bcc = new BlockCacheConfiguration(cc);
+    BlockCacheConfiguration bcc = new BlockCacheConfiguration(cc, Property.TSERV_PREFIX,
+        Property.TSERV_INDEXCACHE_SIZE, Property.TSERV_DATACACHE_SIZE,
+        Property.TSERV_SUMMARYCACHE_SIZE, Property.TSERV_DEFAULT_BLOCKSIZE);

Review Comment:
   This pattern keeps getting repeated. It's pretty verbose boilerplate to construct the block cache configuration now. It'd probably be easier to pass a flag/enum to specify whether TSERV or SSERV props should be used, and then resolve that logic inside the constructor.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java:
##########
@@ -136,6 +136,11 @@ public List<String> getManagerLocations() {
     return context.getManagerLocations();
   }
 
+  @Override
+  public List<String> getScanServers() {
+    return List.copyOf(context.getScanServers().keySet());
+  }

Review Comment:
   Why is this returning a list instead of a Set?



##########
core/src/test/java/org/apache/accumulo/core/conf/cluster/ClusterConfigParserTest.java:
##########
@@ -121,12 +131,21 @@ public void testShellOutput() throws Exception {
     ClusterConfigParser.outputShellVariables(contents, ps);
     ps.close();
 
-    Map<String,
-        String> expected = Map.of("MANAGER_HOSTS", "\"localhost1 localhost2\"", "MONITOR_HOSTS",
-            "\"localhost1 localhost2\"", "GC_HOSTS", "\"localhost\"", "TSERVER_HOSTS",
-            "\"localhost1 localhost2 localhost3 localhost4\"", "COORDINATOR_HOSTS",
-            "\"localhost1 localhost2\"", "COMPACTION_QUEUES", "\"q1 q2\"", "COMPACTOR_HOSTS_q1",
-            "\"localhost1 localhost2\"", "COMPACTOR_HOSTS_q2", "\"localhost3 localhost4\"");
+    Map<String,String> expected = new HashMap<>();
+    expected.put("MANAGER_HOSTS", "localhost1 localhost2");
+    expected.put("MONITOR_HOSTS", "localhost1 localhost2");
+    expected.put("GC_HOSTS", "localhost");
+    expected.put("TSERVER_HOSTS", "localhost1 localhost2 localhost3 localhost4");
+    expected.put("COORDINATOR_HOSTS", "localhost1 localhost2");
+    expected.put("COMPACTION_QUEUES", "q1 q2");
+    expected.put("COMPACTOR_HOSTS_q1", "localhost1 localhost2");
+    expected.put("COMPACTOR_HOSTS_q2", "localhost3 localhost4");
+    expected.put("SSERVER_GROUPS", "default highmem cheap");
+    expected.put("SSERVER_HOSTS_default", "localhost1 localhost2");
+    expected.put("SSERVER_HOSTS_highmem", "hmvm1 hmvm2 hmvm3");
+    expected.put("SSERVER_HOSTS_cheap", "burstyvm1 burstyvm2");
+
+    expected = Maps.transformValues(expected, v -> '"' + v + '"');

Review Comment:
   Should use built-in stream to transform maps, rather than rely on Guava.



##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -373,6 +384,23 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
     }
   }
 
+  /**
+   * Get the configured consistency level
+   *
+   * @return consistency level
+   * @since 2.1.0
+   */
+  public ConsistencyLevel getConsistencyLevel();
+
+  /**
+   * Set the desired consistency level for this scanner.
+   *
+   * @param level
+   *          consistency level
+   * @since 2.1.0
+   */
+  public void setConsistencyLevel(ConsistencyLevel level);
+

Review Comment:
   I'm still not comfortable with this terminology in the API, but I don't have an alternative at this time, so I'll consider this settled for now.



##########
core/src/main/java/org/apache/accumulo/core/metadata/ScanServerRefTabletFile.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.metadata;
+
+import java.util.UUID;
+
+import org.apache.accumulo.core.data.Value;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+public class ScanServerRefTabletFile extends TabletFile {
+
+  private final Value NULL_VALUE = new Value(new byte[0]);
+  private final Text colf;
+  private final Text colq;
+
+  public ScanServerRefTabletFile(String file, String serverAddress, UUID serverLockUUID) {
+    super(new Path(file));
+    this.colf = new Text(serverAddress);
+    this.colq = new Text(serverLockUUID.toString());
+  }
+
+  public ScanServerRefTabletFile(String file, Text colf, Text colq) {
+    super(new Path(file));
+    this.colf = colf;
+    this.colq = colq;
+  }
+
+  public String getRowSuffix() {
+    return this.getPathStr();
+  }
+
+  public Text getServerAddress() {
+    return this.colf;
+  }
+
+  public Text getServerLockUUID() {
+    return this.colq;
+  }
+
+  public Value getValue() {
+    return NULL_VALUE;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((colf == null) ? 0 : colf.hashCode());
+    result = prime * result + ((colq == null) ? 0 : colq.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ScanServerRefTabletFile other = (ScanServerRefTabletFile) obj;
+    if (colf == null) {

Review Comment:
   Or just `return Objects.equals(field1, other.field1) && Objects.equals(field2, other.field2) && ...`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r940265400


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanAttemptsImpl.class);
+
+  static class ScanAttemptImpl
+      implements org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt {
+
+    private final String server;
+    private final long time;
+    private final Result result;
+    private volatile long mutationCount = Long.MAX_VALUE;
+
+    ScanAttemptImpl(Result result, String server, long time) {
+      this.result = result;
+      this.server = Objects.requireNonNull(server);
+      this.time = time;
+    }
+
+    @Override
+    public String getServer() {
+      return server;
+    }
+
+    @Override
+    public long getEndTime() {
+      return time;
+    }
+
+    @Override
+    public Result getResult() {
+      return result;
+    }
+
+    private void setMutationCount(long mc) {
+      this.mutationCount = mc;
+    }
+
+    public long getMutationCount() {
+      return mutationCount;
+    }
+  }
+
+  private Map<TabletId,Collection<ScanAttemptImpl>> attempts = new ConcurrentHashMap<>();
+  private long mutationCounter = 0;
+
+  private void add(TabletId tablet, ScanAttempt.Result result, String server, long endTime) {
+
+    ScanAttemptImpl sa = new ScanAttemptImpl(result, server, endTime);
+
+    attempts.computeIfAbsent(tablet, k -> ConcurrentHashMap.newKeySet()).add(sa);
+
+    synchronized (this) {
+      // now that the scan attempt obj is added to all concurrent data structs, make it visible
+      // need to atomically increment the counter AND set the counter on the object
+      sa.setMutationCount(mutationCounter++);
+    }
+
+  }
+
+  public static interface ScanAttemptReporter {
+    void report(ScanAttempt.Result result);
+  }
+
+  ScanAttemptReporter createReporter(String server, TabletId tablet) {
+    return new ScanAttemptReporter() {
+      @Override
+      public void report(ScanAttempt.Result result) {
+        LOG.trace("Received result: {}", result);
+        add(tablet, result, server, System.currentTimeMillis());
+      }
+    };
+  }
+
+  Map<TabletId,Collection<ScanAttemptImpl>> snapshot() {
+    // allows only seeing scan attempt objs that were added before this call
+
+    long snapMC;
+    synchronized (ScanAttemptsImpl.this) {
+      snapMC = mutationCounter;
+    }
+    var tmp = Maps.transformValues(attempts, tabletAttemptList -> Collections2
+        .filter(tabletAttemptList, sai -> sai.getMutationCount() < snapMC));
+
+    return Maps.filterEntries(tmp, entry -> !entry.getValue().isEmpty());

Review Comment:
   Created #2856 to address this in the future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1113302935

   ## Background
   
   Accumulo TabletServers are responsible for:
   
     1. ingesting new data
     2. compacting (merging) new and old data into files
     3. reading data from files to support system and user scans
     4. performing maintenance on Tablets (assignments, merging, splitting, bulk importing, etc).
   
   To support these activities newly ingested data is hosted in memory (in-memory maps) until it's written to a file, and blocks of accessed files may be cached within the TabletServer for better performance. The TabletServer has configuration properties to control the amout of memory available to the heap, in-memory maps, and block caches, and the size of the various thread pools that perform these activities. For example:
   
       tserver.assignment.concurrent.max
       tserver.bulk.process.threads
       tserver.cache.data.size
       tserver.cache.index.size
       tserver.cache.summary.size
       tserver.compaction.major.concurrent.max
       tserver.compaction.minor.concurrent.max
       tserver.memory.maps.max
       tserver.migrations.concurrent.max
       tserver.recovery.concurrent.max
       tserver.scan.executors.default.threads
       tserver.scan.executors.meta.threads
       tserver.scan.files.open.max
       tserver.server.threads.minimum
       tserver.sort.buffer.size
       tserver.summary.partition.threads
       tserver.summary.remote.threads
       tserver.total.mutation.queue.max
       tserver.workq.threads
   
   When a TabletServer exhausts available memory, for whatever reason, an OutOfMemoryError will be raised and the TabletServer will be terminated. When this happens all of the running scans on that TabletServer are paused while the Tablets are re-hosted and then the scans continue on the new TabletServers once the re-hosting process is complete. If the cause of the TabletServer failure was due to scans on a particular Tablet, then this process will repeat until there are no TabletServers remaining or the pattern is identified by a user/admin and the scan process is terminated.
   
   ## Objective
   
   Provide Accumulo users with the ability to run scans without terminating the TabletServer.
   
   ## Possible approaches
   
     1. Run the scan in a separate process
     2. Restrict memory usage on a per-scan basis
     3. Read directly from files in client side scan code.  This approach does not allow a small number of clients to scale out a large number of expensive queries to tablet and/or scan servers.  It also may lead to an OOM killing a client process that may be executing multiple concurrent scans.  It also does not allow client to leverage cache of data and metadata on a scan server or tablet server.
   
   ## This approach
   
   Create a separate server process that is used to run user scans and give the user the option whether or not to use the new server process on a per-scan basis. Provide the user with the ability to control how many scans will be affected if this new process dies and how many of these new processes to use for a single scan.
   
   ## Implementation
   
   This PR includes:
   
     1. a new server process called the ScanServer.
     2. changes to the Accumulo client
     3. changes to the GarbageCollector
     4. Ancillary changes
   
   ### Scan Server
   
   The ScanServer is a TabletHostingServer that hosts SnapshotTablets and implements the TabletScanClientService Thrift API. When the ScanServer receives a request via the scan API, it creates a SnapshotTablet object from the Tablet metadata (which may be cached), and then uses the ThriftScanClientHandler to complete the scan operations. The user scan is run using the same code that the TabletServer uses; the ScanServer is just responsible for ensuring that the Tablet exists for the scan code. The Tablet hosted within the ScanServer may not contain the exact same data as the corresponding Tablet hosted by the TabletServer. The ScanServer does not have any of the Tablet data that may reside within the in-memory maps and the Tablet may reference files that have been compacted as Tablet metadata can be cached within the ScanServer (see Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION). The number of concurrent scans that the ScanServer will run is configurable (Property.SSERV_SCAN_EXECU
 TORS_DEFAULT_THREADS and Property.SSERV_SCAN_EXECUTORS_PREFIX). The ScanServer has other configuration properties that can be set to allow it to have different settings than the TabletServer (Thrift, block caches, etc). It is also possible that a ScanServer may be hosting multiple versions of a SnapshotTablet in the case where scans are in progress, the TabletMetadata has expires, and a new scan request arrives.
   
   Scan servers implement a busy timeout parameter on their scan RPCs.  The busytimeout allows a client to specify a configurable time during which the scan must either start running or throw a busy thrift exception.  On the client side this busy exception can be detected and a different scan server selected.
   
   ### Client changes
   
   A new method has been added to the client (ScannerBase.setConsistencyLevel) to configure the client to use IMMEDIATE (default) or EVENTUAL consistency for scans. IMMEDIATE means that the user wants to scan all data related to the Tablet at the time of the scan. To accomplish this the client will send the scan request to the TabletServer that is hosting the Tablet. This is the current behavior and is the default configuration, so no code change is required to have the same behavior. The other possible value, EVENTUAL, means that the user is willing to relax the data freshness guarantee that the TabletServer provides and instead potentially improve the chances of their scan completing when their scan is known to take a long time or require a lot of memory. When the consistency level is set to EVENTUAL the client uses a ScanServerDispatcher class to determine which ScanServers to use. The user can supply their own ScanServerDispatcher implementation (ClientProperty.SCAN_SERVER_DISPAT
 CHER) if they don't want to use the DefaultScanServerDispatcher (see class javadoc for a description of the behavior). Scans will be sent to the TabletServer in the event that EVENTUAL consistency is selected for the client and no ScanServers are running.
   
   #### Default scan server dispatcher
   
   The default scan server dispatcher that executes on the client side has the following strategy for selecting a scan server.
   
    * It hashes a tablets tableId, end row, and prev endrow.  This hash is used to consitently map the tablet to one of three random scan servers.  So for a given tablet the same three random scan servers are used by different tablets.
    * The client sends a request to one of the three scan servers with a small busytimeout.
    * If a busytimeout exception happens, then the default scan server dispatcher will notice this and it will choose from a larger set of scan servers.
    * The default scan server dispatcher will expand rapidly to randomly selecting from all scan servers after which point it will start exponentially increasing the busy timeout.
   
   For example if there are 1000 scan servers and a lot of them are busy, the default scan dispatcher might do something like the following.  This example shows how it will rapidly increase the set of servers chosen from and then start rapidly increasing the busy timeout.  The reason to start increasing the busy timeout after observing a lot busy exceptions is that those provide evidence that the entire cluster of scan servers may be busy. So eventually its better to just go to a scan server and queue up rather look for a non-busy scan server.
   
    1. Choose scan server S1 from 3 random scan servers with a busy timeout of 33ms.
    2. If a busy exceptions happens. Choose scan server S2 from 21 random scan servers with a busy timeout of 33ms.
    3. If a busy exceptions happens. Choose scan server S3 from 147 random scan servers with a busy timeout of 33ms.
    4. If a busy exceptions happens. Choose scan server S4 from 1000 random scan servers with a busy timeout of 33ms.
    5. If a busy exceptions happens. Choose scan server S5 from 1000 random scan servers with a busy timeout of 66ms.
    6. If a busy exceptions happens. Choose scan server S6 from 1000 random scan servers with a busy timeout of 132ms.
   
   This default behavior makes tablets sticky to scan servers which is good for cache utilization and reusing cached tablet metadata. In the case where those few scan servers are busy the client starts searching for other places to run.
   
   ### Garbage Collector changes
   
   The ScanServer inserts entries into a new section (~sserv) of the metadata table to place a reservation on the file so that the GarbageCollector process does not remove the files that are being used for the scan. Accordingly GCEnv.getReferences has been modified to include these file reservations in the list of active file references. The ScanServer has a background thread that removes the file reservations from the metadata table after some period of time after the file is no longer used (see Property.SSERVER_SCAN_REFERENCE_EXPIRATION_TIME). The Manager has a new background thread that calls the ScanServerMetadataEntries.clean method on a periodic basis. Users can use the ScanServerMetadataEntries utility to remove file reservations that exist in the metadata table with no corresponding running ScanServer.
   
   In order to avoid race conditions with the Accumulo GC, Scan servers use the following algorithm when first reading a tablets metadata.
   
    1. Read metadata for tablet
    2. Write an ~sserv entries for the tablets files to the metadata table to prevent GC
    3. Read the meadata again and see if it changed.  If it did changes delete the entries from step 2 and go back to step 1.
   
   The above algorithm may be a bit expensive the first time a tablet is scanned on scan server.  However subsequent scans of the same tablet will use cached tablet metadata for a configurable time and not repeate the above steps.  In the future we may want to look into faster ways of preventing GC of files used by scan servers.
   
   ### Ancillary changes
   
     1. Modifications to scripts (accumulo-cluster, accumulo-service and accumulo-env.sh) have been made to start/stop one or more ScanServers per host. 
     2. The shell commands `grep` and `scan` have been modified to accept a consistency level (`cl`) argument
     3. The shell command `listscans` has been modified to include scans running on ScanServers
     4. ZooZap has been modified to remove ScanServer entries in ZooKeeper
     5. MiniAccumuloCluster has been modified to include the ability to start/stop ScanServers (used by the ITs)
     6. A new utility (ScanServerMetadataEntries) has been created to cleanup any dangling scan server file references in the metadata table.
   
   ## Shell Example
   
   Below is an example of how this works using the `scan` command in the shell.
   
   ```
   root@test> createtable test (1)
   root@test test> insert a b c d (2)
   root@test test> scan (3)
   a b:c []	d
   root@test test> scan -cl immediate (4)
   a b:c []	d
   root@test test> scan -cl eventual (5)
   root@test test> flush (6)
   2022-01-28T18:58:10,693 [shell.Shell] INFO : Flush of table test  initiated...
   root@test test> scan (7)
   a b:c []	d
   root@test test> scan -cl eventual (8)
   a b:c []	d
   ```
   
   In this example, I create a table (1) and insert some data (2). When I run a scan (3,4) with the immediate consistency level, which happens to be the default, the client uses the normal code path and issues the scan command against the Tablet Server. Data is returned because the Tablet Server code path also returns data that is in the in-memory map. When I scan with the eventual consistency level (5) no data is returned because the Scan Server only uses the data in the Tablet's files. When I flush (6) the data to write a file in HDFS, the subsequent scans with immediate (7) and eventual (8) consistency level return the data.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r864975982


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   For the first question, if the consistency level is set to EVENTUAL and there are no ScanServers, then the code will send the scans to the TabletServers.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r865380896


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchReaderIterator.java:
##########
@@ -453,26 +482,40 @@ public void run() {
   private void doLookups(Map<String,Map<KeyExtent,List<Range>>> binnedRanges,
       final ResultReceiver receiver, List<Column> columns) {
 
-    if (timedoutServers.containsAll(binnedRanges.keySet())) {
-      // all servers have timed out
-      throw new TimedOutException(timedoutServers);
-    }
-
-    // when there are lots of threads and a few tablet servers
-    // it is good to break request to tablet servers up, the
-    // following code determines if this is the case
     int maxTabletsPerRequest = Integer.MAX_VALUE;
-    if (numThreads / binnedRanges.size() > 1) {
-      int totalNumberOfTablets = 0;
-      for (Entry<String,Map<KeyExtent,List<Range>>> entry : binnedRanges.entrySet()) {
-        totalNumberOfTablets += entry.getValue().size();
-      }
 
-      maxTabletsPerRequest = totalNumberOfTablets / numThreads;
-      if (maxTabletsPerRequest == 0) {
-        maxTabletsPerRequest = 1;
+    long busyTimeout = 0;
+    Duration scanServerDispatcherDelay = null;
+    Map<String,ScanAttemptsImpl.ScanAttemptReporter> reporters = Map.of();
+
+    if (options.getConsistencyLevel().equals(ConsistencyLevel.EVENTUAL)) {

Review Comment:
   To make the ScanServerDispatcher support multiple threads would need to modify the interface `ScanServerDispatcher.Actions` to allow communicating intent about threads.  That interface is how a dispatcher communicates what it wants done.  Would probably also need to modify `ScanServerDispatcher.DispatcherParameters` to pass in information about the number of threads available.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881725923


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMultipleScansIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+    }
+  }
+
+  private static final int NUM_SCANS = 4;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @Test
+  public void testMutipleScansSameTablet() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      var executor = Executors.newCachedThreadPool();
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRange(new Range());
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+      executor.shutdown();
+    }
+  }
+
+  @Test
+  public void testSingleScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+      }
+    }
+  }
+
+  @Test
+  public void testMultipleScansDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+      System.out.println(splits);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      final AtomicInteger counter = new AtomicInteger(0);
+
+      var executor = Executors.newCachedThreadPool();

Review Comment:
   Implemented suggestion in 5c039df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r880869326


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -44,6 +44,17 @@
  */
 public interface ScannerBase extends Iterable<Entry<Key,Value>>, AutoCloseable {
 
+  /**
+   * Consistency level for the scanner. The default level is IMMEDIATE, which means that this

Review Comment:
   It's not just data that has not been minor compacted. The ScanServer contains a cache of metadata entries. If the metadata entries are cached for a Tablet, then when the ScanServer creates a SnapshotTablet, it will be from those files, not necessarily the up-to-date list of files in the metadata table. The tablet metadata cache expiration is set using Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881724768


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +340,75 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  /**
+   * @return map of live scan server addresses to lock uuids.
+   */
+  public Map<String,UUID> getScanServers() {
+    Map<String,UUID> liveScanServers = new HashMap<>();
+    String root = this.getZooKeeperRoot() + Constants.ZSSERVERS;
+    var addrs = this.getZooCache().getChildren(root);
+    for (String addr : addrs) {
+      try {
+        final var zLockPath = ServiceLock.path(root + "/" + addr);
+        ZcStat stat = new ZcStat();
+        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (lockData != null) {
+          UUID uuid = UUID.fromString(new String(lockData, UTF_8));
+          liveScanServers.put(addr, uuid);
+        }
+      } catch (Exception e) {
+        log.error("Error validating zookeeper scan server node: " + addr, e);
+      }
+    }
+    return liveScanServers;
+  }
+
+  /**
+   * @return the scan server dispatcher implementation used for determining which scan servers will
+   *         be used when performing an eventually consistent scan
+   */
+  public synchronized ScanServerDispatcher getScanServerDispatcher() {
+    ensureOpen();
+    if (scanServerDispatcher == null) {
+      String clazz = ClientProperty.SCAN_SERVER_DISPATCHER.getValue(info.getProperties());

Review Comment:
   Addressed in 5c039df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r883197611


##########
shell/src/main/java/org/apache/accumulo/shell/commands/ScanCommand.java:
##########
@@ -419,6 +438,7 @@ public Options getOptions() {
     outputFileOpt.setArgName("file");
     contextOpt.setArgName("context");
     executionHintsOpt.setArgName("<key>=<value>{,<key>=<value>}");
+    scanServerOpt.setArgName("immediate|eventual");

Review Comment:
   >That makes sense if the desired primary outcome was "eventual consistency". It's not, though. It's merely an acceptable side-effect of what the user is really asking for, and not its essential/sufficient characteristic.
   
   I agree that a developer will enable eventual consistency because they want better performance and/or availability and they are ok with stale data.  One important thing to consider is that for evaluating code for correctness (like I am reviewing Accumulo code that someone else wrote) is that the consistency level of the data returned by the scanner is extremely important (probably more important than the performance benefits desired by the author).  The fact that a scanner may return stale data needs to be very clearly communicated when considering other names.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884062198


##########
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java:
##########
@@ -474,6 +474,8 @@ public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long
   public ThreadPoolExecutor createThreadPool(int coreThreads, int maxThreads, long timeOut,
       TimeUnit units, final String name, BlockingQueue<Runnable> queue, OptionalInt priority,
       boolean emitThreadPoolMetrics) {
+    LOG.debug("Creating ThreadPoolExecutor for {} with {} core threads and {} max threads", name,
+        coreThreads, maxThreads);

Review Comment:
   Opened #2745 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ivakegg commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ivakegg commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r861803116


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -44,6 +44,17 @@
  */
 public interface ScannerBase extends Iterable<Entry<Key,Value>>, AutoCloseable {
 
+  /**
+   * Consistency level for the scanner. The default level is IMMEDIATE, which means that this

Review Comment:
      * Consistency level for the scanner. The default level is IMMEDIATE, which means that this
      * scanner will see ALL keys and values that have been successfully written to a TabletServer.
      * EVENTUAL means that the scanner may not see the latest data that was written to a TabletServer,
      * but may instead see an older version of data.  Data that has not yet been minor compacted will
      * not be seen.



##########
core/src/main/java/org/apache/accumulo/core/file/blockfile/cache/impl/BlockCacheConfiguration.java:
##########
@@ -41,13 +42,16 @@ public class BlockCacheConfiguration implements Configuration {
 
   private final long summaryMaxSize;
 
-  public BlockCacheConfiguration(AccumuloConfiguration conf) {
-    genProps = conf.getAllPropertiesWithPrefix(Property.TSERV_PREFIX);
+  public BlockCacheConfiguration(AccumuloConfiguration conf, Property serverPrefix,

Review Comment:
   Just an idea, but the code could be a little less susceptible  to coding error if we pass in a single flag for TSERVER vs SSERVER.  The properties used for each could be encoded in this class.  This is just a thought and does not require change unless you agree with my argument.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r865172579


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java:
##########
@@ -358,7 +380,22 @@ else if (log.isTraceEnabled())
           }
 
           TraceUtil.setException(child2, e, false);
-          sleepMillis = pause(sleepMillis, maxSleepTime);
+          sleepMillis = pause(sleepMillis, maxSleepTime, scanState.runOnScanServer);
+        } catch (ScanServerBusyException e) {
+          error = "Scan failed, scan server was busy " + loc;

Review Comment:
   While testing I encountered an exception on the scan server.  The error message on the client side indicated there was an error on the tserver.  That is because this loc is the tserver.  The code that generated the error message I saw is further up, it was from the following line.
   
   ```
   throw new AccumuloServerException(loc.tablet_location, tae);
   ```
   
   All of the catch blocks in this try that use the loc may have have the same problem.  We may be able to use `scanState.prevLoc` instead of loc as this is always set to the actual server before making the rpc call.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion merged pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion merged PR #2665:
URL: https://github.com/apache/accumulo/pull/2665


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939052009


##########
server/tserver/src/main/java/org/apache/accumulo/tserver/scan/ScanTask.java:
##########
@@ -127,9 +154,6 @@ public T get(long timeout, TimeUnit unit)
     // returned
     resultQueue = null;
 
-    if (r instanceof Error)
-      throw (Error) r; // don't wrap an Error
-

Review Comment:
   The places where ScanTask.addResult are called from (LookupTask.run and NextBatchTask.run) where previously [modified](https://github.com/apache/accumulo/commit/bbd87a6693fc5cdbbe947b0821a4f05a18cf905b) to not catch `Throwable` and only catch `Exception`. `Error`'s that are thrown from `LookupTask` and `NextBatchTask` should now be handled by the `AccumuloUncughtExceptionHandler`. `Error`'s should not be on the queue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939103048


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
+    try {
+      Scanner scanner = context.createScanner(DataLevel.USER.metaTable(), Authorizations.EMPTY);
+      scanner.setRange(ScanServerFileReferenceSection.getRange());
+      int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
+      return StreamSupport.stream(scanner.spliterator(), false)
+          .map(e -> new ScanServerRefTabletFile(e.getKey().getRowData().toString().substring(pLen),
+              e.getKey().getColumnFamily(), e.getKey().getColumnQualifier()));
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   Almost every other method in ServerAmpleImpl throws `RuntimeException`. I will create an issue to fix ServerAmpleImpl.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] DomGarguilo commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
DomGarguilo commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939182111


##########
test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.hadoop.io.Text;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerMultipleScansIT extends SharedMiniClusterBase {
+
+  private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+    }
+  }
+
+  private static final int NUM_SCANS = 4;
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerITConfiguration c = new ScanServerITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER,
+        "localhost");
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  @BeforeEach
+  public void before() throws Exception {
+    executor = Executors.newCachedThreadPool();
+  }
+
+  @AfterEach
+  public void after() throws Exception {
+    executor.shutdown();
+  }
+
+  private ExecutorService executor;
+
+  @Test
+  public void testMutipleScansSameTablet() throws Exception {
+
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRange(new Range());
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+    }
+  }
+
+  @Test
+  public void testSingleScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+        scanner.setRange(new Range());
+        scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+        assertEquals(100, Iterables.size(scanner));
+      }
+    }
+  }
+
+  @Test
+  public void testMultipleScansDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);
+      assertEquals(3, splits.size());
+      System.out.println(splits);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      final AtomicInteger counter = new AtomicInteger(0);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+
+      for (int i = 0; i < NUM_SCANS; i++) {
+        final int threadNum = i;
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY)) {
+            switch (threadNum) {
+              case 0:
+                scanner.setRange(new Range("row_0000000000", "row_0000000002"));
+                break;
+              case 1:
+                scanner.setRange(new Range("row_0000000003", "row_0000000005"));
+                break;
+              case 2:
+                scanner.setRange(new Range("row_0000000006", "row_0000000008"));
+                break;
+              case 3:
+                scanner.setRange(new Range("row_0000000009"));
+                break;
+              default:
+                fail("Invalid threadNum");
+            }
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+            counter.addAndGet(Iterables.size(scanner));
+
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+
+      assertEquals(100, counter.get());
+    }
+  }
+
+  @Test
+  public void testMutipleBatchScansSameTablet() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, tableName);
+
+      client.tableOperations().flush(tableName, null, null, true);
+
+      final CountDownLatch latch = new CountDownLatch(1);
+
+      List<Future<?>> futures = new ArrayList<>(NUM_SCANS);
+
+      for (int i = 0; i < NUM_SCANS; i++) {
+        var future = executor.submit(() -> {
+          try {
+            latch.await();
+          } catch (InterruptedException e1) {
+            fail("InterruptedException waiting for latch");
+          }
+          try (BatchScanner scanner = client.createBatchScanner(tableName, Authorizations.EMPTY)) {
+            scanner.setRanges(Collections.singletonList(new Range()));
+            scanner.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+            assertEquals(100, Iterables.size(scanner));
+          } catch (TableNotFoundException e) {
+            fail("Table not found");
+          }
+        });
+        futures.add(future);
+      }
+      latch.countDown();
+      for (Future<?> future : futures) {
+        future.get();
+      }
+    }
+
+  }
+
+  @Test
+  public void testSingleBatchScanDifferentTablets() throws Exception {
+    try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+      SortedSet<Text> splitPoints = new TreeSet<>();
+      splitPoints.add(new Text("row_0000000002\\0"));
+      splitPoints.add(new Text("row_0000000005\\0"));
+      splitPoints.add(new Text("row_0000000008\\0"));
+      client.tableOperations().addSplits(tableName, splitPoints);
+
+      @SuppressWarnings("deprecation")
+      Collection<Text> splits = client.tableOperations().getSplits(tableName);

Review Comment:
   ```suggestion
         Collection<Text> splits = client.tableOperations().listSplits(tableName);
   ```
   Why not just use `listSplits()` for these instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939180684


##########
core/src/test/java/org/apache/accumulo/core/file/blockfile/cache/TestLruBlockCache.java:
##########
@@ -76,7 +76,9 @@ public void testConfiguration() {
     // this is not set for the cache type, so should fall back to default
     cc.set(defaultPrefix + LruBlockCacheConfiguration.MAP_LOAD_PROPERTY, "0.53");
 
-    BlockCacheConfiguration bcc = new BlockCacheConfiguration(cc);
+    BlockCacheConfiguration bcc = new BlockCacheConfiguration(cc, Property.TSERV_PREFIX,
+        Property.TSERV_INDEXCACHE_SIZE, Property.TSERV_DATACACHE_SIZE,
+        Property.TSERV_SUMMARYCACHE_SIZE, Property.TSERV_DEFAULT_BLOCKSIZE);

Review Comment:
   Addressed in 9ec7f5f. I modified BlockCacheConfiguration to create two static methods that returned either a configuration for the TabletServer or the ScanServer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1157220591

   @keith-turner wrote:
   > I am not sure if you are implying the client side plugin should have control over choosing tservers and sservers. If so, I would like to avoid that and keep the plugin narrowly scoped to choosing scan servers because of the following :
   
   That was what I was suggesting. The advantage of flattening the decision of choosing between tservers and sservers, and choosing among the sservers is that there is only one branching point when you zoom out and look at the the server selection logic, instead of two. One simplifies the bigger picture, but potentially makes the plugin more complicated. Keeping those decisions separate makes the zoomed out view look more complicated, but the job of the plugin is simpler.
   
   Having one decision point also enables more complex selection features in the plugin, like "I don't care if I get a tserver or a sserver... treat them equally", or "try a tserver first, but settle for a sserver if the tserver's load is high". The plugin can't do that if it is narrowly focused on a decision after the tserver is excluded.
   
   > * Any scan server can be chosen to service a query for a tablet.  Only one tserver can be chosen to service a tablet scan.
   
   The selector plugin does not need to be responsible for the all the logic that identifies the one tserver. It can be provided with a Supplier that executes our current logic, so it can have the option of selecting the tserver, but without all the complexity of locating it.
   
   > * Scan servers have a busy timeout and tservers do not.  The plugin specifies the busy timeout to use.
   > * History of busy timeout events is given to the plugin.  This allows it to possibly choose a different scan server based on past events.
   
   I don't think there's any reason a tserver can't have some of those features, in case a selector plugin wanted to treat the tserver as another possible scan server to choose from. Those features wouldn't be of much use if immediate consistency were required... but if it's not required, it would certainly be acceptable for a selector to choose the tserver if the sservers are busy or unavailable.
   
   I'm also wondering if the "busy timeout" concept can be made more generalized. Like, instead of a queue wait timeout, a sserver could be considered "busy" if its CPU load was high or something else. Computing this weight could be another SPI added in future. The first pass could just be the current "busy timeout"... just with a more generic name, so it doesn't strictly have to be a timeout in future, but some other selection weight.
   
   > Also the logic for choosing a tserver is not flexible and there is basically only one way to do it ATM.
   
   As explained here, I'm not proposing that we diverge from the current one way to do this. I'm only proposing that the selector be allowed to select it.
   
   > I think it makes sense to pass the scan exec hint
   
   I concede this point. The selection of the sserver is still part of the overall execution of the scan, and could make use of these hints, even if they are not used to determine whether the tserver is selected or the sservers are selected.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1152861564

   > > I'm thinking it would be better to leverage the scan hints to control a ScanServer-aware dispatcher, rather than add a new API for the consistency level.
   > 
   > I think this goes against the purpose of scan execution hints. They were created to modify execution behavior like priority, caching, and thread pool selection. They were never intended to change anything about data returned, it says so in their javadocs.
   
   That's only because we didn't have even the possibility of returning data that wasn't immediately consistent before. Servers that returned data never previously had the option of returning data that was stale before. But, now we have a whole new server type that we can dispatch to. It's not the scan execution hints that are modifying the behavior... it's the configured dispatcher. And, the scan hints are still not affecting the data returned... it's the server that it was dispatched to that is doing that.
   
   A scan hint that explicitly says the eventual consistency is tolerable seems perfectly reasonable to me. It fits very naturally into the whole design of scan hints affecting dispatching. And the ability for a dispatcher to use ScanServers instead of TabletServers also seems perfectly natural. No fundamental design changes at all, and no special-purpose APIs needed to support the feature. The feature all works with the existing design elements wired together in a particular way.
   
   We can easily update the javadoc to clarify that the scan hints affect how the scan is dispatched only, and not the data, but that the dispatcher could dispatch to a server that provides stale data (in the case of scan servers) if the scan hint specified to do so.
   
   > 
   > https://github.com/apache/accumulo/blob/d5f81877fcc794c8158f38b840d02331e3c563dc/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java#L342-L361
   > 
   > Slightly related I created a new default scan server dispatcher. Its currently a PR against Dave's branch: [dlmarion#29](https://github.com/dlmarion/accumulo/pull/29). When running 100+ test scenarios this is what I realized I wanted. I wish I had had it when running all of those test, I could have a run a few more test that I wanted to but could not. This new dispatcher is completely configuration driven (replacing the algorithm the previous default dispatcher had) and can be influenced by scan execution hints. If we merge this PR, I could close the PR on Dave's fork and make a PR on the main Accumulo GH.
   
   The idea of creating a custom dispatcher that would work with the scan servers is exactly what I had in mind. However, I don't think it should be the default.
   
   I think in order to leverage scan servers, the user should:
   1. Run some ScanServers,
   2. Configure a `table.scan.dispatcher` to an implementation that is ScanServer-aware, and
   3. Configure individual scans with the scan execution hint recognized by that dispatcher to instruct it to dispatch to ScanServers
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1154436966

   > I'm also thinking that there should be a table configuration that enables/disables this feature. Currently, an admin can spin up some ScanServers and an application developer can enable eventual consistency, but do we want that on the metadata table for example?
   
   For a table where that are zero known uses cases for eventual consistency and enabling it is likely a mistake having that feature makes sense to me.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1153850271

   > They [scan hints] were never intended to change anything about data returned
   
   I equate scan hints with Oracle's [optimizer hints](https://docs.oracle.com/cd/B19306_01/server.102/b14211/hintsref.htm#i17496) which allow the user to alter the execution plan for the query. I agree that scan hints should affect how the scan gets executed and not affect the results being returned.
   
   > It's weird to ask for eventual consistency. Nobody wants eventual consistency. Eventual consistency is always tolerated, never desired.
   
   I think setting the consistency level is the means for achieving a specific goal. Some applications may require strict consistency for correctness and others may be more concerned with speed. I looked for how other products exposed consistency levels in their api. Specifically I was looking for products that used consistency level and not some other name to show that it's not uncommon. This is by no means a complete list, but just some examples of products that document their consistency levels and then provide a method for setting the consistency level in their api.
   
   1. [Microsoft Cosmos DB](https://docs.microsoft.com/en-us/azure/cosmos-db/sql/how-to-manage-consistency?tabs=portal%2Cdotnetv2%2Capi-async#configure-the-default-consistency-level) uses a method called `consistencyLevel`
   2. [Amazon DynamoDB](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/HowItWorks.ReadConsistency.html) has read operations that take a `ConsistentRead` parameter
   3. [HashiCorp Consul](https://www.consul.io/api-docs/features/consistency#overriding-a-request-s-consistency-mode) has a HTTP API query parameter for specifying the consistency level
   
   Since immediate/strict consistency is the default, maybe we just need a method to disable it for a specific query instead of specifying the value. For example, `enableEventualConsistency()`, `relaxReadGuarantees()`, `disableConsistentReads()`, `allowStaleReads()`, etc.
   
   I'm also thinking that there should be a table configuration that enables/disables this feature. Currently, an admin can spin up some ScanServers and an application developer can enable eventual consistency, but do we want that on the `metadata` table for example?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1156635752

   > The client side dispatcher concept is very different from the executor dispatching that is done in the tserver, but has a very similar name. It might be helpful to have this named completely differently... like "server chooser" or "tablet scanner server type selector"
   
   I kinda like `ScanServerChooser`
   
   > So, that does imply a new kind of SPI or configuration to do the server selection inside the client (or... a different client entirely rather than modify the existing client).
   
   I am not sure if you are implying the client side plugin should have control over choosing tservers and sservers.  If so, I would like to avoid that and keep the plugin narrowly scoped to choosing scan servers because of the following : 
   
    * Any scan server can be chosen to service a query for a tablet.  Only one tserver can be chosen to service a tablet scan.
    * Scan servers have a busy timeout and tservers do not.  The plugin specifies the busy timeout to use.
    * History of busy timeout events is given to the plugin.  This allows it to possibly choose a different scan server based on past events.
   
   The way we choose which tserver vs which scan server is very different and I don't think it would be good to try to have one plugin do both.  Also the logic for choosing a tserver is not flexible and there is basically only one way to do it ATM.
   
   Working on this I have realized if we did have anything pluggable for tservers, it would probably not be around choosing a tserver but more about backoff strategies in the case of failures. I think that would be another narrowly scoped plugin that makes very specific decisions.
   
   > Those are specifically scan executor hints, and should be used only by the dispatcher inside the server, because the dispatcher inside the server dispatches to an executor.
   
   I think it makes sense to pass the scan exec hint so the ScanServerChooser/ScanServerDispatcher plugin in addition to plugins dealing with caching, prioritizationm and thread pool selection on the server side.  Consider the case where in the code I set scan_hints to either `scan_type=gold`, `scan_type=silver`, or `scan_type=iron`. I could start off configuring multiple run time plugins to do the following (on tserver and scan server).
   
     * When we see scan_type=gold enable full caching, use a dedicated thread pool A with 32 threads
     * When we see scan_type=silver enable opportunistic caching, use a thread pool B with 8 threads, set the scan prio to 1 in the thread pool queue
     * When we see scan_type=iron enable disable caching, and use a thread pool B with 8 threads, set the scan prio to 2 in the thread pool queue
   
   Then later I could change config at runtime to react to the scan types differently like
   
   * When we see scan_type gold and its eventual, then use a dedicated group of scan servers with large memory and full caching enabled
   * When we see scan_type silver and its eventual use the default set of scan servers.  On the scan server enable caching, use a thread pool B with 8 threads, set the scan prio to 1 in the thread pool queue for this scan type.
   * When we see scan_type iron and its eventual use the default set of scan servers.  On the scan server disable caching, use a thread pool B with 8 threads, set the scan prio to 2 in the thread pool queue for this scan type.
   
   So by passing the hints to any plugin involved in scan execution we can change runtime config to respond to those hints in different ways over time (using feedback from metrics) including scan server selection.
   
   
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881724108


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ScanAttemptsImpl.class);
+
+  static class ScanAttemptImpl
+      implements org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt {

Review Comment:
   Addressed in 5c039df



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881782493


##########
test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test;
+
+import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase.ConsistencyLevel;
+import org.apache.accumulo.core.conf.ClientProperty;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.harness.MiniClusterConfigurationCallback;
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ReadWriteIT;
+import org.apache.zookeeper.KeeperException;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import com.google.common.collect.Iterables;
+
+@Tag(MINI_CLUSTER_ONLY)
+public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase {
+
+  private static class ScanServerConcurrentTabletScanITConfiguration
+      implements MiniClusterConfigurationCallback {
+
+    @Override
+    public void configureMiniCluster(MiniAccumuloConfigImpl cfg,
+        org.apache.hadoop.conf.Configuration coreSite) {
+      cfg.setNumScanServers(1);
+      cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "3s");
+      cfg.setProperty(Property.SSERV_MINTHREADS, "4");
+    }
+  }
+
+  @BeforeAll
+  public static void start() throws Exception {
+    ScanServerConcurrentTabletScanITConfiguration c =
+        new ScanServerConcurrentTabletScanITConfiguration();
+    SharedMiniClusterBase.startMiniClusterWithConfig(c);
+  }
+
+  @AfterAll
+  public static void stop() throws Exception {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private void startScanServer(boolean cacheEnabled)
+      throws IOException, KeeperException, InterruptedException {
+
+    String zooRoot = getCluster().getServerContext().getZooKeeperRoot();
+    ZooReaderWriter zrw = getCluster().getServerContext().getZooReaderWriter();
+    String scanServerRoot = zooRoot + Constants.ZSSERVERS;
+
+    SharedMiniClusterBase.getCluster().getClusterControl().stop(ServerType.SCAN_SERVER);
+
+    Map<String,String> overrides = new HashMap<>();
+    overrides.put(Property.SSERV_CACHED_TABLET_METADATA_EXPIRATION.getKey(),
+        cacheEnabled ? "300m" : "0m");
+    SharedMiniClusterBase.getCluster().getClusterControl().start(ServerType.SCAN_SERVER, overrides,
+        1);
+    while (zrw.getChildren(scanServerRoot).size() == 0) {
+      Thread.sleep(500);
+    }
+
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheEnabled() throws Exception {
+
+    startScanServer(true);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1000 k/v because the tablet metadata is cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);
+
+      int count2 = 0;
+      while (iter2.hasNext()) {
+        iter2.next();
+        count2++;
+      }
+      assertEquals(1000, count2);
+
+      scanner1.close();
+
+      try (Scanner scanner2 = client.createScanner(tableName, Authorizations.EMPTY)) {
+        assertEquals(1100, Iterables.size(scanner2));
+      }
+    }
+  }
+
+  @Test
+  public void testScanSameTabletDifferentDataTabletMetadataCacheDisabled() throws Exception {
+
+    startScanServer(false);
+
+    Properties clientProperties = getClientProps();
+    clientProperties.put(ClientProperty.SCANNER_BATCH_SIZE.getKey(), "100");
+
+    try (AccumuloClient client = Accumulo.newClient().from(clientProperties).build()) {
+      String tableName = getUniqueNames(1)[0];
+
+      client.tableOperations().create(tableName);
+
+      // Load 1000 k/v
+      ReadWriteIT.ingest(client, 10, 100, 50, 0, "COLA", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      Scanner scanner1 = client.createScanner(tableName, Authorizations.EMPTY);
+      scanner1.setRange(new Range());
+      scanner1.setBatchSize(100);
+      scanner1.setReadaheadThreshold(0);
+      scanner1.setConsistencyLevel(ConsistencyLevel.EVENTUAL);
+
+      // iter1 should read 1000 k/v
+      Iterator<Entry<Key,Value>> iter1 = scanner1.iterator();
+
+      // Partially read the data and then start a 2nd scan
+      int count1 = 0;
+      while (iter1.hasNext() && count1 < 10) {
+        iter1.next();
+        count1++;
+      }
+
+      // Load another 100 k/v
+      ReadWriteIT.ingest(client, 10, 10, 50, 0, "COLB", tableName);
+      client.tableOperations().flush(tableName, null, null, true);
+
+      // iter2 should read 1100 k/v because the tablet metadata is not cached.
+      Iterator<Entry<Key,Value>> iter2 = scanner1.iterator();
+
+      while (iter1.hasNext()) {
+        iter1.next();
+        count1++;
+      }
+      assertEquals(1000, count1);

Review Comment:
   I don't think this makes sense after my changes in fb4f193



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] milleruntime commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
milleruntime commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1138536523

   > It's weird to ask for eventual consistency. Nobody wants eventual consistency. Eventual consistency is always tolerated, never desired.
   
   Eventual consistency for reads is desired in this case, but perhaps it isn't the best name. The user is willing to sacrifice the liveness of reads for increased availability of the tservers. We could call it Strong Eventual Consistency, but I think that is used mostly for writes https://en.wikipedia.org/wiki/Eventual_consistency#Strong_eventual_consistency. Maybe we could call the traditional scans "Live Scans", like we do with Live Ingest. And then calling the new scans "Cached Scans" or something.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r881548659


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {

Review Comment:
   class javadoc added in ec7cbad



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r884060098


##########
assemble/conf/accumulo-env.sh:
##########
@@ -94,6 +94,7 @@ case "$cmd" in
   tserver) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx768m' '-Xms768m') ;;
   compaction-coordinator) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx512m' '-Xms512m') ;;
   compactor) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx256m' '-Xms256m') ;;
+  sserver) JAVA_OPTS=("${JAVA_OPTS[@]}" '-Xmx512m' '-Xms512m') ;;

Review Comment:
   When testing running scan servers in Kubernetes using this [configuration](https://github.com/keith-turner/accumulo-testing/blob/322b32c62113b974b9dd18404d0e57902dc13b8e/sstest/accumulo-scanservers.yaml#L92) I had to make the following change inorder to be able to set the memory in the deployment descriptor.
   
   ```suggestion
     sserver) JAVA_OPTS=( '-Xmx512m' '-Xms512m' "${JAVA_OPTS[@]}") ;;
   ```
   
   I don't think we should apply this change in this branch. Thinking if the change makes sense that it should be applied to all the server config as its own PR.  That is why I opened #2744.  Just wanted to comment here in case anyone is running Kubernetes tests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
keith-turner commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r866146866


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ScanAttemptsImpl.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.clientImpl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.accumulo.core.data.TabletId;
+import org.apache.accumulo.core.spi.scan.ScanServerDispatcher.ScanAttempt;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+
+public class ScanAttemptsImpl {

Review Comment:
   I willl add some class docs for that.  May be a a few days I am working on testing this branch ATM.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939132068


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -153,6 +162,56 @@ private static <T> Supplier<T> memoizeWithExpiration(Supplier<T> s) {
     return () -> Suppliers.memoizeWithExpiration(s::get, 100, MILLISECONDS).get();
   }
 
+  private ScanServerSelector createScanServerSelector() {
+    String clazz = ClientProperty.SCAN_SERVER_SELECTOR.getValue(info.getProperties());
+    try {
+      Class<? extends ScanServerSelector> impl =
+          Class.forName(clazz).asSubclass(ScanServerSelector.class);
+      ScanServerSelector scanServerSelector = impl.getDeclaredConstructor().newInstance();
+
+      Map<String,String> sserverProps = new HashMap<>();
+      ClientProperty
+          .getPrefix(info.getProperties(), ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey())
+          .forEach((k, v) -> {
+            sserverProps.put(
+                k.toString()
+                    .substring(ClientProperty.SCAN_SERVER_SELECTOR_OPTS_PREFIX.getKey().length()),
+                v.toString());
+          });
+
+      scanServerSelector.init(new ScanServerSelector.InitParameters() {
+        @Override
+        public Map<String,String> getOptions() {
+          return Collections.unmodifiableMap(sserverProps);
+        }
+
+        @Override
+        public ServiceEnvironment getServiceEnv() {
+          return new ClientServiceEnvironmentImpl(ClientContext.this);
+        }
+
+        @Override
+        public Supplier<Collection<ScanServer>> getScanServers() {
+          return () -> ClientContext.this.getScanServers().entrySet().stream()
+              .map(entry -> new ScanServer() {
+                @Override
+                public String getAddress() {
+                  return entry.getKey();
+                }
+
+                @Override
+                public String getGroup() {
+                  return entry.getValue().group;
+                }
+              }).collect(Collectors.toSet());
+        }
+      });
+      return scanServerSelector;
+    } catch (Exception e) {

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939132929


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -332,6 +392,51 @@ public synchronized BatchWriterConfig getBatchWriterConfig() {
     return batchWriterConfig;
   }
 
+  public static class ScanServerInfo {
+    public final UUID uuid;
+    public final String group;
+
+    public ScanServerInfo(UUID uuid, String group) {
+      this.uuid = uuid;
+      this.group = group;
+    }
+
+  }
+
+  /**
+   * @return map of live scan server addresses to lock uuids.
+   */
+  public Map<String,ScanServerInfo> getScanServers() {
+    Map<String,ScanServerInfo> liveScanServers = new HashMap<>();
+    String root = this.getZooKeeperRoot() + Constants.ZSSERVERS;
+    var addrs = this.getZooCache().getChildren(root);
+    for (String addr : addrs) {
+      try {
+        final var zLockPath = ServiceLock.path(root + "/" + addr);
+        ZcStat stat = new ZcStat();
+        byte[] lockData = ServiceLock.getLockData(getZooCache(), zLockPath, stat);
+        if (lockData != null) {
+          String[] fields = new String(lockData, UTF_8).split(",", 2);
+          UUID uuid = UUID.fromString(fields[0]);
+          String group = fields[1];
+          liveScanServers.put(addr, new ScanServerInfo(uuid, group));
+        }
+      } catch (Exception e) {
+        log.error("Error validating zookeeper scan server node: " + addr, e);

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939130049


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/InstanceOperationsImpl.java:
##########
@@ -136,6 +136,11 @@ public List<String> getManagerLocations() {
     return context.getManagerLocations();
   }
 
+  @Override
+  public List<String> getScanServers() {
+    return List.copyOf(context.getScanServers().keySet());
+  }

Review Comment:
   Resolved in f437fed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] milleruntime commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
milleruntime commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939001697


##########
core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java:
##########
@@ -373,6 +384,23 @@ default void forEach(BiConsumer<? super Key,? super Value> keyValueConsumer) {
     }
   }
 
+  /**
+   * Get the configured consistency level
+   *
+   * @return consistency level
+   * @since 2.1.0
+   */
+  public ConsistencyLevel getConsistencyLevel();
+
+  /**
+   * Set the desired consistency level for this scanner.
+   *
+   * @param level
+   *          consistency level
+   * @since 2.1.0
+   */
+  public void setConsistencyLevel(ConsistencyLevel level);
+

Review Comment:
   Using the term "snapshot" goes nicely with the new class `SnapshotTablet`. I think this is beneficial since our users tend to be developers as well and will probably see this new class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
dlmarion commented on code in PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#discussion_r939148137


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -266,4 +273,71 @@ public Stream<ExternalCompactionFinalState> getExternalCompactionFinalStates() {
       throw new RuntimeException(e);
     }
   }
+
+  @Override
+  public void putScanServerFileReferences(Collection<ScanServerRefTabletFile> scanRefs) {
+    try (BatchWriter writer = context.createBatchWriter(DataLevel.USER.metaTable())) {
+      String prefix = ScanServerFileReferenceSection.getRowPrefix();
+      for (ScanServerRefTabletFile ref : scanRefs) {
+        Mutation m = new Mutation(prefix + ref.getRowSuffix());
+        m.put(ref.getServerAddress(), ref.getServerLockUUID(), ref.getValue());
+        writer.addMutation(m);
+      }
+    } catch (MutationsRejectedException | TableNotFoundException e) {
+      throw new RuntimeException(e);

Review Comment:
   Created #2853



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #2665: Eventually Consistent scans / ScanServer feature

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on PR #2665:
URL: https://github.com/apache/accumulo/pull/2665#issuecomment-1208358218

   I haven't looked at the changes since my last PR, but feel free to merge... I can look at them later. I saw a bunch of new issues, but they are all over the place, and it's hard for me to track those all. After this is merged, I'll probably go line by line through my comments here to ensure that each of them have been done, or I'll do them as a follow on. If I do that, I'm not going to track down each of the separate issues that were created, though... I'm just going to do one PR to fix what I think needs fixing (if it hasn't already been done by the time I take a look).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org